[NO ISSUE][CLUS] Prevent Possible Deadlock in GlobalRecoveryManager
- user model changes: no
- storage format changes: no
- interface changes: no
Details:
- Perform global recovery on a different thread to prevent
possible deadlocks in ClusterStateManager.
- Shutdown the system in case of global recovery failure
since the system will be in inconsistent state.
Change-Id: I3b0491a84fb24b428d5ce98f392adafc6dfacff9
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2072
Sonar-Qube: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Ian Maxon <imaxon@apache.org>
Reviewed-by: Michael Blow <mblow@apache.org>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
index 13f3afa..d6854b12 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
@@ -48,6 +48,8 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.control.nc.NCShutdownHook;
+import org.apache.hyracks.util.ExitUtil;
public class GlobalRecoveryManager implements IGlobalRecoveryManager {
@@ -56,6 +58,7 @@
protected final ICCServiceContext serviceCtx;
protected IHyracksClientConnection hcc;
protected volatile boolean recoveryCompleted;
+ protected volatile boolean recovering;
public GlobalRecoveryManager(ICCServiceContext serviceCtx, IHyracksClientConnection hcc,
IStorageComponentProvider componentProvider) {
@@ -81,41 +84,42 @@
}
@Override
- public void startGlobalRecovery(ICcApplicationContext appCtx) throws HyracksDataException {
- if (!recoveryCompleted) {
- recover(appCtx);
+ public void startGlobalRecovery(ICcApplicationContext appCtx) {
+ if (!recoveryCompleted && !recovering) {
+ synchronized (this) {
+ if (!recovering) {
+ recovering = true;
+ /**
+ * Perform recovery on a different thread to avoid deadlocks in
+ * {@link org.apache.asterix.common.cluster.IClusterStateManager}
+ */
+ serviceCtx.getControllerService().getExecutor().submit(() -> {
+ try {
+ recover(appCtx);
+ } catch (HyracksDataException e) {
+ LOGGER.log(Level.SEVERE, "Global recovery failed. Shutting down...", e);
+ ExitUtil.exit(NCShutdownHook.FAILED_TO_RECOVER_EXIT_CODE);
+ }
+ });
+ }
+ }
}
}
protected void recover(ICcApplicationContext appCtx) throws HyracksDataException {
- LOGGER.info("Starting Global Recovery");
- MetadataTransactionContext mdTxnCtx = null;
try {
+ LOGGER.info("Starting Global Recovery");
MetadataManager.INSTANCE.init();
- mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
mdTxnCtx = doRecovery(appCtx, mdTxnCtx);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ recoveryCompleted = true;
+ recovering = false;
+ LOGGER.info("Global Recovery Completed. Refreshing cluster state...");
+ appCtx.getClusterStateManager().refreshState();
} catch (Exception e) {
- // This needs to be fixed <-- Needs to shutdown the system -->
- /*
- * Note: Throwing this illegal state exception will terminate this thread
- * and feeds listeners will not be notified.
- */
- LOGGER.log(Level.SEVERE, "Global recovery was not completed successfully: ", e);
- if (mdTxnCtx != null) {
- try {
- MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
- } catch (Exception e1) {
- LOGGER.log(Level.SEVERE, "Exception aborting metadata transaction", e1);
- e.addSuppressed(e1);
- throw new IllegalStateException(e);
- }
- }
throw HyracksDataException.create(e);
}
- recoveryCompleted = true;
- LOGGER.info("Global Recovery Completed");
- appCtx.getClusterStateManager().refreshState();
}
protected MetadataTransactionContext doRecovery(ICcApplicationContext appCtx, MetadataTransactionContext mdTxnCtx)
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryManager.java
index b4559c8..a3add90 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryManager.java
@@ -20,19 +20,15 @@
import org.apache.asterix.common.api.IClusterEventsSubscriber;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface IGlobalRecoveryManager extends IClusterEventsSubscriber {
/**
* Starts the global recovery process after the cluster state has changed to ACTIVE.
*
- * @param appCtx
- * the application context
- * @throws HyracksDataException
- * if the global recovery fails
+ * @param appCtx the application context
*/
- void startGlobalRecovery(ICcApplicationContext appCtx) throws HyracksDataException;
+ void startGlobalRecovery(ICcApplicationContext appCtx);
/**
* @return true, if global recovery has been completed successfully
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NCShutdownHook.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NCShutdownHook.java
index 0a02635..6308373 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NCShutdownHook.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NCShutdownHook.java
@@ -31,6 +31,7 @@
public class NCShutdownHook extends Thread {
public static final int FAILED_TO_STARTUP_EXIT_CODE = 2;
+ public static final int FAILED_TO_RECOVER_EXIT_CODE = 3;
private static final Logger LOGGER = Logger.getLogger(NCShutdownHook.class.getName());
private static final long SHUTDOWN_WAIT_TIME = 10 * 60 * 1000L;
private final Thread watchDog;