You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org> on 2018/01/07 09:31:21 UTC

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Murtadha Hubail has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/2266

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................

[ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator

- user model changes: no
- storage format changes: no
- interface changes: yes
  - Rename IFaultToleranceStrategy -> INcLifecycleCoordinator

Details:
- Rename IFaultToleranceStrategy since it doesn't provide
  fault tolerance anymore but rather coordinates node
  lifecycle.
- Make INcLifecycleCoordinator extensible.
- Remove unneeded FaultToleranceStrategyFactory.

Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
D asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/FaultToleranceStrategyFactory.java
R asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
R asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INcLifecycleCoordinator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
12 files changed, 52 insertions(+), 110 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/66/2266/1

diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
index 38ee10c..d239038 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
@@ -169,7 +169,7 @@
 
     private void processMetadataNode(IServletRequest request, IServletResponse response) throws HyracksDataException {
         final String node = request.getParameter("node");
-        appCtx.getFaultToleranceStrategy().notifyMetadataNodeChange(node);
+        appCtx.getNcLifecycleCoordinator().notifyMetadataNodeChange(node);
         response.setStatus(HttpResponseStatus.OK);
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/FaultToleranceStrategyFactory.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/FaultToleranceStrategyFactory.java
deleted file mode 100644
index 5a715d6..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/FaultToleranceStrategyFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.app.replication;
-
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
-import org.apache.hyracks.api.application.ICCServiceContext;
-
-public class FaultToleranceStrategyFactory {
-
-    private FaultToleranceStrategyFactory() {
-        throw new AssertionError();
-    }
-
-    public static IFaultToleranceStrategy create(ICCServiceContext serviceCtx, boolean replicationEnabled) {
-        Class<? extends IFaultToleranceStrategy> clazz = NoFaultToleranceStrategy.class;
-        try {
-            return clazz.newInstance().from(serviceCtx, replicationEnabled);
-        } catch (InstantiationException | IllegalAccessException e) {
-            throw new IllegalStateException(e);
-        }
-    }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
similarity index 89%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
index 49f4b32..54b1a01 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
@@ -44,8 +44,8 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.messaging.api.ICCMessageBroker;
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
 import org.apache.asterix.common.replication.INCLifecycleMessage;
+import org.apache.asterix.common.replication.INcLifecycleCoordinator;
 import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.hyracks.api.application.ICCServiceContext;
@@ -55,17 +55,22 @@
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-public class NoFaultToleranceStrategy implements IFaultToleranceStrategy {
+public class NcLifecycleCoordinator implements INcLifecycleCoordinator {
 
     private static final Logger LOGGER = LogManager.getLogger();
-    private IClusterStateManager clusterManager;
-    private String metadataNodeId;
-    private Set<String> pendingStartupCompletionNodes = new HashSet<>();
-    private ICCMessageBroker messageBroker;
-    private boolean replicationEnabled;
+    protected IClusterStateManager clusterManager;
+    protected String metadataNodeId;
+    protected Set<String> pendingStartupCompletionNodes = new HashSet<>();
+    protected final ICCMessageBroker messageBroker;
+    private final boolean replicationEnabled;
+
+    public NcLifecycleCoordinator(ICCServiceContext serviceCtx, boolean replicationEnabled) {
+        this.messageBroker = (ICCMessageBroker) serviceCtx.getMessageBroker();
+        this.replicationEnabled = replicationEnabled;
+    }
 
     @Override
-    public void notifyNodeJoin(String nodeId) throws HyracksDataException {
+    public void notifyNodeJoin(String nodeId) {
         pendingStartupCompletionNodes.add(nodeId);
     }
 
@@ -94,14 +99,6 @@
             default:
                 throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MESSAGE_TYPE, message.getType().name());
         }
-    }
-
-    @Override
-    public IFaultToleranceStrategy from(ICCServiceContext serviceCtx, boolean replicationEnabled) {
-        NoFaultToleranceStrategy ft = new NoFaultToleranceStrategy();
-        ft.messageBroker = (ICCMessageBroker) serviceCtx.getMessageBroker();
-        ft.replicationEnabled = replicationEnabled;
-        return ft;
     }
 
     @Override
@@ -136,7 +133,7 @@
         }
     }
 
-    private List<INCLifecycleTask> buildNCRegTasks(String nodeId, NodeStatus nodeStatus, SystemState state) {
+    protected List<INCLifecycleTask> buildNCRegTasks(String nodeId, NodeStatus nodeStatus, SystemState state) {
         LOGGER.log(Level.INFO, () -> "Building registration tasks for node: " + nodeId + " with state: " + state);
         final boolean isMetadataNode = nodeId.equals(metadataNodeId);
         if (nodeStatus == NodeStatus.ACTIVE) {
@@ -170,7 +167,7 @@
         return tasks;
     }
 
-    private List<INCLifecycleTask> buildActiveNCRegTasks(boolean metadataNode) {
+    protected List<INCLifecycleTask> buildActiveNCRegTasks(boolean metadataNode) {
         final List<INCLifecycleTask> tasks = new ArrayList<>();
         if (metadataNode) {
             // need to unbind from old distributed state then rebind to new one
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java
index ebde9b9..815d878 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java
@@ -41,7 +41,7 @@
 
     @Override
     public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        ((CcApplicationContext) appCtx).getFaultToleranceStrategy().process(this);
+        ((CcApplicationContext) appCtx).getNcLifecycleCoordinator().process(this);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
index b654fd8..5ac3a0c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
@@ -37,7 +37,7 @@
 
     @Override
     public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        appCtx.getFaultToleranceStrategy().process(this);
+        appCtx.getNcLifecycleCoordinator().process(this);
     }
 
     public String getNodeId() {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java
index b60ef8b..6ca576a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java
@@ -58,7 +58,7 @@
 
     @Override
     public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        appCtx.getFaultToleranceStrategy().process(this);
+        appCtx.getNcLifecycleCoordinator().process(this);
     }
 
     public SystemState getState() {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index c2a0fc1..11f4e1c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -55,7 +55,7 @@
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.cc.CCExtensionManager;
 import org.apache.asterix.app.external.ExternalLibraryUtils;
-import org.apache.asterix.app.replication.FaultToleranceStrategyFactory;
+import org.apache.asterix.app.replication.NcLifecycleCoordinator;
 import org.apache.asterix.common.api.AsterixThreadFactory;
 import org.apache.asterix.common.api.IClusterManagementWork;
 import org.apache.asterix.common.api.INodeJobTracker;
@@ -67,9 +67,7 @@
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.library.ILibraryManager;
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
-import org.apache.asterix.common.replication.IReplicationStrategy;
-import org.apache.asterix.common.replication.ReplicationStrategyFactory;
+import org.apache.asterix.common.replication.INcLifecycleCoordinator;
 import org.apache.asterix.common.utils.Servlets;
 import org.apache.asterix.external.library.ExternalLibraryManager;
 import org.apache.asterix.file.StorageComponentProvider;
@@ -142,13 +140,12 @@
         ILibraryManager libraryManager = new ExternalLibraryManager();
         ReplicationProperties repProp = new ReplicationProperties(
                 PropertiesAccessor.getInstance(ccServiceCtx.getAppConfig()));
-        IFaultToleranceStrategy ftStrategy =
-                FaultToleranceStrategyFactory.create(ccServiceCtx, repProp.isReplicationEnabled());
+        INcLifecycleCoordinator lifecycleCoordinator = createNcLifeCycleCoordinator(repProp.isReplicationEnabled());
         ExternalLibraryUtils.setUpExternaLibraries(libraryManager, false);
         componentProvider = new StorageComponentProvider();
         GlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
         statementExecutorCtx = new StatementExecutorContext();
-        appCtx = createApplicationContext(libraryManager, globalRecoveryManager, ftStrategy);
+        appCtx = createApplicationContext(libraryManager, globalRecoveryManager, lifecycleCoordinator);
         List<AsterixExtension> extensions = new ArrayList<>();
         extensions.addAll(this.getExtensions());
         ccExtensionManager = new CCExtensionManager(extensions);
@@ -177,10 +174,10 @@
     }
 
     protected ICcApplicationContext createApplicationContext(ILibraryManager libraryManager,
-            GlobalRecoveryManager globalRecoveryManager, IFaultToleranceStrategy ftStrategy)
+            GlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator lifecycleCoordinator)
             throws AlgebricksException, IOException {
         return new CcApplicationContext(ccServiceCtx, getHcc(), libraryManager, () -> MetadataManager.INSTANCE,
-                globalRecoveryManager, ftStrategy, new ActiveNotificationHandler(), componentProvider,
+                globalRecoveryManager, lifecycleCoordinator, new ActiveNotificationHandler(), componentProvider,
                 new MetadataLockManager());
     }
 
@@ -188,6 +185,10 @@
         return new GlobalRecoveryManager(ccServiceCtx, getHcc(), componentProvider);
     }
 
+    protected INcLifecycleCoordinator createNcLifeCycleCoordinator(boolean replicationEnabled) {
+        return new NcLifecycleCoordinator(ccServiceCtx, replicationEnabled);
+    }
+
     @Override
     protected void configureLoggingLevel(Level level) {
         super.configureLoggingLevel(level);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
index 07d3584..ca314aa 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
@@ -24,7 +24,7 @@
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.asterix.app.replication.NoFaultToleranceStrategy;
+import org.apache.asterix.app.replication.NcLifecycleCoordinator;
 import org.apache.asterix.app.replication.message.NCLifecycleTaskReportMessage;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.cluster.ClusterPartition;
@@ -196,7 +196,7 @@
             throws HyracksDataException {
         NCLifecycleTaskReportMessage msg = new NCLifecycleTaskReportMessage(nodeId, true);
         applicationContext.getResourceIdManager().report(nodeId, 0);
-        applicationContext.getFaultToleranceStrategy().process(msg);
+        applicationContext.getNcLifecycleCoordinator().process(msg);
     }
 
     private CcApplicationContext ccAppContext(ClusterStateManager csm) throws HyracksDataException {
@@ -207,9 +207,9 @@
         Mockito.when(iccServiceContext.getAppConfig()).thenReturn(applicationConfig);
         Mockito.when(ccApplicationContext.getServiceContext()).thenReturn(iccServiceContext);
 
-        NoFaultToleranceStrategy fts = new NoFaultToleranceStrategy();
-        fts.bindTo(csm);
-        Mockito.when(ccApplicationContext.getFaultToleranceStrategy()).thenReturn(fts);
+        NcLifecycleCoordinator coordinator = new NcLifecycleCoordinator(ccApplicationContext.getServiceContext(), false);
+        coordinator.bindTo(csm);
+        Mockito.when(ccApplicationContext.getNcLifecycleCoordinator()).thenReturn(coordinator);
 
         MetadataProperties metadataProperties = mockMetadataProperties();
         Mockito.when(ccApplicationContext.getMetadataProperties()).thenReturn(metadataProperties);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
index 6181ade..e02482d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
@@ -26,7 +26,7 @@
 import org.apache.asterix.common.config.ExtensionProperties;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.metadata.IMetadataBootstrap;
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
+import org.apache.asterix.common.replication.INcLifecycleCoordinator;
 import org.apache.asterix.common.transactions.IResourceIdManager;
 import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.job.IJobLifecycleListener;
@@ -62,9 +62,9 @@
     IGlobalRecoveryManager getGlobalRecoveryManager();
 
     /**
-     * @return the fault tolerance strategy in use for the cluster
+     * @return the NC lifecycle coordinator in use for the cluster
      */
-    IFaultToleranceStrategy getFaultToleranceStrategy();
+    INcLifecycleCoordinator getNcLifecycleCoordinator();
 
     /**
      * @return the active notification handler at Cluster controller
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IFaultToleranceStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INcLifecycleCoordinator.java
similarity index 70%
rename from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IFaultToleranceStrategy.java
rename to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INcLifecycleCoordinator.java
index bf4cac9..1a7c3c8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IFaultToleranceStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INcLifecycleCoordinator.java
@@ -19,13 +19,12 @@
 package org.apache.asterix.common.replication;
 
 import org.apache.asterix.common.cluster.IClusterStateManager;
-import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public interface IFaultToleranceStrategy {
+public interface INcLifecycleCoordinator {
 
     /**
-     * Defines the logic of a {@link IFaultToleranceStrategy} when a node joins the cluster.
+     * Defines the logic of a {@link INcLifecycleCoordinator} when a node joins the cluster.
      *
      * @param nodeId
      * @throws HyracksDataException
@@ -33,7 +32,7 @@
     void notifyNodeJoin(String nodeId) throws HyracksDataException;
 
     /**
-     * Defines the logic of a {@link IFaultToleranceStrategy} when a node leaves the cluster.
+     * Defines the logic of a {@link INcLifecycleCoordinator} when a node leaves the cluster.
      *
      * @param nodeId
      * @throws HyracksDataException
@@ -41,7 +40,7 @@
     void notifyNodeFailure(String nodeId) throws HyracksDataException;
 
     /**
-     * Binds the fault tolerance strategy to {@code cluserManager}.
+     * Binds the coordinator to {@code cluserManager}.
      *
      * @param clusterManager
      */
@@ -56,20 +55,9 @@
     void process(INCLifecycleMessage message) throws HyracksDataException;
 
     /**
-     * Constructs a fault tolerance strategy.
-     *
-     * @param serviceCtx
-     * @param replicationEnabled
-     * @return the fault tolerance strategy
-     */
-    IFaultToleranceStrategy from(ICCServiceContext serviceCtx, boolean replicationEnabled);
-
-    /**
      * Performs the required steps to change the metadata node to {@code node}
      *
      * @param node
      */
-    default void notifyMetadataNodeChange(String node) throws HyracksDataException {
-        throw new UnsupportedOperationException(getClass() + " does not support metadata node change");
-    }
+    void notifyMetadataNodeChange(String node) throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
index f991cd4..d8f14a2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
@@ -42,7 +42,7 @@
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.metadata.IMetadataBootstrap;
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
+import org.apache.asterix.common.replication.INcLifecycleCoordinator;
 import org.apache.asterix.common.transactions.IResourceIdManager;
 import org.apache.asterix.runtime.job.listener.NodeJobTracker;
 import org.apache.asterix.runtime.transaction.ResourceIdManager;
@@ -80,7 +80,7 @@
     private Supplier<IMetadataBootstrap> metadataBootstrapSupplier;
     private IHyracksClientConnection hcc;
     private Object extensionManager;
-    private IFaultToleranceStrategy ftStrategy;
+    private INcLifecycleCoordinator ftStrategy;
     private IJobLifecycleListener activeLifeCycleListener;
     private IMetadataLockManager mdLockManager;
     private IClusterStateManager clusterStateManager;
@@ -88,7 +88,7 @@
 
     public CcApplicationContext(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
             ILibraryManager libraryManager, Supplier<IMetadataBootstrap> metadataBootstrapSupplier,
-            IGlobalRecoveryManager globalRecoveryManager, IFaultToleranceStrategy ftStrategy,
+            IGlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator ftStrategy,
             IJobLifecycleListener activeLifeCycleListener, IStorageComponentProvider storageComponentProvider,
             IMetadataLockManager mdLockManager) throws AlgebricksException, IOException {
         this.ccServiceCtx = ccServiceCtx;
@@ -232,7 +232,7 @@
     }
 
     @Override
-    public IFaultToleranceStrategy getFaultToleranceStrategy() {
+    public INcLifecycleCoordinator getNcLifecycleCoordinator() {
         return ftStrategy;
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
index c07200a..6994862 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
@@ -23,7 +23,6 @@
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
@@ -35,7 +34,7 @@
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
+import org.apache.asterix.common.replication.INcLifecycleCoordinator;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.transactions.IResourceIdManager;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
@@ -60,11 +59,6 @@
  */
 
 public class ClusterStateManager implements IClusterStateManager {
-    /*
-     * TODO: currently after instance restarts we require all nodes to join again,
-     * otherwise the cluster wont be ACTIVE. we may overcome this by storing the cluster state before the instance
-     * shutdown and using it on startup to identify the nodes that are expected the join.
-     */
 
     private static final Logger LOGGER = LogManager.getLogger();
     private final Map<String, Map<IOption, Object>> ncConfigMap = new HashMap<>();
@@ -77,7 +71,7 @@
     private boolean metadataNodeActive = false;
     private Set<String> failedNodes = new HashSet<>();
     private Set<String> participantNodes = new HashSet<>();
-    private IFaultToleranceStrategy ftStrategy;
+    private INcLifecycleCoordinator lifecycleCoordinator;
     private ICcApplicationContext appCtx;
 
     @Override
@@ -86,8 +80,8 @@
         node2PartitionsMap = appCtx.getMetadataProperties().getNodePartitions();
         clusterPartitions = appCtx.getMetadataProperties().getClusterPartitions();
         currentMetadataNode = appCtx.getMetadataProperties().getMetadataNodeName();
-        ftStrategy = appCtx.getFaultToleranceStrategy();
-        ftStrategy.bindTo(this);
+        lifecycleCoordinator = appCtx.getNcLifecycleCoordinator();
+        lifecycleCoordinator.bindTo(this);
     }
 
     @Override
@@ -98,7 +92,7 @@
         failedNodes.add(nodeId);
         ncConfigMap.remove(nodeId);
         pendingRemoval.remove(nodeId);
-        ftStrategy.notifyNodeFailure(nodeId);
+        lifecycleCoordinator.notifyNodeFailure(nodeId);
     }
 
     @Override
@@ -109,7 +103,7 @@
         failedNodes.remove(nodeId);
         ncConfigMap.put(nodeId, configuration);
         updateNodeConfig(nodeId, configuration);
-        ftStrategy.notifyNodeJoin(nodeId);
+        lifecycleCoordinator.notifyNodeJoin(nodeId);
     }
 
     @Override

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/243/ (4/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/2465/ (12/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/2792/ (2/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-rebalance-cancellation/1366/ (6/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Murtadha Hubail has submitted this change and it was merged.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


[ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator

- user model changes: no
- storage format changes: no
- interface changes: yes
  - Rename IFaultToleranceStrategy -> INcLifecycleCoordinator

Details:
- Rename IFaultToleranceStrategy since it doesn't provide
  fault tolerance anymore but rather coordinates node
  lifecycle.
- Make INcLifecycleCoordinator extensible.
- Remove unneeded FaultToleranceStrategyFactory.

Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2266
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
D asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/FaultToleranceStrategyFactory.java
R asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
R asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INcLifecycleCoordinator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
12 files changed, 52 insertions(+), 110 deletions(-)

Approvals:
  Anon. E. Moose #1000171: 
  Till Westmann: Looks good to me, approved
  Jenkins: Verified; No violations found; ; Verified



diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
index 38ee10c..d239038 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
@@ -169,7 +169,7 @@
 
     private void processMetadataNode(IServletRequest request, IServletResponse response) throws HyracksDataException {
         final String node = request.getParameter("node");
-        appCtx.getFaultToleranceStrategy().notifyMetadataNodeChange(node);
+        appCtx.getNcLifecycleCoordinator().notifyMetadataNodeChange(node);
         response.setStatus(HttpResponseStatus.OK);
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/FaultToleranceStrategyFactory.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/FaultToleranceStrategyFactory.java
deleted file mode 100644
index 5a715d6..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/FaultToleranceStrategyFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.app.replication;
-
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
-import org.apache.hyracks.api.application.ICCServiceContext;
-
-public class FaultToleranceStrategyFactory {
-
-    private FaultToleranceStrategyFactory() {
-        throw new AssertionError();
-    }
-
-    public static IFaultToleranceStrategy create(ICCServiceContext serviceCtx, boolean replicationEnabled) {
-        Class<? extends IFaultToleranceStrategy> clazz = NoFaultToleranceStrategy.class;
-        try {
-            return clazz.newInstance().from(serviceCtx, replicationEnabled);
-        } catch (InstantiationException | IllegalAccessException e) {
-            throw new IllegalStateException(e);
-        }
-    }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
similarity index 89%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
index 49f4b32..54b1a01 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
@@ -44,8 +44,8 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.messaging.api.ICCMessageBroker;
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
 import org.apache.asterix.common.replication.INCLifecycleMessage;
+import org.apache.asterix.common.replication.INcLifecycleCoordinator;
 import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.hyracks.api.application.ICCServiceContext;
@@ -55,17 +55,22 @@
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-public class NoFaultToleranceStrategy implements IFaultToleranceStrategy {
+public class NcLifecycleCoordinator implements INcLifecycleCoordinator {
 
     private static final Logger LOGGER = LogManager.getLogger();
-    private IClusterStateManager clusterManager;
-    private String metadataNodeId;
-    private Set<String> pendingStartupCompletionNodes = new HashSet<>();
-    private ICCMessageBroker messageBroker;
-    private boolean replicationEnabled;
+    protected IClusterStateManager clusterManager;
+    protected String metadataNodeId;
+    protected Set<String> pendingStartupCompletionNodes = new HashSet<>();
+    protected final ICCMessageBroker messageBroker;
+    private final boolean replicationEnabled;
+
+    public NcLifecycleCoordinator(ICCServiceContext serviceCtx, boolean replicationEnabled) {
+        this.messageBroker = (ICCMessageBroker) serviceCtx.getMessageBroker();
+        this.replicationEnabled = replicationEnabled;
+    }
 
     @Override
-    public void notifyNodeJoin(String nodeId) throws HyracksDataException {
+    public void notifyNodeJoin(String nodeId) {
         pendingStartupCompletionNodes.add(nodeId);
     }
 
@@ -94,14 +99,6 @@
             default:
                 throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MESSAGE_TYPE, message.getType().name());
         }
-    }
-
-    @Override
-    public IFaultToleranceStrategy from(ICCServiceContext serviceCtx, boolean replicationEnabled) {
-        NoFaultToleranceStrategy ft = new NoFaultToleranceStrategy();
-        ft.messageBroker = (ICCMessageBroker) serviceCtx.getMessageBroker();
-        ft.replicationEnabled = replicationEnabled;
-        return ft;
     }
 
     @Override
@@ -136,7 +133,7 @@
         }
     }
 
-    private List<INCLifecycleTask> buildNCRegTasks(String nodeId, NodeStatus nodeStatus, SystemState state) {
+    protected List<INCLifecycleTask> buildNCRegTasks(String nodeId, NodeStatus nodeStatus, SystemState state) {
         LOGGER.log(Level.INFO, () -> "Building registration tasks for node: " + nodeId + " with state: " + state);
         final boolean isMetadataNode = nodeId.equals(metadataNodeId);
         if (nodeStatus == NodeStatus.ACTIVE) {
@@ -170,7 +167,7 @@
         return tasks;
     }
 
-    private List<INCLifecycleTask> buildActiveNCRegTasks(boolean metadataNode) {
+    protected List<INCLifecycleTask> buildActiveNCRegTasks(boolean metadataNode) {
         final List<INCLifecycleTask> tasks = new ArrayList<>();
         if (metadataNode) {
             // need to unbind from old distributed state then rebind to new one
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java
index ebde9b9..815d878 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java
@@ -41,7 +41,7 @@
 
     @Override
     public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        ((CcApplicationContext) appCtx).getFaultToleranceStrategy().process(this);
+        ((CcApplicationContext) appCtx).getNcLifecycleCoordinator().process(this);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
index b654fd8..5ac3a0c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
@@ -37,7 +37,7 @@
 
     @Override
     public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        appCtx.getFaultToleranceStrategy().process(this);
+        appCtx.getNcLifecycleCoordinator().process(this);
     }
 
     public String getNodeId() {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java
index b60ef8b..6ca576a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java
@@ -58,7 +58,7 @@
 
     @Override
     public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        appCtx.getFaultToleranceStrategy().process(this);
+        appCtx.getNcLifecycleCoordinator().process(this);
     }
 
     public SystemState getState() {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index c2a0fc1..11f4e1c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -55,7 +55,7 @@
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.cc.CCExtensionManager;
 import org.apache.asterix.app.external.ExternalLibraryUtils;
-import org.apache.asterix.app.replication.FaultToleranceStrategyFactory;
+import org.apache.asterix.app.replication.NcLifecycleCoordinator;
 import org.apache.asterix.common.api.AsterixThreadFactory;
 import org.apache.asterix.common.api.IClusterManagementWork;
 import org.apache.asterix.common.api.INodeJobTracker;
@@ -67,9 +67,7 @@
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.library.ILibraryManager;
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
-import org.apache.asterix.common.replication.IReplicationStrategy;
-import org.apache.asterix.common.replication.ReplicationStrategyFactory;
+import org.apache.asterix.common.replication.INcLifecycleCoordinator;
 import org.apache.asterix.common.utils.Servlets;
 import org.apache.asterix.external.library.ExternalLibraryManager;
 import org.apache.asterix.file.StorageComponentProvider;
@@ -142,13 +140,12 @@
         ILibraryManager libraryManager = new ExternalLibraryManager();
         ReplicationProperties repProp = new ReplicationProperties(
                 PropertiesAccessor.getInstance(ccServiceCtx.getAppConfig()));
-        IFaultToleranceStrategy ftStrategy =
-                FaultToleranceStrategyFactory.create(ccServiceCtx, repProp.isReplicationEnabled());
+        INcLifecycleCoordinator lifecycleCoordinator = createNcLifeCycleCoordinator(repProp.isReplicationEnabled());
         ExternalLibraryUtils.setUpExternaLibraries(libraryManager, false);
         componentProvider = new StorageComponentProvider();
         GlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
         statementExecutorCtx = new StatementExecutorContext();
-        appCtx = createApplicationContext(libraryManager, globalRecoveryManager, ftStrategy);
+        appCtx = createApplicationContext(libraryManager, globalRecoveryManager, lifecycleCoordinator);
         List<AsterixExtension> extensions = new ArrayList<>();
         extensions.addAll(this.getExtensions());
         ccExtensionManager = new CCExtensionManager(extensions);
@@ -177,10 +174,10 @@
     }
 
     protected ICcApplicationContext createApplicationContext(ILibraryManager libraryManager,
-            GlobalRecoveryManager globalRecoveryManager, IFaultToleranceStrategy ftStrategy)
+            GlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator lifecycleCoordinator)
             throws AlgebricksException, IOException {
         return new CcApplicationContext(ccServiceCtx, getHcc(), libraryManager, () -> MetadataManager.INSTANCE,
-                globalRecoveryManager, ftStrategy, new ActiveNotificationHandler(), componentProvider,
+                globalRecoveryManager, lifecycleCoordinator, new ActiveNotificationHandler(), componentProvider,
                 new MetadataLockManager());
     }
 
@@ -188,6 +185,10 @@
         return new GlobalRecoveryManager(ccServiceCtx, getHcc(), componentProvider);
     }
 
+    protected INcLifecycleCoordinator createNcLifeCycleCoordinator(boolean replicationEnabled) {
+        return new NcLifecycleCoordinator(ccServiceCtx, replicationEnabled);
+    }
+
     @Override
     protected void configureLoggingLevel(Level level) {
         super.configureLoggingLevel(level);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
index 07d3584..ca314aa 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
@@ -24,7 +24,7 @@
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.asterix.app.replication.NoFaultToleranceStrategy;
+import org.apache.asterix.app.replication.NcLifecycleCoordinator;
 import org.apache.asterix.app.replication.message.NCLifecycleTaskReportMessage;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.cluster.ClusterPartition;
@@ -196,7 +196,7 @@
             throws HyracksDataException {
         NCLifecycleTaskReportMessage msg = new NCLifecycleTaskReportMessage(nodeId, true);
         applicationContext.getResourceIdManager().report(nodeId, 0);
-        applicationContext.getFaultToleranceStrategy().process(msg);
+        applicationContext.getNcLifecycleCoordinator().process(msg);
     }
 
     private CcApplicationContext ccAppContext(ClusterStateManager csm) throws HyracksDataException {
@@ -207,9 +207,9 @@
         Mockito.when(iccServiceContext.getAppConfig()).thenReturn(applicationConfig);
         Mockito.when(ccApplicationContext.getServiceContext()).thenReturn(iccServiceContext);
 
-        NoFaultToleranceStrategy fts = new NoFaultToleranceStrategy();
-        fts.bindTo(csm);
-        Mockito.when(ccApplicationContext.getFaultToleranceStrategy()).thenReturn(fts);
+        NcLifecycleCoordinator coordinator = new NcLifecycleCoordinator(ccApplicationContext.getServiceContext(), false);
+        coordinator.bindTo(csm);
+        Mockito.when(ccApplicationContext.getNcLifecycleCoordinator()).thenReturn(coordinator);
 
         MetadataProperties metadataProperties = mockMetadataProperties();
         Mockito.when(ccApplicationContext.getMetadataProperties()).thenReturn(metadataProperties);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
index 6181ade..e02482d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
@@ -26,7 +26,7 @@
 import org.apache.asterix.common.config.ExtensionProperties;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.metadata.IMetadataBootstrap;
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
+import org.apache.asterix.common.replication.INcLifecycleCoordinator;
 import org.apache.asterix.common.transactions.IResourceIdManager;
 import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.job.IJobLifecycleListener;
@@ -62,9 +62,9 @@
     IGlobalRecoveryManager getGlobalRecoveryManager();
 
     /**
-     * @return the fault tolerance strategy in use for the cluster
+     * @return the NC lifecycle coordinator in use for the cluster
      */
-    IFaultToleranceStrategy getFaultToleranceStrategy();
+    INcLifecycleCoordinator getNcLifecycleCoordinator();
 
     /**
      * @return the active notification handler at Cluster controller
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IFaultToleranceStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INcLifecycleCoordinator.java
similarity index 70%
rename from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IFaultToleranceStrategy.java
rename to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INcLifecycleCoordinator.java
index bf4cac9..1a7c3c8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IFaultToleranceStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INcLifecycleCoordinator.java
@@ -19,13 +19,12 @@
 package org.apache.asterix.common.replication;
 
 import org.apache.asterix.common.cluster.IClusterStateManager;
-import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public interface IFaultToleranceStrategy {
+public interface INcLifecycleCoordinator {
 
     /**
-     * Defines the logic of a {@link IFaultToleranceStrategy} when a node joins the cluster.
+     * Defines the logic of a {@link INcLifecycleCoordinator} when a node joins the cluster.
      *
      * @param nodeId
      * @throws HyracksDataException
@@ -33,7 +32,7 @@
     void notifyNodeJoin(String nodeId) throws HyracksDataException;
 
     /**
-     * Defines the logic of a {@link IFaultToleranceStrategy} when a node leaves the cluster.
+     * Defines the logic of a {@link INcLifecycleCoordinator} when a node leaves the cluster.
      *
      * @param nodeId
      * @throws HyracksDataException
@@ -41,7 +40,7 @@
     void notifyNodeFailure(String nodeId) throws HyracksDataException;
 
     /**
-     * Binds the fault tolerance strategy to {@code cluserManager}.
+     * Binds the coordinator to {@code cluserManager}.
      *
      * @param clusterManager
      */
@@ -56,20 +55,9 @@
     void process(INCLifecycleMessage message) throws HyracksDataException;
 
     /**
-     * Constructs a fault tolerance strategy.
-     *
-     * @param serviceCtx
-     * @param replicationEnabled
-     * @return the fault tolerance strategy
-     */
-    IFaultToleranceStrategy from(ICCServiceContext serviceCtx, boolean replicationEnabled);
-
-    /**
      * Performs the required steps to change the metadata node to {@code node}
      *
      * @param node
      */
-    default void notifyMetadataNodeChange(String node) throws HyracksDataException {
-        throw new UnsupportedOperationException(getClass() + " does not support metadata node change");
-    }
+    void notifyMetadataNodeChange(String node) throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
index f991cd4..d8f14a2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
@@ -42,7 +42,7 @@
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.metadata.IMetadataBootstrap;
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
+import org.apache.asterix.common.replication.INcLifecycleCoordinator;
 import org.apache.asterix.common.transactions.IResourceIdManager;
 import org.apache.asterix.runtime.job.listener.NodeJobTracker;
 import org.apache.asterix.runtime.transaction.ResourceIdManager;
@@ -80,7 +80,7 @@
     private Supplier<IMetadataBootstrap> metadataBootstrapSupplier;
     private IHyracksClientConnection hcc;
     private Object extensionManager;
-    private IFaultToleranceStrategy ftStrategy;
+    private INcLifecycleCoordinator ftStrategy;
     private IJobLifecycleListener activeLifeCycleListener;
     private IMetadataLockManager mdLockManager;
     private IClusterStateManager clusterStateManager;
@@ -88,7 +88,7 @@
 
     public CcApplicationContext(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
             ILibraryManager libraryManager, Supplier<IMetadataBootstrap> metadataBootstrapSupplier,
-            IGlobalRecoveryManager globalRecoveryManager, IFaultToleranceStrategy ftStrategy,
+            IGlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator ftStrategy,
             IJobLifecycleListener activeLifeCycleListener, IStorageComponentProvider storageComponentProvider,
             IMetadataLockManager mdLockManager) throws AlgebricksException, IOException {
         this.ccServiceCtx = ccServiceCtx;
@@ -232,7 +232,7 @@
     }
 
     @Override
-    public IFaultToleranceStrategy getFaultToleranceStrategy() {
+    public INcLifecycleCoordinator getNcLifecycleCoordinator() {
         return ftStrategy;
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
index c07200a..6994862 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
@@ -23,7 +23,6 @@
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
@@ -35,7 +34,7 @@
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
+import org.apache.asterix.common.replication.INcLifecycleCoordinator;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.transactions.IResourceIdManager;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
@@ -60,11 +59,6 @@
  */
 
 public class ClusterStateManager implements IClusterStateManager {
-    /*
-     * TODO: currently after instance restarts we require all nodes to join again,
-     * otherwise the cluster wont be ACTIVE. we may overcome this by storing the cluster state before the instance
-     * shutdown and using it on startup to identify the nodes that are expected the join.
-     */
 
     private static final Logger LOGGER = LogManager.getLogger();
     private final Map<String, Map<IOption, Object>> ncConfigMap = new HashMap<>();
@@ -77,7 +71,7 @@
     private boolean metadataNodeActive = false;
     private Set<String> failedNodes = new HashSet<>();
     private Set<String> participantNodes = new HashSet<>();
-    private IFaultToleranceStrategy ftStrategy;
+    private INcLifecycleCoordinator lifecycleCoordinator;
     private ICcApplicationContext appCtx;
 
     @Override
@@ -86,8 +80,8 @@
         node2PartitionsMap = appCtx.getMetadataProperties().getNodePartitions();
         clusterPartitions = appCtx.getMetadataProperties().getClusterPartitions();
         currentMetadataNode = appCtx.getMetadataProperties().getMetadataNodeName();
-        ftStrategy = appCtx.getFaultToleranceStrategy();
-        ftStrategy.bindTo(this);
+        lifecycleCoordinator = appCtx.getNcLifecycleCoordinator();
+        lifecycleCoordinator.bindTo(this);
     }
 
     @Override
@@ -98,7 +92,7 @@
         failedNodes.add(nodeId);
         ncConfigMap.remove(nodeId);
         pendingRemoval.remove(nodeId);
-        ftStrategy.notifyNodeFailure(nodeId);
+        lifecycleCoordinator.notifyNodeFailure(nodeId);
     }
 
     @Override
@@ -109,7 +103,7 @@
         failedNodes.remove(nodeId);
         ncConfigMap.put(nodeId, configuration);
         updateNodeConfig(nodeId, configuration);
-        ftStrategy.notifyNodeJoin(nodeId);
+        lifecycleCoordinator.notifyNodeJoin(nodeId);
     }
 
     @Override

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/2766/ (10/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/2135/ (8/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/2774/ (5/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/5179/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/7689/ (7/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/2174/ (1/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/2155/ (9/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2433/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/6185/ (3/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/5179/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2433/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/cJRha4 : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecyc...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2228][CLUS] IFaultToleranceStrategy -> INcLifecycleCoordinator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/3018/ (11/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2266
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I51a2d6f5402630683f709806e6c01f0c7e83914f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No