[ASTERIXDB-2195][REPL] Replace Static Replication

- user model changes: no
- storage format changes: no
- interface changes: yes
  - Redesigned all replication interfaces

Details:
- Replace static replication and fault tolerance by
  dynamic storage API.
- Remove static based fault tolerance strategies.
- Redesign replication APIs and classes to smaller
  maintainable parts.
- Clean up replication properties.
- Unify logic for checkpoints when replication is
  enabled.
- Remove static replication test cases.
- Add replication runtime test cases for:
  - Bulkload component replication.
  - Memory component recovery.
  - Flushed component replication.
- Add replication integration test for:
  - Resync failed replica.

Change-Id: Ic5c4b0ac199a4530c807e558c8aebb1eb1284048
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2252
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: Michael Blow <mblow@apache.org>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 425cbe4..9c53c18 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -54,11 +54,8 @@
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.library.ILibraryManager;
-import org.apache.asterix.common.replication.IRemoteRecoveryManager;
-import org.apache.asterix.common.replication.IReplicaResourcesManager;
 import org.apache.asterix.common.replication.IReplicationChannel;
 import org.apache.asterix.common.replication.IReplicationManager;
-import org.apache.asterix.common.replication.Replica;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.asterix.common.storage.IReplicaManager;
 import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
@@ -72,10 +69,8 @@
 import org.apache.asterix.metadata.api.IAsterixStateProxy;
 import org.apache.asterix.metadata.api.IMetadataNode;
 import org.apache.asterix.metadata.bootstrap.MetadataBootstrap;
-import org.apache.asterix.replication.management.ReplicationChannel;
 import org.apache.asterix.replication.management.ReplicationManager;
-import org.apache.asterix.replication.recovery.RemoteRecoveryManager;
-import org.apache.asterix.replication.storage.ReplicaResourcesManager;
+import org.apache.asterix.replication.management.ReplicationChannel;
 import org.apache.asterix.runtime.transaction.GlobalResourceIdFactoryProvider;
 import org.apache.asterix.runtime.utils.NoOpCoordinationService;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
@@ -139,8 +134,6 @@
     private ActiveManager activeManager;
     private IReplicationChannel replicationChannel;
     private IReplicationManager replicationManager;
-    private IRemoteRecoveryManager remoteRecoveryManager;
-    private IReplicaResourcesManager replicaResourcesManager;
     private final ILibraryManager libraryManager;
     private final NCExtensionManager ncExtensionManager;
     private final IStorageComponentProvider componentProvider;
@@ -190,8 +183,7 @@
         indexCheckpointManagerProvider = new IndexCheckpointManagerProvider(ioManager);
 
         ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory =
-                new PersistentLocalResourceRepositoryFactory(ioManager, getServiceContext().getNodeId(),
-                        metadataProperties, indexCheckpointManagerProvider);
+                new PersistentLocalResourceRepositoryFactory(ioManager, indexCheckpointManagerProvider);
 
         localResourceRepository =
                 (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
@@ -224,47 +216,21 @@
                 this.ncServiceContext);
 
         if (replicationProperties.isReplicationEnabled()) {
+            replicationManager = new ReplicationManager(this, replicationProperties);
 
-            replicaResourcesManager = new ReplicaResourcesManager(localResourceRepository, metadataProperties,
-                    indexCheckpointManagerProvider);
+            //pass replication manager to replication required object
+            //LogManager to replicate logs
+            txnSubsystem.getLogManager().setReplicationManager(replicationManager);
 
-            replicationManager = new ReplicationManager(nodeId, replicationProperties, replicaResourcesManager,
-                    txnSubsystem.getLogManager(), asterixAppRuntimeContextProvider, ncServiceContext);
+            //PersistentLocalResourceRepository to replicate metadata files and delete backups on drop index
+            localResourceRepository.setReplicationManager(replicationManager);
 
-            if (replicationManager.getReplicationStrategy().isParticipant(getServiceContext().getNodeId())) {
+            //initialize replication channel
+            replicationChannel = new ReplicationChannel(this);
 
-                //pass replication manager to replication required object
-                //LogManager to replicate logs
-                txnSubsystem.getLogManager().setReplicationManager(replicationManager);
-
-                //PersistentLocalResourceRepository to replicate metadata files and delete backups on drop index
-                localResourceRepository.setReplicationManager(replicationManager);
-
-                /*
-                 * add the partitions that will be replicated in this node as inactive partitions
-                 */
-                //get nodes which replicate to this node
-                Set<String> remotePrimaryReplicas = replicationManager.getReplicationStrategy()
-                        .getRemotePrimaryReplicas(nodeId).stream().map(Replica::getId).collect(Collectors.toSet());
-                for (String clientId : remotePrimaryReplicas) {
-                    //get the partitions of each client
-                    ClusterPartition[] clientPartitions = metadataProperties.getNodePartitions().get(clientId);
-                    for (ClusterPartition partition : clientPartitions) {
-                        localResourceRepository.addInactivePartition(partition.getPartitionId());
-                    }
-                }
-
-                //initialize replication channel
-                replicationChannel = new ReplicationChannel(nodeId, replicationProperties, txnSubsystem.getLogManager(),
-                        replicaResourcesManager, replicationManager, getServiceContext(),
-                        asterixAppRuntimeContextProvider, replicationManager.getReplicationStrategy());
-
-                remoteRecoveryManager = new RemoteRecoveryManager(replicationManager, this, replicationProperties);
-
-                bufferCache = new BufferCache(ioManager, prs, pcp, new FileMapManager(),
-                        storageProperties.getBufferCacheMaxOpenFiles(), getServiceContext().getThreadFactory(),
-                        replicationManager);
-            }
+            bufferCache = new BufferCache(ioManager, prs, pcp, new FileMapManager(),
+                    storageProperties.getBufferCacheMaxOpenFiles(), getServiceContext().getThreadFactory(),
+                    replicationManager);
         } else {
             bufferCache = new BufferCache(ioManager, prs, pcp, new FileMapManager(),
                     storageProperties.getBufferCacheMaxOpenFiles(), getServiceContext().getThreadFactory());
@@ -441,16 +407,6 @@
     }
 
     @Override
-    public IReplicaResourcesManager getReplicaResourcesManager() {
-        return replicaResourcesManager;
-    }
-
-    @Override
-    public IRemoteRecoveryManager getRemoteRecoveryManager() {
-        return remoteRecoveryManager;
-    }
-
-    @Override
     public IReplicationManager getReplicationManager() {
         return replicationManager;
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index f458688..080ad48 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -39,6 +39,7 @@
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.INcApplicationContext;
@@ -46,7 +47,6 @@
 import org.apache.asterix.common.dataflow.DatasetLocalResource;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.replication.IReplicaResourcesManager;
 import org.apache.asterix.common.storage.DatasetResourceReference;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.asterix.common.transactions.Checkpoint;
@@ -96,16 +96,17 @@
     private final ICheckpointManager checkpointManager;
     private SystemState state;
     private final INCServiceContext serviceCtx;
+    private final INcApplicationContext appCtx;
+
 
     public RecoveryManager(ITransactionSubsystem txnSubsystem, INCServiceContext serviceCtx) {
         this.serviceCtx = serviceCtx;
         this.txnSubsystem = txnSubsystem;
+        this.appCtx = txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext();
         logMgr = (LogManager) txnSubsystem.getLogManager();
-        ReplicationProperties repProperties = txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext()
-                .getReplicationProperties();
+        ReplicationProperties repProperties = appCtx.getReplicationProperties();
         replicationEnabled = repProperties.isReplicationEnabled();
-        localResourceRepository = (PersistentLocalResourceRepository) txnSubsystem.getAsterixAppRuntimeContextProvider()
-                .getLocalResourceRepository();
+        localResourceRepository = (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
         cachedEntityCommitsPerJobSize = txnSubsystem.getTransactionProperties().getJobRecoveryMemorySize();
         checkpointManager = txnSubsystem.getCheckpointManager();
     }
@@ -129,32 +130,18 @@
             LOGGER.info("The checkpoint file doesn't exist: systemState = PERMANENT_DATA_LOSS");
             return state;
         }
-
-        if (replicationEnabled) {
-            if (checkpointObject.getMinMCTFirstLsn() == AbstractCheckpointManager.SHARP_CHECKPOINT_LSN) {
-                //no logs exist
-                state = SystemState.HEALTHY;
-            } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN() && checkpointObject.isSharp()) {
-                //only remote logs exist
-                state = SystemState.HEALTHY;
-            } else {
-                //need to perform remote recovery
-                state = SystemState.CORRUPTED;
+        long readableSmallestLSN = logMgr.getReadableSmallestLSN();
+        if (logMgr.getAppendLSN() == readableSmallestLSN) {
+            if (checkpointObject.getMinMCTFirstLsn() != AbstractCheckpointManager.SHARP_CHECKPOINT_LSN) {
+                LOGGER.warn("Some(or all) of transaction log files are lost.");
+                //No choice but continuing when the log files are lost.
             }
+            state = SystemState.HEALTHY;
+        } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN()
+                && checkpointObject.getMinMCTFirstLsn() == AbstractCheckpointManager.SHARP_CHECKPOINT_LSN) {
+            state = SystemState.HEALTHY;
         } else {
-            long readableSmallestLSN = logMgr.getReadableSmallestLSN();
-            if (logMgr.getAppendLSN() == readableSmallestLSN) {
-                if (checkpointObject.getMinMCTFirstLsn() != AbstractCheckpointManager.SHARP_CHECKPOINT_LSN) {
-                    LOGGER.warn("Some(or all) of transaction log files are lost.");
-                    //No choice but continuing when the log files are lost.
-                }
-                state = SystemState.HEALTHY;
-            } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN()
-                    && checkpointObject.getMinMCTFirstLsn() == AbstractCheckpointManager.SHARP_CHECKPOINT_LSN) {
-                state = SystemState.HEALTHY;
-            } else {
-                state = SystemState.CORRUPTED;
-            }
+            state = SystemState.CORRUPTED;
         }
         return state;
     }
@@ -442,9 +429,47 @@
     }
 
     private long getRemoteMinFirstLSN() throws HyracksDataException {
-        IReplicaResourcesManager remoteResourcesManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
-                .getAppContext().getReplicaResourcesManager();
-        return remoteResourcesManager.getPartitionsMinLSN(localResourceRepository.getInactivePartitions());
+        // find the min first lsn of partitions that are replicated on this node
+        final Set<Integer> allPartitions = localResourceRepository.getAllPartitions();
+        final INcApplicationContext appContext = txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext();
+        final Set<Integer> masterPartitions = appContext.getReplicaManager().getPartitions();
+        allPartitions.removeAll(masterPartitions);
+        return getPartitionsMinLSN(allPartitions);
+    }
+
+    private long getPartitionsMinLSN(Set<Integer> partitions) throws HyracksDataException {
+        final IIndexCheckpointManagerProvider idxCheckpointMgrProvider = appCtx.getIndexCheckpointManagerProvider();
+        long minRemoteLSN = Long.MAX_VALUE;
+        for (Integer partition : partitions) {
+            final List<DatasetResourceReference> partitionResources = localResourceRepository.getResources(resource -> {
+                DatasetLocalResource dsResource = (DatasetLocalResource) resource.getResource();
+                return dsResource.getPartition() == partition;
+            }).values().stream().map(DatasetResourceReference::of).collect(Collectors.toList());
+            for (DatasetResourceReference indexRef : partitionResources) {
+                long remoteIndexMaxLSN = idxCheckpointMgrProvider.get(indexRef).getLowWatermark();
+                minRemoteLSN = Math.min(minRemoteLSN, remoteIndexMaxLSN);
+            }
+        }
+        return minRemoteLSN;
+    }
+
+    @Override
+    public void replayReplicaPartitionLogs(Set<Integer> partitions, boolean flush) throws HyracksDataException {
+        long minLSN = getPartitionsMinLSN(partitions);
+        long readableSmallestLSN = logMgr.getReadableSmallestLSN();
+        if (minLSN < readableSmallestLSN) {
+            minLSN = readableSmallestLSN;
+        }
+
+        //replay logs > minLSN that belong to these partitions
+        try {
+            replayPartitionsLogs(partitions, logMgr.getLogReader(true), minLSN);
+            if (flush) {
+                appCtx.getDatasetLifecycleManager().flushAllDatasets();
+            }
+        } catch (IOException | ACIDException e) {
+            throw HyracksDataException.create(e);
+        }
     }
 
     @Override
@@ -535,7 +560,7 @@
         TxnEntityId loserEntity;
         List<Long> undoLSNSet = null;
         //get active partitions on this node
-        Set<Integer> activePartitions = localResourceRepository.getActivePartitions();
+        Set<Integer> activePartitions = appCtx.getReplicaManager().getPartitions();
         ILogReader logReader = logMgr.getLogReader(false);
         try {
             logReader.setPosition(firstLSN);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java
index bf17a5b..155fa1d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java
@@ -29,10 +29,10 @@
 
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.replication.IPartitionReplica;
-import org.apache.asterix.common.replication.IRemoteRecoveryManager;
 import org.apache.asterix.common.storage.IReplicaManager;
 import org.apache.asterix.common.storage.ReplicaIdentifier;
-import org.apache.asterix.replication.storage.PartitionReplica;
+import org.apache.asterix.common.transactions.IRecoveryManager;
+import org.apache.asterix.replication.api.PartitionReplica;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class ReplicaManager implements IReplicaManager {
@@ -67,7 +67,9 @@
         if (!replicas.containsKey(id)) {
             throw new IllegalStateException("replica with id(" + id + ") does not exist");
         }
-        replicas.remove(id);
+        PartitionReplica replica = replicas.remove(id);
+        appCtx.getReplicationManager().unregister(replica);
+
     }
 
     @Override
@@ -83,8 +85,8 @@
 
     @Override
     public synchronized void promote(int partition) throws HyracksDataException {
-        final IRemoteRecoveryManager remoteRecoveryManager = appCtx.getRemoteRecoveryManager();
-        remoteRecoveryManager.replayReplicaPartitionLogs(Stream.of(partition).collect(Collectors.toSet()), true);
+        final IRecoveryManager recoveryManager = appCtx.getTransactionSubsystem().getRecoveryManager();
+        recoveryManager.replayReplicaPartitionLogs(Stream.of(partition).collect(Collectors.toSet()), true);
         partitions.add(partition);
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
index cd9b617..fae0413 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
@@ -76,7 +76,7 @@
         if (LOGGER.isInfoEnabled()) {
             LOGGER.log(Level.INFO, "Checkpoint Properties: " + checkpointProperties);
         }
-        checkpointManager = CheckpointManagerFactory.create(this, checkpointProperties, replicationEnabled);
+        checkpointManager = CheckpointManagerFactory.create(this, checkpointProperties);
         final Checkpoint latestCheckpoint = checkpointManager.getLatest();
         if (latestCheckpoint != null) {
             transactionManager.ensureMaxTxnId(latestCheckpoint.getMaxTxnId());
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/RemoteRecoveryTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/RemoteRecoveryTask.java
deleted file mode 100644
index d330684..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/RemoteRecoveryTask.java
+++ /dev/null
@@ -1,57 +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.nc.task;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.asterix.common.api.INCLifecycleTask;
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.service.IControllerService;
-
-public class RemoteRecoveryTask implements INCLifecycleTask {
-
-    private static final long serialVersionUID = 1L;
-    private final Map<String, Set<Integer>> recoveryPlan;
-
-    public RemoteRecoveryTask(Map<String, Set<Integer>> recoveryPlan) {
-        this.recoveryPlan = recoveryPlan;
-    }
-
-    @Override
-    public void perform(IControllerService cs) throws HyracksDataException {
-        INcApplicationContext appContext = (INcApplicationContext) cs.getApplicationContext();
-        appContext.getRemoteRecoveryManager().doRemoteRecoveryPlan(recoveryPlan);
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws IOException {
-        out.defaultWriteObject();
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
-        in.defaultReadObject();
-    }
-
-    @Override
-    public String toString() {
-        return "{ \"class\" : \"" + getClass().getSimpleName() + "\", \"recovery-plan\" : " + recoveryPlan + " }";
-    }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartFailbackTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartFailbackTask.java
deleted file mode 100644
index ecd93a3..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartFailbackTask.java
+++ /dev/null
@@ -1,40 +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.nc.task;
-
-import org.apache.asterix.common.api.INCLifecycleTask;
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.service.IControllerService;
-
-public class StartFailbackTask implements INCLifecycleTask {
-
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public void perform(IControllerService cs) throws HyracksDataException {
-        INcApplicationContext appContext = (INcApplicationContext) cs.getApplicationContext();
-        appContext.getRemoteRecoveryManager().startFailbackProcess();
-    }
-
-    @Override
-    public String toString() {
-        return "{ \"class\" : \"" + getClass().getSimpleName() + "\" }";
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartReplicationServiceTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartReplicationServiceTask.java
index 253f121..7071271 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartReplicationServiceTask.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartReplicationServiceTask.java
@@ -20,7 +20,6 @@
 
 import org.apache.asterix.common.api.INCLifecycleTask;
 import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.replication.IReplicationManager;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.service.IControllerService;
 
@@ -32,13 +31,8 @@
     public void perform(IControllerService cs) throws HyracksDataException {
         INcApplicationContext appContext = (INcApplicationContext) cs.getApplicationContext();
         try {
-            //Open replication channel
+            // open replication channel
             appContext.getReplicationChannel().start();
-            final IReplicationManager replicationManager = appContext.getReplicationManager();
-            //Check the state of remote replicas
-            replicationManager.initializeReplicasState();
-            //Start replication after the state of remote replicas has been initialized.
-            replicationManager.startReplicationThreads();
         } catch (Exception e) {
             throw HyracksDataException.create(e);
         }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/AutoFaultToleranceStrategy.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/AutoFaultToleranceStrategy.java
deleted file mode 100644
index 37b4d4f..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/AutoFaultToleranceStrategy.java
+++ /dev/null
@@ -1,542 +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 java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.asterix.app.nc.task.BindMetadataNodeTask;
-import org.apache.asterix.app.nc.task.CheckpointTask;
-import org.apache.asterix.app.nc.task.ExternalLibrarySetupTask;
-import org.apache.asterix.app.nc.task.MetadataBootstrapTask;
-import org.apache.asterix.app.nc.task.ReportLocalCountersTask;
-import org.apache.asterix.app.nc.task.StartFailbackTask;
-import org.apache.asterix.app.nc.task.StartLifecycleComponentsTask;
-import org.apache.asterix.app.nc.task.StartReplicationServiceTask;
-import org.apache.asterix.app.replication.NodeFailbackPlan.FailbackPlanState;
-import org.apache.asterix.app.replication.message.CompleteFailbackRequestMessage;
-import org.apache.asterix.app.replication.message.CompleteFailbackResponseMessage;
-import org.apache.asterix.app.replication.message.NCLifecycleTaskReportMessage;
-import org.apache.asterix.app.replication.message.PreparePartitionsFailbackRequestMessage;
-import org.apache.asterix.app.replication.message.PreparePartitionsFailbackResponseMessage;
-import org.apache.asterix.app.replication.message.RegistrationTasksRequestMessage;
-import org.apache.asterix.app.replication.message.RegistrationTasksResponseMessage;
-import org.apache.asterix.app.replication.message.MetadataNodeRequestMessage;
-import org.apache.asterix.app.replication.message.MetadataNodeResponseMessage;
-import org.apache.asterix.app.replication.message.TakeoverPartitionsRequestMessage;
-import org.apache.asterix.app.replication.message.TakeoverPartitionsResponseMessage;
-import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
-import org.apache.asterix.common.api.INCLifecycleTask;
-import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.cluster.IClusterStateManager;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-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.IReplicationStrategy;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
-import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.util.FaultToleranceUtil;
-import org.apache.hyracks.api.application.ICCServiceContext;
-import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class AutoFaultToleranceStrategy implements IFaultToleranceStrategy {
-
-    private static final Logger LOGGER = LogManager.getLogger();
-    private long clusterRequestId = 0;
-
-    private Set<String> failedNodes = new HashSet<>();
-    private LinkedList<NodeFailbackPlan> pendingProcessingFailbackPlans = new LinkedList<>();
-    private Map<Long, NodeFailbackPlan> planId2FailbackPlanMap = new HashMap<>();
-    private Map<Long, TakeoverPartitionsRequestMessage> pendingTakeoverRequests = new HashMap<>();;
-    private String currentMetadataNode;
-    private boolean metadataNodeActive = false;
-    private IClusterStateManager clusterManager;
-    private ICCMessageBroker messageBroker;
-    private IReplicationStrategy replicationStrategy;
-    private ICCServiceContext serviceCtx;
-    private Set<String> pendingStartupCompletionNodes = new HashSet<>();
-    private List<String> nodeIds;
-    private Map<String, SystemState> startupQueue = new HashMap<>();
-
-    @Override
-    public void notifyNodeJoin(String nodeId) {
-        pendingStartupCompletionNodes.add(nodeId);
-    }
-
-    @Override
-    public void notifyNodeFailure(String nodeId) throws HyracksDataException {
-        //if this node was waiting for failback and failed before it completed
-        if (failedNodes.contains(nodeId)) {
-            notifyFailbackPlansNodeFailure(nodeId);
-            revertFailedFailbackPlanEffects();
-            return;
-        }
-        //an active node failed
-        failedNodes.add(nodeId);
-        clusterManager.updateNodePartitions(nodeId, false);
-        if (nodeId.equals(currentMetadataNode)) {
-            metadataNodeActive = false;
-            clusterManager.updateMetadataNode(nodeId, metadataNodeActive);
-        }
-        validateClusterState();
-        FaultToleranceUtil.notifyImpactedReplicas(nodeId, ClusterEventType.NODE_FAILURE, clusterManager, messageBroker,
-                replicationStrategy);
-        notifyFailbackPlansNodeFailure(nodeId);
-        requestPartitionsTakeover(nodeId);
-    }
-
-    private synchronized void notifyFailbackPlansNodeFailure(String nodeId) {
-        for (NodeFailbackPlan plan : planId2FailbackPlanMap.values()) {
-            plan.notifyNodeFailure(nodeId);
-        }
-    }
-
-    private synchronized void revertFailedFailbackPlanEffects() {
-        Iterator<NodeFailbackPlan> iterator = planId2FailbackPlanMap.values().iterator();
-        while (iterator.hasNext()) {
-            NodeFailbackPlan plan = iterator.next();
-            if (plan.getState() == FailbackPlanState.PENDING_ROLLBACK) {
-                //TODO if the failing back node is still active, notify it to construct a new plan for it
-                iterator.remove();
-
-                //reassign the partitions that were supposed to be failed back to an active replica
-                requestPartitionsTakeover(plan.getNodeId());
-            }
-        }
-    }
-
-    private synchronized void requestPartitionsTakeover(String failedNodeId) {
-        //replica -> list of partitions to takeover
-        Map<String, List<Integer>> partitionRecoveryPlan = new HashMap<>();
-        //collect the partitions of the failed NC
-        List<ClusterPartition> lostPartitions = getNodeAssignedPartitions(failedNodeId);
-        if (!lostPartitions.isEmpty()) {
-            for (ClusterPartition partition : lostPartitions) {
-                //find replicas for this partitions
-                Set<String> partitionReplicas = replicationStrategy.getRemoteReplicas(partition.getNodeId()).stream()
-                        .map(Replica::getId).collect(Collectors.toSet());
-                //find a replica that is still active
-                for (String replica : partitionReplicas) {
-                    //TODO (mhubail) currently this assigns the partition to the first found active replica.
-                    //It needs to be modified to consider load balancing.
-                    if (addActiveReplica(replica, partition, partitionRecoveryPlan)) {
-                        break;
-                    }
-                }
-            }
-
-            if (partitionRecoveryPlan.size() == 0) {
-                //no active replicas were found for the failed node
-                LOGGER.error("Could not find active replicas for the partitions " + lostPartitions);
-                return;
-            } else {
-                LOGGER.info("Partitions to recover: " + lostPartitions);
-            }
-            //For each replica, send a request to takeover the assigned partitions
-            partitionRecoveryPlan.forEach((replica, value) -> {
-                Integer[] partitionsToTakeover = value.toArray(new Integer[value.size()]);
-                long requestId = clusterRequestId++;
-                TakeoverPartitionsRequestMessage takeoverRequest = new TakeoverPartitionsRequestMessage(requestId,
-                        replica, partitionsToTakeover);
-                pendingTakeoverRequests.put(requestId, takeoverRequest);
-                try {
-                    messageBroker.sendApplicationMessageToNC(takeoverRequest, replica);
-                } catch (Exception e) {
-                    /*
-                     * if we fail to send the request, it means the NC we tried to send the request to
-                     * has failed. When the failure notification arrives, we will send any pending request
-                     * that belongs to the failed NC to a different active replica.
-                     */
-                    LOGGER.log(Level.WARN, "Failed to send takeover request: " + takeoverRequest, e);
-                }
-            });
-        }
-    }
-
-    private boolean addActiveReplica(String replica, ClusterPartition partition,
-            Map<String, List<Integer>> partitionRecoveryPlan) {
-        final Set<String> participantNodes = clusterManager.getParticipantNodes();
-        if (participantNodes.contains(replica) && !failedNodes.contains(replica)) {
-            if (!partitionRecoveryPlan.containsKey(replica)) {
-                List<Integer> replicaPartitions = new ArrayList<>();
-                replicaPartitions.add(partition.getPartitionId());
-                partitionRecoveryPlan.put(replica, replicaPartitions);
-            } else {
-                partitionRecoveryPlan.get(replica).add(partition.getPartitionId());
-            }
-            return true;
-        }
-        return false;
-    }
-
-    private synchronized void prepareFailbackPlan(String failingBackNodeId) {
-        NodeFailbackPlan plan = NodeFailbackPlan.createPlan(failingBackNodeId);
-        pendingProcessingFailbackPlans.add(plan);
-        planId2FailbackPlanMap.put(plan.getPlanId(), plan);
-
-        //get all partitions this node requires to resync
-        Set<String> nodeReplicas = replicationStrategy.getRemoteReplicas(failingBackNodeId).stream().map(Replica::getId)
-                .collect(Collectors.toSet());
-        clusterManager.getClusterPartitons();
-        for (String replicaId : nodeReplicas) {
-            ClusterPartition[] nodePartitions = clusterManager.getNodePartitions(replicaId);
-            for (ClusterPartition partition : nodePartitions) {
-                plan.addParticipant(partition.getActiveNodeId());
-                /*
-                 * if the partition original node is the returning node,
-                 * add it to the list of the partitions which will be failed back
-                 */
-                if (partition.getNodeId().equals(failingBackNodeId)) {
-                    plan.addPartitionToFailback(partition.getPartitionId(), partition.getActiveNodeId());
-                }
-            }
-        }
-
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Prepared Failback plan: " + plan.toString());
-        }
-
-        processPendingFailbackPlans();
-    }
-
-    private synchronized void processPendingFailbackPlans() {
-        ClusterState state = clusterManager.getState();
-        /*
-         * if the cluster state is not ACTIVE, then failbacks should not be processed
-         * since some partitions are not active
-         */
-        if (state == ClusterState.ACTIVE) {
-            while (!pendingProcessingFailbackPlans.isEmpty()) {
-                //take the first pending failback plan
-                NodeFailbackPlan plan = pendingProcessingFailbackPlans.pop();
-                /*
-                 * A plan at this stage will be in one of two states:
-                 * 1. PREPARING -> the participants were selected but we haven't sent any request.
-                 * 2. PENDING_ROLLBACK -> a participant failed before we send any requests
-                 */
-                if (plan.getState() == FailbackPlanState.PREPARING) {
-                    //set the partitions that will be failed back as inactive
-                    String failbackNode = plan.getNodeId();
-                    for (Integer partitionId : plan.getPartitionsToFailback()) {
-                        //partition expected to be returned to the failing back node
-                        clusterManager.updateClusterPartition(partitionId, failbackNode, false);
-                    }
-
-                    /*
-                     * if the returning node is the original metadata node,
-                     * then metadata node will change after the failback completes
-                     */
-                    ICcApplicationContext appCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
-                    String originalMetadataNode = appCtx.getMetadataProperties().getMetadataNodeName();
-                    if (originalMetadataNode.equals(failbackNode)) {
-                        plan.setNodeToReleaseMetadataManager(currentMetadataNode);
-                        currentMetadataNode = "";
-                        metadataNodeActive = false;
-                        clusterManager.updateMetadataNode(currentMetadataNode, metadataNodeActive);
-                    }
-
-                    //force new jobs to wait
-                    clusterManager.setState(ClusterState.REBALANCING);
-                    handleFailbackRequests(plan, messageBroker);
-                    /*
-                     * wait until the current plan is completed before processing the next plan.
-                     * when the current one completes or is reverted, the cluster state will be
-                     * ACTIVE again, and the next failback plan (if any) will be processed.
-                     */
-                    break;
-                } else if (plan.getState() == FailbackPlanState.PENDING_ROLLBACK) {
-                    //this plan failed before sending any requests -> nothing to rollback
-                    planId2FailbackPlanMap.remove(plan.getPlanId());
-                }
-            }
-        }
-    }
-
-    private void handleFailbackRequests(NodeFailbackPlan plan, ICCMessageBroker messageBroker) {
-        //send requests to other nodes to complete on-going jobs and prepare partitions for failback
-        for (PreparePartitionsFailbackRequestMessage request : plan.getPlanFailbackRequests()) {
-            try {
-                messageBroker.sendApplicationMessageToNC(request, request.getNodeID());
-                plan.addPendingRequest(request);
-            } catch (Exception e) {
-                LOGGER.log(Level.WARN, "Failed to send failback request to: " + request.getNodeID(), e);
-                plan.notifyNodeFailure(request.getNodeID());
-                revertFailedFailbackPlanEffects();
-                break;
-            }
-        }
-    }
-
-    public synchronized List<ClusterPartition> getNodeAssignedPartitions(String nodeId) {
-        List<ClusterPartition> nodePartitions = new ArrayList<>();
-        ClusterPartition[] clusterPartitons = clusterManager.getClusterPartitons();
-        Map<Integer, ClusterPartition> clusterPartitionsMap = new HashMap<>();
-        for (ClusterPartition partition : clusterPartitons) {
-            clusterPartitionsMap.put(partition.getPartitionId(), partition);
-        }
-        for (ClusterPartition partition : clusterPartitons) {
-            if (nodeId.equals(partition.getActiveNodeId())) {
-                nodePartitions.add(partition);
-            }
-        }
-        /*
-         * if there is any pending takeover request this node was supposed to handle,
-         * it needs to be sent to a different replica
-         */
-        List<Long> failedTakeoverRequests = new ArrayList<>();
-        for (TakeoverPartitionsRequestMessage request : pendingTakeoverRequests.values()) {
-            if (request.getNodeId().equals(nodeId)) {
-                for (Integer partitionId : request.getPartitions()) {
-                    nodePartitions.add(clusterPartitionsMap.get(partitionId));
-                }
-                failedTakeoverRequests.add(request.getRequestId());
-            }
-        }
-
-        //remove failed requests
-        for (Long requestId : failedTakeoverRequests) {
-            pendingTakeoverRequests.remove(requestId);
-        }
-        return nodePartitions;
-    }
-
-    public synchronized void process(TakeoverPartitionsResponseMessage response) throws HyracksDataException {
-        for (Integer partitonId : response.getPartitions()) {
-            clusterManager.updateClusterPartition(partitonId, response.getNodeId(), true);
-        }
-        pendingTakeoverRequests.remove(response.getRequestId());
-        validateClusterState();
-    }
-
-    public synchronized void process(MetadataNodeResponseMessage response) throws HyracksDataException {
-        currentMetadataNode = response.getNodeId();
-        metadataNodeActive = true;
-        clusterManager.updateMetadataNode(currentMetadataNode, metadataNodeActive);
-        validateClusterState();
-    }
-
-    private void validateClusterState() throws HyracksDataException {
-        clusterManager.refreshState();
-        ClusterState newState = clusterManager.getState();
-        // PENDING: all partitions are active but metadata node is not
-        if (newState == ClusterState.PENDING) {
-            requestMetadataNodeTakeover();
-        } else if (newState == ClusterState.ACTIVE) {
-            processPendingFailbackPlans();
-        }
-    }
-
-    public synchronized void process(PreparePartitionsFailbackResponseMessage msg) {
-        NodeFailbackPlan plan = planId2FailbackPlanMap.get(msg.getPlanId());
-        plan.markRequestCompleted(msg.getRequestId());
-        /*
-         * A plan at this stage will be in one of three states:
-         * 1. PENDING_PARTICIPANT_REPONSE -> one or more responses are still expected (wait).
-         * 2. PENDING_COMPLETION -> all responses received (time to send completion request).
-         * 3. PENDING_ROLLBACK -> the plan failed and we just received the final pending response (revert).
-         */
-        if (plan.getState() == FailbackPlanState.PENDING_COMPLETION) {
-            CompleteFailbackRequestMessage request = plan.getCompleteFailbackRequestMessage();
-
-            //send complete resync and takeover partitions to the failing back node
-            try {
-                messageBroker.sendApplicationMessageToNC(request, request.getNodeId());
-            } catch (Exception e) {
-                LOGGER.log(Level.WARN, "Failed to send complete failback request to: " + request.getNodeId(), e);
-                notifyFailbackPlansNodeFailure(request.getNodeId());
-                revertFailedFailbackPlanEffects();
-            }
-        } else if (plan.getState() == FailbackPlanState.PENDING_ROLLBACK) {
-            revertFailedFailbackPlanEffects();
-        }
-    }
-
-    public synchronized void process(CompleteFailbackResponseMessage response) throws HyracksDataException {
-        /*
-         * the failback plan completed successfully:
-         * Remove all references to it.
-         * Remove the the failing back node from the failed nodes list.
-         * Notify its replicas to reconnect to it.
-         * Set the failing back node partitions as active.
-         */
-        NodeFailbackPlan plan = planId2FailbackPlanMap.remove(response.getPlanId());
-        String nodeId = plan.getNodeId();
-        failedNodes.remove(nodeId);
-        //notify impacted replicas they can reconnect to this node
-        FaultToleranceUtil.notifyImpactedReplicas(nodeId, ClusterEventType.NODE_JOIN, clusterManager, messageBroker,
-                replicationStrategy);
-        clusterManager.updateNodePartitions(nodeId, true);
-        validateClusterState();
-    }
-
-    private synchronized void requestMetadataNodeTakeover() {
-        //need a new node to takeover metadata node
-        ICcApplicationContext appCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
-        ClusterPartition metadataPartiton = appCtx.getMetadataProperties().getMetadataPartition();
-        //request the metadataPartition node to register itself as the metadata node
-        MetadataNodeRequestMessage takeoverRequest = new MetadataNodeRequestMessage(true);
-        try {
-            messageBroker.sendApplicationMessageToNC(takeoverRequest, metadataPartiton.getActiveNodeId());
-            // Since the metadata node will be changed, we need to rebind the proxy object
-            MetadataManager.INSTANCE.rebindMetadataNode();
-        } catch (Exception e) {
-            /*
-             * if we fail to send the request, it means the NC we tried to send the request to
-             * has failed. When the failure notification arrives, a new NC will be assigned to
-             * the metadata partition and a new metadata node takeover request will be sent to it.
-             */
-            LOGGER.log(Level.WARN,
-                    "Failed to send metadata node takeover request to: " + metadataPartiton.getActiveNodeId(), e);
-        }
-    }
-
-    @Override
-    public IFaultToleranceStrategy from(ICCServiceContext serviceCtx, IReplicationStrategy replicationStrategy) {
-        AutoFaultToleranceStrategy ft = new AutoFaultToleranceStrategy();
-        ft.messageBroker = (ICCMessageBroker) serviceCtx.getMessageBroker();
-        ft.replicationStrategy = replicationStrategy;
-        ft.serviceCtx = serviceCtx;
-        ft.nodeIds = serviceCtx.getAppConfig().getNCNames();
-        return ft;
-    }
-
-    @Override
-    public synchronized void process(INCLifecycleMessage message) throws HyracksDataException {
-        switch (message.getType()) {
-            case REGISTRATION_TASKS_REQUEST:
-                process((RegistrationTasksRequestMessage) message);
-                break;
-            case REGISTRATION_TASKS_RESULT:
-                process((NCLifecycleTaskReportMessage) message);
-                break;
-            case TAKEOVER_PARTITION_RESPONSE:
-                process((TakeoverPartitionsResponseMessage) message);
-                break;
-            case METADATA_NODE_RESPONSE:
-                process((MetadataNodeResponseMessage) message);
-                break;
-            case PREPARE_FAILBACK_RESPONSE:
-                process((PreparePartitionsFailbackResponseMessage) message);
-                break;
-            case COMPLETE_FAILBACK_RESPONSE:
-                process((CompleteFailbackResponseMessage) message);
-                break;
-            default:
-                throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MESSAGE_TYPE, message.getType().name());
-        }
-    }
-
-    private synchronized void process(NCLifecycleTaskReportMessage msg) throws HyracksDataException {
-        final String nodeId = msg.getNodeId();
-        pendingStartupCompletionNodes.remove(nodeId);
-        if (msg.isSuccess()) {
-            if (failedNodes.contains(nodeId)) {
-                prepareFailbackPlan(nodeId);
-                return;
-            }
-            // If this node failed and recovered, notify impacted replicas to reconnect to it
-            if (replicationStrategy.isParticipant(nodeId) && failedNodes.remove(nodeId)) {
-                FaultToleranceUtil.notifyImpactedReplicas(nodeId, ClusterEventType.NODE_JOIN, clusterManager,
-                        messageBroker, replicationStrategy);
-            }
-            clusterManager.updateNodePartitions(nodeId, true);
-            if (msg.getNodeId().equals(currentMetadataNode)) {
-                clusterManager.updateMetadataNode(currentMetadataNode, true);
-            }
-            clusterManager.refreshState();
-        } else {
-            LOGGER.log(Level.ERROR, msg.getNodeId() + " failed to complete startup. ", msg.getException());
-        }
-    }
-
-    @Override
-    public void bindTo(IClusterStateManager clusterManager) {
-        this.clusterManager = clusterManager;
-        currentMetadataNode = clusterManager.getCurrentMetadataNodeId();
-    }
-
-    private synchronized void process(RegistrationTasksRequestMessage msg) throws HyracksDataException {
-        final String nodeId = msg.getNodeId();
-        final SystemState state = msg.getState();
-        //last node needed to start
-        if (startupQueue.keySet().size() == nodeIds.size() - 1) {
-            startupQueue.put(nodeId, state);
-            for (Map.Entry<String, SystemState> nodeState : startupQueue.entrySet()) {
-                List<INCLifecycleTask> tasks = buildStartupSequence(nodeState.getKey());
-                RegistrationTasksResponseMessage response = new RegistrationTasksResponseMessage(nodeState.getKey(),
-                        tasks);
-                try {
-                    messageBroker.sendApplicationMessageToNC(response, nodeState.getKey());
-                } catch (Exception e) {
-                    throw HyracksDataException.create(e);
-                }
-            }
-        } else if (!failedNodes.isEmpty()) {
-            List<INCLifecycleTask> tasks = buildFailbackStartupSequence();
-            RegistrationTasksResponseMessage response = new RegistrationTasksResponseMessage(nodeId, tasks);
-            try {
-                messageBroker.sendApplicationMessageToNC(response, msg.getNodeId());
-            } catch (Exception e) {
-                throw HyracksDataException.create(e);
-            }
-        } else {
-            startupQueue.put(nodeId, state);
-        }
-    }
-
-    private List<INCLifecycleTask> buildFailbackStartupSequence() {
-        final List<INCLifecycleTask> tasks = new ArrayList<>();
-        tasks.add(new StartFailbackTask());
-        tasks.add(new ReportLocalCountersTask());
-        tasks.add(new StartLifecycleComponentsTask());
-        return tasks;
-    }
-
-    private List<INCLifecycleTask> buildStartupSequence(String nodeId) {
-        final List<INCLifecycleTask> tasks = new ArrayList<>();
-        tasks.add(new StartReplicationServiceTask());
-        final boolean isMetadataNode = nodeId.equals(currentMetadataNode);
-        if (isMetadataNode) {
-            tasks.add(new MetadataBootstrapTask());
-        }
-        tasks.add(new ExternalLibrarySetupTask(isMetadataNode));
-        tasks.add(new ReportLocalCountersTask());
-        tasks.add(new CheckpointTask());
-        tasks.add(new StartLifecycleComponentsTask());
-        if (isMetadataNode) {
-            tasks.add(new BindMetadataNodeTask(true));
-        }
-        return tasks;
-    }
-}
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
index fe24fca..5a715d6 100644
--- 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
@@ -18,9 +18,7 @@
  */
 package org.apache.asterix.app.replication;
 
-import org.apache.asterix.common.config.ReplicationProperties;
 import org.apache.asterix.common.replication.IFaultToleranceStrategy;
-import org.apache.asterix.common.replication.IReplicationStrategy;
 import org.apache.hyracks.api.application.ICCServiceContext;
 
 public class FaultToleranceStrategyFactory {
@@ -29,20 +27,10 @@
         throw new AssertionError();
     }
 
-    public static final String STRATEGY_NAME = "metadata_only";
-
-    public static IFaultToleranceStrategy create(ICCServiceContext serviceCtx, ReplicationProperties repProp,
-            IReplicationStrategy strategy) {
-        Class<? extends IFaultToleranceStrategy> clazz;
-        if (!repProp.isReplicationEnabled()) {
-            clazz = NoFaultToleranceStrategy.class;
-        } else if (STRATEGY_NAME.equals(repProp.getReplicationStrategy())) {
-            clazz = MetadataNodeFaultToleranceStrategy.class;
-        } else {
-            clazz = AutoFaultToleranceStrategy.class;
-        }
+    public static IFaultToleranceStrategy create(ICCServiceContext serviceCtx, boolean replicationEnabled) {
+        Class<? extends IFaultToleranceStrategy> clazz = NoFaultToleranceStrategy.class;
         try {
-            return clazz.newInstance().from(serviceCtx, strategy);
+            return clazz.newInstance().from(serviceCtx, replicationEnabled);
         } catch (InstantiationException | IllegalAccessException e) {
             throw new IllegalStateException(e);
         }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
deleted file mode 100644
index 02174da..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
+++ /dev/null
@@ -1,304 +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 java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.asterix.app.nc.task.BindMetadataNodeTask;
-import org.apache.asterix.app.nc.task.CheckpointTask;
-import org.apache.asterix.app.nc.task.ExternalLibrarySetupTask;
-import org.apache.asterix.app.nc.task.LocalRecoveryTask;
-import org.apache.asterix.app.nc.task.MetadataBootstrapTask;
-import org.apache.asterix.app.nc.task.RemoteRecoveryTask;
-import org.apache.asterix.app.nc.task.ReportLocalCountersTask;
-import org.apache.asterix.app.nc.task.StartLifecycleComponentsTask;
-import org.apache.asterix.app.nc.task.StartReplicationServiceTask;
-import org.apache.asterix.app.replication.message.MetadataNodeRequestMessage;
-import org.apache.asterix.app.replication.message.MetadataNodeResponseMessage;
-import org.apache.asterix.app.replication.message.NCLifecycleTaskReportMessage;
-import org.apache.asterix.app.replication.message.RegistrationTasksRequestMessage;
-import org.apache.asterix.app.replication.message.ReplayPartitionLogsRequestMessage;
-import org.apache.asterix.app.replication.message.ReplayPartitionLogsResponseMessage;
-import org.apache.asterix.app.replication.message.RegistrationTasksResponseMessage;
-import org.apache.asterix.common.api.INCLifecycleTask;
-import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.cluster.IClusterStateManager;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-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.IReplicationStrategy;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
-import org.apache.asterix.util.FaultToleranceUtil;
-import org.apache.hyracks.api.application.ICCServiceContext;
-import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class MetadataNodeFaultToleranceStrategy implements IFaultToleranceStrategy {
-
-    private static final Logger LOGGER = LogManager.getLogger();
-    private IClusterStateManager clusterManager;
-    private String metadataNodeId;
-    private IReplicationStrategy replicationStrategy;
-    private ICCMessageBroker messageBroker;
-    private ICCServiceContext serviceCtx;
-    private final Set<String> hotStandbyMetadataReplica = new HashSet<>();
-    private final Set<String> failedNodes = new HashSet<>();
-    private Set<String> pendingStartupCompletionNodes = new HashSet<>();
-
-    @Override
-    public synchronized void notifyNodeJoin(String nodeId) throws HyracksDataException {
-        pendingStartupCompletionNodes.add(nodeId);
-    }
-
-    @Override
-    public synchronized void notifyNodeFailure(String nodeId) throws HyracksDataException {
-        failedNodes.add(nodeId);
-        hotStandbyMetadataReplica.remove(nodeId);
-        clusterManager.updateNodePartitions(nodeId, false);
-        if (nodeId.equals(metadataNodeId)) {
-            clusterManager.updateMetadataNode(metadataNodeId, false);
-        }
-        clusterManager.refreshState();
-        if (replicationStrategy.isParticipant(nodeId)) {
-            // Notify impacted replica
-            FaultToleranceUtil.notifyImpactedReplicas(nodeId, ClusterEventType.NODE_FAILURE, clusterManager,
-                    messageBroker, replicationStrategy);
-        }
-        // If the failed node is the metadata node, ask its replicas to replay any committed jobs
-        if (nodeId.equals(metadataNodeId)) {
-            ICcApplicationContext appCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
-            int metadataPartitionId = appCtx.getMetadataProperties().getMetadataPartition().getPartitionId();
-            Set<Integer> metadataPartition = new HashSet<>(Arrays.asList(metadataPartitionId));
-            Set<Replica> activeRemoteReplicas = replicationStrategy.getRemoteReplicas(metadataNodeId).stream()
-                    .filter(replica -> !failedNodes.contains(replica.getId())).collect(Collectors.toSet());
-            //TODO Do election to identity the node with latest state
-            for (Replica replica : activeRemoteReplicas) {
-                ReplayPartitionLogsRequestMessage msg = new ReplayPartitionLogsRequestMessage(metadataPartition);
-                try {
-                    messageBroker.sendApplicationMessageToNC(msg, replica.getId());
-                } catch (Exception e) {
-                    LOGGER.log(Level.WARN, "Failed sending an application message to an NC", e);
-                    continue;
-                }
-            }
-        }
-    }
-
-    @Override
-    public IFaultToleranceStrategy from(ICCServiceContext serviceCtx, IReplicationStrategy replicationStrategy) {
-        MetadataNodeFaultToleranceStrategy ft = new MetadataNodeFaultToleranceStrategy();
-        ft.replicationStrategy = replicationStrategy;
-        ft.messageBroker = (ICCMessageBroker) serviceCtx.getMessageBroker();
-        ft.serviceCtx = serviceCtx;
-        return ft;
-    }
-
-    @Override
-    public synchronized void process(INCLifecycleMessage message) throws HyracksDataException {
-        switch (message.getType()) {
-            case REGISTRATION_TASKS_REQUEST:
-                process((RegistrationTasksRequestMessage) message);
-                break;
-            case REGISTRATION_TASKS_RESULT:
-                process((NCLifecycleTaskReportMessage) message);
-                break;
-            case REPLAY_LOGS_RESPONSE:
-                process((ReplayPartitionLogsResponseMessage) message);
-                break;
-            case METADATA_NODE_RESPONSE:
-                process((MetadataNodeResponseMessage) message);
-                break;
-            default:
-                throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MESSAGE_TYPE, message.getType().name());
-        }
-    }
-
-    @Override
-    public synchronized void bindTo(IClusterStateManager clusterManager) {
-        this.clusterManager = clusterManager;
-        this.metadataNodeId = clusterManager.getCurrentMetadataNodeId();
-    }
-
-    @Override
-    public void notifyMetadataNodeChange(String node) throws HyracksDataException {
-        if (metadataNodeId.equals(node)) {
-            return;
-        }
-        // if current metadata node is active, we need to unbind its metadata proxy object
-        if (clusterManager.isMetadataNodeActive()) {
-            MetadataNodeRequestMessage msg = new MetadataNodeRequestMessage(false);
-            try {
-                messageBroker.sendApplicationMessageToNC(msg, metadataNodeId);
-                // when the current node responses, we will bind to the new one
-                metadataNodeId = node;
-            } catch (Exception e) {
-                throw HyracksDataException.create(e);
-            }
-        } else {
-            requestMetadataNodeTakeover(node);
-        }
-    }
-
-    private synchronized void process(ReplayPartitionLogsResponseMessage msg) {
-        hotStandbyMetadataReplica.add(msg.getNodeId());
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Hot Standby Metadata Replicas: " + hotStandbyMetadataReplica);
-        }
-    }
-
-    private synchronized void process(RegistrationTasksRequestMessage msg) throws HyracksDataException {
-        final String nodeId = msg.getNodeId();
-        final SystemState state = msg.getState();
-        final boolean isParticipant = replicationStrategy.isParticipant(nodeId);
-        List<INCLifecycleTask> tasks;
-        if (!isParticipant) {
-            tasks = buildNonParticipantStartupSequence(nodeId, state);
-        } else {
-            tasks = buildParticipantStartupSequence(nodeId, state);
-        }
-        RegistrationTasksResponseMessage response = new RegistrationTasksResponseMessage(nodeId, tasks);
-        try {
-            messageBroker.sendApplicationMessageToNC(response, msg.getNodeId());
-        } catch (Exception e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    private synchronized void process(NCLifecycleTaskReportMessage msg) throws HyracksDataException {
-        final String nodeId = msg.getNodeId();
-        pendingStartupCompletionNodes.remove(nodeId);
-        if (msg.isSuccess()) {
-            // If this node failed and recovered, notify impacted replicas to reconnect to it
-            if (replicationStrategy.isParticipant(nodeId) && failedNodes.remove(nodeId)) {
-                FaultToleranceUtil.notifyImpactedReplicas(nodeId, ClusterEventType.NODE_JOIN, clusterManager,
-                        messageBroker, replicationStrategy);
-            }
-            clusterManager.updateNodePartitions(msg.getNodeId(), true);
-            if (msg.getNodeId().equals(metadataNodeId)) {
-                clusterManager.updateMetadataNode(metadataNodeId, true);
-                // When metadata node is active, it is the only hot standby replica
-                hotStandbyMetadataReplica.clear();
-                hotStandbyMetadataReplica.add(metadataNodeId);
-            }
-            clusterManager.refreshState();
-        } else {
-            LOGGER.log(Level.ERROR, msg.getNodeId() + " failed to complete startup. ", msg.getException());
-        }
-    }
-
-    private List<INCLifecycleTask> buildNonParticipantStartupSequence(String nodeId, SystemState state) {
-        final List<INCLifecycleTask> tasks = new ArrayList<>();
-        if (state == SystemState.CORRUPTED) {
-            //need to perform local recovery for node partitions
-            LocalRecoveryTask rt = new LocalRecoveryTask(Arrays.asList(clusterManager.getNodePartitions(nodeId))
-                    .stream().map(ClusterPartition::getPartitionId).collect(Collectors.toSet()));
-            tasks.add(rt);
-        }
-        tasks.add(new ExternalLibrarySetupTask(false));
-        tasks.add(new ReportLocalCountersTask());
-        tasks.add(new CheckpointTask());
-        tasks.add(new StartLifecycleComponentsTask());
-        return tasks;
-    }
-
-    private List<INCLifecycleTask> buildParticipantStartupSequence(String nodeId, SystemState state) {
-        final List<INCLifecycleTask> tasks = new ArrayList<>();
-        switch (state) {
-            case PERMANENT_DATA_LOSS:
-                if (failedNodes.isEmpty()) { //bootstrap
-                    break;
-                }
-                // If the metadata node (or replica) failed and lost its data
-                // => Metadata Remote Recovery from standby replica
-                tasks.add(getMetadataPartitionRecoveryPlan());
-                // Checkpoint after remote recovery to move node to HEALTHY state
-                tasks.add(new CheckpointTask());
-                break;
-            case CORRUPTED:
-                // If the metadata node (or replica) failed and started again without losing data => Local Recovery
-                LocalRecoveryTask rt = new LocalRecoveryTask(Arrays.asList(clusterManager.getNodePartitions(nodeId))
-                        .stream().map(ClusterPartition::getPartitionId).collect(Collectors.toSet()));
-                tasks.add(rt);
-                break;
-            case BOOTSTRAPPING:
-            case HEALTHY:
-            case RECOVERING:
-                break;
-            default:
-                break;
-        }
-        tasks.add(new StartReplicationServiceTask());
-        final boolean isMetadataNode = nodeId.equals(metadataNodeId);
-        if (isMetadataNode) {
-            tasks.add(new MetadataBootstrapTask());
-        }
-        tasks.add(new ExternalLibrarySetupTask(isMetadataNode));
-        tasks.add(new ReportLocalCountersTask());
-        tasks.add(new CheckpointTask());
-        tasks.add(new StartLifecycleComponentsTask());
-        if (isMetadataNode) {
-            tasks.add(new BindMetadataNodeTask(true));
-        }
-        return tasks;
-    }
-
-    private RemoteRecoveryTask getMetadataPartitionRecoveryPlan() {
-        if (hotStandbyMetadataReplica.isEmpty()) {
-            throw new IllegalStateException("No metadata replicas to recover from");
-        }
-        // Construct recovery plan: Node => Set of partitions to recover from it
-        Map<String, Set<Integer>> recoveryPlan = new HashMap<>();
-        // Recover metadata partition from any metadata hot standby replica
-        ICcApplicationContext appCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
-        int metadataPartitionId = appCtx.getMetadataProperties().getMetadataPartition().getPartitionId();
-        Set<Integer> metadataPartition = new HashSet<>(Arrays.asList(metadataPartitionId));
-        recoveryPlan.put(hotStandbyMetadataReplica.iterator().next(), metadataPartition);
-        return new RemoteRecoveryTask(recoveryPlan);
-    }
-
-    private void process(MetadataNodeResponseMessage response) throws HyracksDataException {
-        clusterManager.updateMetadataNode(response.getNodeId(), response.isExported());
-        if (!response.isExported()) {
-            requestMetadataNodeTakeover(metadataNodeId);
-        }
-    }
-
-    private void requestMetadataNodeTakeover(String node) throws HyracksDataException {
-        MetadataNodeRequestMessage msg = new MetadataNodeRequestMessage(true);
-        try {
-            messageBroker.sendApplicationMessageToNC(msg, node);
-        } catch (Exception e) {
-            throw HyracksDataException.create(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/NoFaultToleranceStrategy.java
index 3c5442e..49f4b32 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/NoFaultToleranceStrategy.java
@@ -32,6 +32,9 @@
 import org.apache.asterix.app.nc.task.MetadataBootstrapTask;
 import org.apache.asterix.app.nc.task.ReportLocalCountersTask;
 import org.apache.asterix.app.nc.task.StartLifecycleComponentsTask;
+import org.apache.asterix.app.nc.task.StartReplicationServiceTask;
+import org.apache.asterix.app.replication.message.MetadataNodeRequestMessage;
+import org.apache.asterix.app.replication.message.MetadataNodeResponseMessage;
 import org.apache.asterix.app.replication.message.NCLifecycleTaskReportMessage;
 import org.apache.asterix.app.replication.message.RegistrationTasksRequestMessage;
 import org.apache.asterix.app.replication.message.RegistrationTasksResponseMessage;
@@ -43,8 +46,8 @@
 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.IReplicationStrategy;
 import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
+import org.apache.asterix.metadata.MetadataManager;
 import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.client.NodeStatus;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -59,6 +62,7 @@
     private String metadataNodeId;
     private Set<String> pendingStartupCompletionNodes = new HashSet<>();
     private ICCMessageBroker messageBroker;
+    private boolean replicationEnabled;
 
     @Override
     public void notifyNodeJoin(String nodeId) throws HyracksDataException {
@@ -84,15 +88,19 @@
             case REGISTRATION_TASKS_RESULT:
                 process((NCLifecycleTaskReportMessage) message);
                 break;
+            case METADATA_NODE_RESPONSE:
+                process((MetadataNodeResponseMessage) message);
+                break;
             default:
                 throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MESSAGE_TYPE, message.getType().name());
         }
     }
 
     @Override
-    public IFaultToleranceStrategy from(ICCServiceContext serviceCtx, IReplicationStrategy replicationStrategy) {
+    public IFaultToleranceStrategy from(ICCServiceContext serviceCtx, boolean replicationEnabled) {
         NoFaultToleranceStrategy ft = new NoFaultToleranceStrategy();
         ft.messageBroker = (ICCMessageBroker) serviceCtx.getMessageBroker();
+        ft.replicationEnabled = replicationEnabled;
         return ft;
     }
 
@@ -141,10 +149,14 @@
         final List<INCLifecycleTask> tasks = new ArrayList<>();
         if (state == SystemState.CORRUPTED) {
             //need to perform local recovery for node partitions
-            LocalRecoveryTask rt = new LocalRecoveryTask(Arrays.asList(clusterManager.getNodePartitions(nodeId))
-                    .stream().map(ClusterPartition::getPartitionId).collect(Collectors.toSet()));
+            LocalRecoveryTask rt = new LocalRecoveryTask(
+                    Arrays.asList(clusterManager.getNodePartitions(nodeId)).stream()
+                            .map(ClusterPartition::getPartitionId).collect(Collectors.toSet()));
             tasks.add(rt);
         }
+        if (replicationEnabled) {
+            tasks.add(new StartReplicationServiceTask());
+        }
         if (isMetadataNode) {
             tasks.add(new MetadataBootstrapTask());
         }
@@ -168,4 +180,42 @@
         tasks.add(new ReportLocalCountersTask());
         return tasks;
     }
+
+    @Override
+    public void notifyMetadataNodeChange(String node) throws HyracksDataException {
+        if (metadataNodeId.equals(node)) {
+            return;
+        }
+        // if current metadata node is active, we need to unbind its metadata proxy object
+        if (clusterManager.isMetadataNodeActive()) {
+            MetadataNodeRequestMessage msg = new MetadataNodeRequestMessage(false);
+            try {
+                messageBroker.sendApplicationMessageToNC(msg, metadataNodeId);
+                // when the current node responses, we will bind to the new one
+                metadataNodeId = node;
+            } catch (Exception e) {
+                throw HyracksDataException.create(e);
+            }
+        } else {
+            requestMetadataNodeTakeover(node);
+        }
+    }
+
+    private void process(MetadataNodeResponseMessage response) throws HyracksDataException {
+        // rebind metadata node since it might be changing
+        MetadataManager.INSTANCE.rebindMetadataNode();
+        clusterManager.updateMetadataNode(response.getNodeId(), response.isExported());
+        if (!response.isExported()) {
+            requestMetadataNodeTakeover(metadataNodeId);
+        }
+    }
+
+    private void requestMetadataNodeTakeover(String node) throws HyracksDataException {
+        MetadataNodeRequestMessage msg = new MetadataNodeRequestMessage(true);
+        try {
+            messageBroker.sendApplicationMessageToNC(msg, node);
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+    }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NodeFailbackPlan.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NodeFailbackPlan.java
deleted file mode 100644
index ab938d2..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NodeFailbackPlan.java
+++ /dev/null
@@ -1,210 +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 java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.asterix.app.replication.message.CompleteFailbackRequestMessage;
-import org.apache.asterix.app.replication.message.PreparePartitionsFailbackRequestMessage;
-import org.apache.asterix.app.replication.message.PreparePartitionsFailbackResponseMessage;
-
-public class NodeFailbackPlan {
-
-    public enum FailbackPlanState {
-        /**
-         * Initial state while selecting the nodes that will participate
-         * in the node failback plan.
-         */
-        PREPARING,
-        /**
-         * Once a pending {@link PreparePartitionsFailbackRequestMessage} request is added,
-         * the state is changed from PREPARING to PENDING_PARTICIPANT_REPONSE to indicate
-         * a response is expected and need to wait for it.
-         */
-        PENDING_PARTICIPANT_REPONSE,
-        /**
-         * Upon receiving the last {@link PreparePartitionsFailbackResponseMessage} response,
-         * the state changes from PENDING_PARTICIPANT_REPONSE to PENDING_COMPLETION to indicate
-         * the need to send {@link CompleteFailbackRequestMessage} to the failing back node.
-         */
-        PENDING_COMPLETION,
-        /**
-         * if any of the participants fail or the failing back node itself fails during
-         * and of these states (PREPARING, PENDING_PARTICIPANT_REPONSE, PENDING_COMPLETION),
-         * the state is changed to FAILED.
-         */
-        FAILED,
-        /**
-         * if the state is FAILED, and all pending responses (if any) have been received,
-         * the state changes from FAILED to PENDING_ROLLBACK to indicate the need to revert
-         * the effects of this plan (if any).
-         */
-        PENDING_ROLLBACK
-    }
-
-    private static long planIdGenerator = 0;
-    private long planId;
-    private final String nodeId;
-    private final Set<String> participants;
-    private final Map<Integer, String> partition2nodeMap;
-    private String nodeToReleaseMetadataManager;
-    private int requestId;
-    private Map<Integer, PreparePartitionsFailbackRequestMessage> pendingRequests;
-    private FailbackPlanState state;
-
-    public static NodeFailbackPlan createPlan(String nodeId) {
-        return new NodeFailbackPlan(planIdGenerator++, nodeId);
-    }
-
-    private NodeFailbackPlan(long planId, String nodeId) {
-        this.planId = planId;
-        this.nodeId = nodeId;
-        participants = new HashSet<>();
-        partition2nodeMap = new HashMap<>();
-        pendingRequests = new HashMap<>();
-        state = FailbackPlanState.PREPARING;
-    }
-
-    public synchronized void addPartitionToFailback(int partitionId, String currentActiveNode) {
-        partition2nodeMap.put(partitionId, currentActiveNode);
-    }
-
-    public synchronized void addParticipant(String nodeId) {
-        participants.add(nodeId);
-    }
-
-    public synchronized void notifyNodeFailure(String failedNode) {
-        if (participants.contains(failedNode)) {
-            if (state == FailbackPlanState.PREPARING) {
-                state = FailbackPlanState.FAILED;
-            } else if (state == FailbackPlanState.PENDING_PARTICIPANT_REPONSE) {
-                /**
-                 * if there is any pending request from this failed node,
-                 * it should be marked as completed and the plan should be marked as failed
-                 */
-                Set<Integer> failedRequests = new HashSet<>();
-                for (PreparePartitionsFailbackRequestMessage request : pendingRequests.values()) {
-                    if (request.getNodeID().equals(failedNode)) {
-                        failedRequests.add(request.getRequestId());
-                    }
-                }
-
-                if (!failedRequests.isEmpty()) {
-                    state = FailbackPlanState.FAILED;
-                    for (Integer failedRequestId : failedRequests) {
-                        markRequestCompleted(failedRequestId);
-                    }
-                }
-            }
-        } else if (nodeId.equals(failedNode)) {
-            //if the failing back node is the failed node itself
-            state = FailbackPlanState.FAILED;
-            updateState();
-        }
-    }
-
-    public synchronized Set<Integer> getPartitionsToFailback() {
-        return new HashSet<>(partition2nodeMap.keySet());
-    }
-
-    public synchronized void addPendingRequest(PreparePartitionsFailbackRequestMessage msg) {
-        //if this is the first request
-        if (pendingRequests.size() == 0) {
-            state = FailbackPlanState.PENDING_PARTICIPANT_REPONSE;
-        }
-        pendingRequests.put(msg.getRequestId(), msg);
-    }
-
-    public synchronized void markRequestCompleted(int requestId) {
-        pendingRequests.remove(requestId);
-        updateState();
-    }
-
-    private void updateState() {
-        if (pendingRequests.size() == 0) {
-            switch (state) {
-                case PREPARING:
-                case FAILED:
-                    state = FailbackPlanState.PENDING_ROLLBACK;
-                    break;
-                case PENDING_PARTICIPANT_REPONSE:
-                    state = FailbackPlanState.PENDING_COMPLETION;
-                    break;
-                default:
-                    break;
-            }
-        }
-    }
-
-    public synchronized Set<PreparePartitionsFailbackRequestMessage> getPlanFailbackRequests() {
-        Set<PreparePartitionsFailbackRequestMessage> node2Partitions = new HashSet<>();
-        /**
-         * for each participant, construct a request with the partitions
-         * that will be failed back or flushed.
-         */
-        for (String participant : participants) {
-            Set<Integer> partitionToPrepareForFailback = new HashSet<>();
-            partition2nodeMap.forEach((key, value) -> {
-                if (value.equals(participant)) {
-                    partitionToPrepareForFailback.add(key);
-                }
-            });
-            PreparePartitionsFailbackRequestMessage msg = new PreparePartitionsFailbackRequestMessage(planId,
-                    requestId++, participant, partitionToPrepareForFailback);
-            if (participant.equals(nodeToReleaseMetadataManager)) {
-                msg.setReleaseMetadataNode(true);
-            }
-            node2Partitions.add(msg);
-        }
-        return node2Partitions;
-    }
-
-    public synchronized CompleteFailbackRequestMessage getCompleteFailbackRequestMessage() {
-        return new CompleteFailbackRequestMessage(planId, requestId++, nodeId, getPartitionsToFailback());
-    }
-
-    public String getNodeId() {
-        return nodeId;
-    }
-
-    public long getPlanId() {
-        return planId;
-    }
-
-    public void setNodeToReleaseMetadataManager(String nodeToReleaseMetadataManager) {
-        this.nodeToReleaseMetadataManager = nodeToReleaseMetadataManager;
-    }
-
-    public synchronized FailbackPlanState getState() {
-        return state;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("Plan ID: " + planId);
-        sb.append(" Failing back node: " + nodeId);
-        sb.append(" Participants: " + participants);
-        sb.append(" Partitions to Failback: " + partition2nodeMap.keySet());
-        return sb.toString();
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/CompleteFailbackRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/CompleteFailbackRequestMessage.java
deleted file mode 100644
index ad4afd0..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/CompleteFailbackRequestMessage.java
+++ /dev/null
@@ -1,94 +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.message;
-
-import java.io.IOException;
-import java.util.Set;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.messaging.api.INCMessageBroker;
-import org.apache.asterix.common.messaging.api.INcAddressedMessage;
-import org.apache.asterix.common.replication.IRemoteRecoveryManager;
-import org.apache.asterix.runtime.message.AbstractFailbackPlanMessage;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class CompleteFailbackRequestMessage extends AbstractFailbackPlanMessage implements INcAddressedMessage {
-
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = LogManager.getLogger();
-    private final Set<Integer> partitions;
-    private final String nodeId;
-
-    public CompleteFailbackRequestMessage(long planId, int requestId, String nodeId, Set<Integer> partitions) {
-        super(planId, requestId);
-        this.nodeId = nodeId;
-        this.partitions = partitions;
-    }
-
-    public Set<Integer> getPartitions() {
-        return partitions;
-    }
-
-    public String getNodeId() {
-        return nodeId;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append(CompleteFailbackRequestMessage.class.getSimpleName());
-        sb.append(" Plan ID: " + planId);
-        sb.append(" Node ID: " + nodeId);
-        sb.append(" Partitions: " + partitions);
-        return sb.toString();
-    }
-
-    @Override
-    public void handle(INcApplicationContext appContext) throws HyracksDataException, InterruptedException {
-        INCMessageBroker broker = (INCMessageBroker) appContext.getServiceContext().getMessageBroker();
-        HyracksDataException hde = null;
-        try {
-            IRemoteRecoveryManager remoteRecoeryManager = appContext.getRemoteRecoveryManager();
-            remoteRecoeryManager.completeFailbackProcess();
-        } catch (IOException | InterruptedException e) {
-            LOGGER.log(Level.ERROR, "Failure during completion of failback process", e);
-            hde = HyracksDataException.create(e);
-        } finally {
-            CompleteFailbackResponseMessage reponse =
-                    new CompleteFailbackResponseMessage(planId, requestId, partitions);
-            try {
-                broker.sendMessageToCC(reponse);
-            } catch (Exception e) {
-                LOGGER.log(Level.ERROR, "Failure sending message to CC", e);
-                hde = HyracksDataException.suppress(hde, e);
-            }
-        }
-        if (hde != null) {
-            throw hde;
-        }
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.COMPLETE_FAILBACK_REQUEST;
-    }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/CompleteFailbackResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/CompleteFailbackResponseMessage.java
deleted file mode 100644
index 0c5678f..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/CompleteFailbackResponseMessage.java
+++ /dev/null
@@ -1,61 +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.message;
-
-import java.util.Set;
-
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
-import org.apache.asterix.runtime.message.AbstractFailbackPlanMessage;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class CompleteFailbackResponseMessage extends AbstractFailbackPlanMessage implements ICcAddressedMessage {
-
-    private static final long serialVersionUID = 1L;
-    private final Set<Integer> partitions;
-
-    public CompleteFailbackResponseMessage(long planId, int requestId, Set<Integer> partitions) {
-        super(planId, requestId);
-        this.partitions = partitions;
-    }
-
-    public Set<Integer> getPartitions() {
-        return partitions;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append(CompleteFailbackResponseMessage.class.getSimpleName());
-        sb.append(" Plan ID: " + planId);
-        sb.append(" Partitions: " + partitions);
-        return sb.toString();
-    }
-
-    @Override
-    public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        ((CcApplicationContext) appCtx).getFaultToleranceStrategy().process(this);
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.COMPLETE_FAILBACK_RESPONSE;
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/PreparePartitionsFailbackRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/PreparePartitionsFailbackRequestMessage.java
deleted file mode 100644
index 6b85050..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/PreparePartitionsFailbackRequestMessage.java
+++ /dev/null
@@ -1,122 +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.message;
-
-import java.rmi.RemoteException;
-import java.util.Set;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.messaging.api.INCMessageBroker;
-import org.apache.asterix.common.messaging.api.INcAddressedMessage;
-import org.apache.asterix.runtime.message.AbstractFailbackPlanMessage;
-import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class PreparePartitionsFailbackRequestMessage extends AbstractFailbackPlanMessage
-        implements INcAddressedMessage {
-
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = LogManager.getLogger();
-    private final Set<Integer> partitions;
-    private boolean releaseMetadataNode = false;
-    private final String nodeID;
-
-    public PreparePartitionsFailbackRequestMessage(long planId, int requestId, String nodeId, Set<Integer> partitions) {
-        super(planId, requestId);
-        this.nodeID = nodeId;
-        this.partitions = partitions;
-    }
-
-    public Set<Integer> getPartitions() {
-        return partitions;
-    }
-
-    public boolean isReleaseMetadataNode() {
-        return releaseMetadataNode;
-    }
-
-    public void setReleaseMetadataNode(boolean releaseMetadataNode) {
-        this.releaseMetadataNode = releaseMetadataNode;
-    }
-
-    public String getNodeID() {
-        return nodeID;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append(PreparePartitionsFailbackRequestMessage.class.getSimpleName());
-        sb.append(" Plan ID: " + planId);
-        sb.append(" Partitions: " + partitions);
-        sb.append(" releaseMetadataNode: " + releaseMetadataNode);
-        return sb.toString();
-    }
-
-    @Override
-    public void handle(INcApplicationContext appContext) throws HyracksDataException, InterruptedException {
-        INCMessageBroker broker = (INCMessageBroker) appContext.getServiceContext().getMessageBroker();
-        /**
-         * if the metadata partition will be failed back
-         * we need to flush and close all datasets including metadata datasets
-         * otherwise we need to close all non-metadata datasets and flush metadata datasets
-         * so that their memory components will be copied to the failing back node
-         */
-        if (releaseMetadataNode) {
-            appContext.getDatasetLifecycleManager().closeAllDatasets();
-            //remove the metadata node stub from RMI registry
-            try {
-                appContext.unexportMetadataNodeStub();
-            } catch (RemoteException e) {
-                LOGGER.log(Level.ERROR, "Failed unexporting metadata stub", e);
-                throw HyracksDataException.create(e);
-            }
-        } else {
-            //close all non-metadata datasets
-            appContext.getDatasetLifecycleManager().closeUserDatasets();
-            //flush the remaining metadata datasets that were not closed
-            appContext.getDatasetLifecycleManager().flushAllDatasets();
-        }
-
-        //mark the partitions to be closed as inactive
-        PersistentLocalResourceRepository localResourceRepo =
-                (PersistentLocalResourceRepository) appContext.getLocalResourceRepository();
-        for (Integer partitionId : partitions) {
-            localResourceRepo.addInactivePartition(partitionId);
-        }
-
-        //send response after partitions prepared for failback
-        PreparePartitionsFailbackResponseMessage reponse =
-                new PreparePartitionsFailbackResponseMessage(planId, requestId, partitions);
-        try {
-            broker.sendMessageToCC(reponse);
-        } catch (Exception e) {
-            LOGGER.log(Level.ERROR, "Failed sending message to cc", e);
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.PREPARE_FAILBACK_REQUEST;
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/PreparePartitionsFailbackResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/PreparePartitionsFailbackResponseMessage.java
deleted file mode 100644
index bea1039..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/PreparePartitionsFailbackResponseMessage.java
+++ /dev/null
@@ -1,58 +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.message;
-
-import java.util.Set;
-
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
-import org.apache.asterix.runtime.message.AbstractFailbackPlanMessage;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class PreparePartitionsFailbackResponseMessage extends AbstractFailbackPlanMessage
-        implements ICcAddressedMessage {
-
-    private static final long serialVersionUID = 1L;
-    private final Set<Integer> partitions;
-
-    public PreparePartitionsFailbackResponseMessage(long planId, int requestId, Set<Integer> partitions) {
-        super(planId, requestId);
-        this.partitions = partitions;
-    }
-
-    public Set<Integer> getPartitions() {
-        return partitions;
-    }
-
-    @Override
-    public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        ((CcApplicationContext) appCtx).getFaultToleranceStrategy().process(this);
-    }
-
-    @Override
-    public String toString() {
-        return PreparePartitionsFailbackResponseMessage.class.getSimpleName() + " " + partitions.toString();
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.PREPARE_FAILBACK_RESPONSE;
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/ReplayPartitionLogsRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/ReplayPartitionLogsRequestMessage.java
deleted file mode 100644
index beac2b5..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/ReplayPartitionLogsRequestMessage.java
+++ /dev/null
@@ -1,63 +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.message;
-
-import java.util.Set;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.messaging.api.INCMessageBroker;
-import org.apache.asterix.common.messaging.api.INcAddressedMessage;
-import org.apache.asterix.common.replication.INCLifecycleMessage;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.control.nc.NodeControllerService;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class ReplayPartitionLogsRequestMessage implements INCLifecycleMessage, INcAddressedMessage {
-
-    private static final Logger LOGGER = LogManager.getLogger();
-    private static final long serialVersionUID = 1L;
-    private final Set<Integer> partitions;
-
-    public ReplayPartitionLogsRequestMessage(Set<Integer> partitions) {
-        this.partitions = partitions;
-    }
-
-    @Override
-    public void handle(INcApplicationContext appContext) throws HyracksDataException, InterruptedException {
-        NodeControllerService ncs = (NodeControllerService) appContext.getServiceContext().getControllerService();
-        // Replay the logs for these partitions and flush any impacted dataset
-        appContext.getRemoteRecoveryManager().replayReplicaPartitionLogs(partitions, true);
-
-        INCMessageBroker broker = (INCMessageBroker) ncs.getContext().getMessageBroker();
-        ReplayPartitionLogsResponseMessage reponse = new ReplayPartitionLogsResponseMessage(ncs.getId(), partitions);
-        try {
-            broker.sendMessageToCC(reponse);
-        } catch (Exception e) {
-            LOGGER.log(Level.ERROR, "Failed sending message to cc", e);
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.REPLAY_LOGS_REQUEST;
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/ReplayPartitionLogsResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/ReplayPartitionLogsResponseMessage.java
deleted file mode 100644
index e05fd47..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/ReplayPartitionLogsResponseMessage.java
+++ /dev/null
@@ -1,57 +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.message;
-
-import java.util.Set;
-
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
-import org.apache.asterix.common.replication.INCLifecycleMessage;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class ReplayPartitionLogsResponseMessage implements INCLifecycleMessage, ICcAddressedMessage {
-
-    private static final long serialVersionUID = 1L;
-    private final Set<Integer> partitions;
-    private final String nodeId;
-
-    public ReplayPartitionLogsResponseMessage(String nodeId, Set<Integer> partitions) {
-        this.partitions = partitions;
-        this.nodeId = nodeId;
-    }
-
-    @Override
-    public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        ((CcApplicationContext) appCtx).getFaultToleranceStrategy().process(this);
-    }
-
-    public Set<Integer> getPartitions() {
-        return partitions;
-    }
-
-    public String getNodeId() {
-        return nodeId;
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.REPLAY_LOGS_RESPONSE;
-    }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/TakeoverPartitionsRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/TakeoverPartitionsRequestMessage.java
deleted file mode 100644
index 86be516..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/TakeoverPartitionsRequestMessage.java
+++ /dev/null
@@ -1,108 +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.message;
-
-import java.io.IOException;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.messaging.api.INCMessageBroker;
-import org.apache.asterix.common.messaging.api.INcAddressedMessage;
-import org.apache.asterix.common.replication.INCLifecycleMessage;
-import org.apache.asterix.common.replication.IRemoteRecoveryManager;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class TakeoverPartitionsRequestMessage implements INCLifecycleMessage, INcAddressedMessage {
-
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = LogManager.getLogger();
-    private final Integer[] partitions;
-    private final long requestId;
-    private final String nodeId;
-
-    public TakeoverPartitionsRequestMessage(long requestId, String nodeId, Integer[] partitionsToTakeover) {
-        this.requestId = requestId;
-        this.nodeId = nodeId;
-        this.partitions = partitionsToTakeover;
-    }
-
-    public Integer[] getPartitions() {
-        return partitions;
-    }
-
-    public long getRequestId() {
-        return requestId;
-    }
-
-    public String getNodeId() {
-        return nodeId;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append(TakeoverPartitionsRequestMessage.class.getSimpleName());
-        sb.append(" Request ID: " + requestId);
-        sb.append(" Node ID: " + nodeId);
-        sb.append(" Partitions: ");
-        for (Integer partitionId : partitions) {
-            sb.append(partitionId + ",");
-        }
-        //remove last comma
-        sb.charAt(sb.length() - 1);
-        return sb.toString();
-    }
-
-    @Override
-    public void handle(INcApplicationContext appContext) throws HyracksDataException, InterruptedException {
-        INCMessageBroker broker = (INCMessageBroker) appContext.getServiceContext().getMessageBroker();
-        //if the NC is shutting down, it should ignore takeover partitions request
-        if (!appContext.isShuttingdown()) {
-            HyracksDataException hde = null;
-            try {
-                IRemoteRecoveryManager remoteRecoeryManager = appContext.getRemoteRecoveryManager();
-                remoteRecoeryManager.takeoverPartitons(partitions);
-            } catch (IOException | ACIDException e) {
-                LOGGER.log(Level.ERROR, "Failure taking over partitions", e);
-                hde = HyracksDataException.suppress(hde, e);
-            } finally {
-                //send response after takeover is completed
-                TakeoverPartitionsResponseMessage reponse = new TakeoverPartitionsResponseMessage(requestId,
-                        appContext.getTransactionSubsystem().getId(), partitions);
-                try {
-                    broker.sendMessageToCC(reponse);
-                } catch (Exception e) {
-                    LOGGER.log(Level.ERROR, "Failure taking over partitions", e);
-                    hde = HyracksDataException.suppress(hde, e);
-                }
-            }
-            if (hde != null) {
-                throw hde;
-            }
-        }
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.TAKEOVER_PARTITION_REQUEST;
-    }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/TakeoverPartitionsResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/TakeoverPartitionsResponseMessage.java
deleted file mode 100644
index d9484f9..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/TakeoverPartitionsResponseMessage.java
+++ /dev/null
@@ -1,66 +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.message;
-
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
-import org.apache.asterix.common.replication.INCLifecycleMessage;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class TakeoverPartitionsResponseMessage implements INCLifecycleMessage, ICcAddressedMessage {
-
-    private static final long serialVersionUID = 1L;
-    private final Integer[] partitions;
-    private final String nodeId;
-    private final long requestId;
-
-    public TakeoverPartitionsResponseMessage(long requestId, String nodeId, Integer[] partitionsToTakeover) {
-        this.requestId = requestId;
-        this.nodeId = nodeId;
-        this.partitions = partitionsToTakeover;
-    }
-
-    public Integer[] getPartitions() {
-        return partitions;
-    }
-
-    public String getNodeId() {
-        return nodeId;
-    }
-
-    public long getRequestId() {
-        return requestId;
-    }
-
-    @Override
-    public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        ((CcApplicationContext) appCtx).getFaultToleranceStrategy().process(this);
-    }
-
-    @Override
-    public String toString() {
-        return TakeoverPartitionsResponseMessage.class.getSimpleName();
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.TAKEOVER_PARTITION_RESPONSE;
-    }
-}
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 622e28f..c2a0fc1 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
@@ -142,9 +142,8 @@
         ILibraryManager libraryManager = new ExternalLibraryManager();
         ReplicationProperties repProp = new ReplicationProperties(
                 PropertiesAccessor.getInstance(ccServiceCtx.getAppConfig()));
-        IReplicationStrategy repStrategy = ReplicationStrategyFactory.create(repProp.getReplicationStrategy(), repProp,
-                getConfigManager());
-        IFaultToleranceStrategy ftStrategy = FaultToleranceStrategyFactory.create(ccServiceCtx, repProp, repStrategy);
+        IFaultToleranceStrategy ftStrategy =
+                FaultToleranceStrategyFactory.create(ccServiceCtx, repProp.isReplicationEnabled());
         ExternalLibraryUtils.setUpExternaLibraries(libraryManager, false);
         componentProvider = new StorageComponentProvider();
         GlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
deleted file mode 100644
index ddd0967..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
+++ /dev/null
@@ -1,71 +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.util;
-
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import org.apache.asterix.common.cluster.IClusterStateManager;
-import org.apache.asterix.common.messaging.api.ICCMessageBroker;
-import org.apache.asterix.common.replication.IReplicationStrategy;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.runtime.message.ReplicaEventMessage;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
-import org.apache.hyracks.api.config.IOption;
-import org.apache.hyracks.control.common.controllers.NCConfig;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class FaultToleranceUtil {
-
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    private FaultToleranceUtil() {
-        throw new AssertionError();
-    }
-
-    public static void notifyImpactedReplicas(String nodeId, ClusterEventType event,
-            IClusterStateManager clusterManager, ICCMessageBroker messageBroker,
-            IReplicationStrategy replicationStrategy) {
-        List<String> primaryRemoteReplicas = replicationStrategy.getRemotePrimaryReplicas(nodeId).stream()
-                .map(Replica::getId).collect(Collectors.toList());
-        String nodeIdAddress = StringUtils.EMPTY;
-        int nodePort = -1;
-        Map<String, Map<IOption, Object>> activeNcConfiguration = clusterManager.getActiveNcConfiguration();
-
-        // In case the node joined with a new IP address, we need to send it to the other replicas
-        if (event == ClusterEventType.NODE_JOIN) {
-            nodeIdAddress = (String) activeNcConfiguration.get(nodeId).get(NCConfig.Option.REPLICATION_PUBLIC_ADDRESS);
-            nodePort = (int) activeNcConfiguration.get(nodeId).get(NCConfig.Option.REPLICATION_PUBLIC_PORT);
-        }
-        ReplicaEventMessage msg = new ReplicaEventMessage(nodeId, nodeIdAddress, nodePort, event);
-        for (String replica : primaryRemoteReplicas) {
-            // If the remote replica is alive, send the event
-            if (activeNcConfiguration.containsKey(replica)) {
-                try {
-                    messageBroker.sendApplicationMessageToNC(msg, replica);
-                } catch (Exception e) {
-                    LOGGER.warn("Failed sending an application message to an NC", e);
-                }
-            }
-        }
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/resources/cc-rep.conf b/asterixdb/asterix-app/src/main/resources/cc-rep.conf
index 9c093dc..bfca677 100644
--- a/asterixdb/asterix-app/src/main/resources/cc-rep.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc-rep.conf
@@ -48,5 +48,4 @@
 [common]
 log.level = INFO
 replication.enabled=true
-replication.strategy=metadata_only
-replication.factor=2
+replication.strategy=all
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.1.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.1.sto.cmd
index b750c64..1947749 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.1.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /addReplica 2 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.10.post.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.10.post.http
index b750c64..a3ea801 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.10.post.http
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/partition/master?partition=3&node=asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.11.pollget.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.11.pollget.http
index b750c64..32e2f78 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.11.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.12.query.sqlpp
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.12.query.sqlpp
index b750c64..cd777cf 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.12.query.sqlpp
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+
+use tpch;
+
+select value count(*) from LineItem;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.13.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.13.sto.cmd
index b750c64..1e192f4 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.13.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /removeReplica 2 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.14.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.14.sto.cmd
index b750c64..530432f 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.14.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /removeReplica 3 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.2.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.2.sto.cmd
index b750c64..f3810f8 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.2.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /addReplica 3 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.3.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.3.pollget.http
index b750c64..4ea16d7 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.3.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc2 /admin/storage/partition/2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.4.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.4.pollget.http
index b750c64..22558bc 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.4.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc2 /admin/storage/partition/3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.5.ddl.sqlpp
new file mode 100644
index 0000000..071a40f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.5.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.6.update.sqlpp
similarity index 82%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.6.update.sqlpp
index b750c64..e53f462 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.6.update.sqlpp
@@ -16,4 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+
+use tpch;
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),
+      (`format`=`delimited-text`),(`delimiter`=`|`));
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.7.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.7.sto.cmd
index b750c64..389cf68 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.7.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /promote 2 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.8.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.8.sto.cmd
index b750c64..257f26a 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.8.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /promote 3 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.9.post.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.9.post.http
index b750c64..36e1d00 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.9.post.http
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/partition/master?partition=2&node=asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.1.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.1.sto.cmd
index b750c64..7ddaa20 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.1.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /addReplica 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.10.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.10.sto.cmd
index b750c64..71621ac 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.10.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /removeReplica 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.2.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.2.pollget.http
index b750c64..6867a5d 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.2.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc1 /admin/storage/partition/0
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.3.ddl.sqlpp
similarity index 89%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.3.ddl.sqlpp
index b750c64..15bc3c5 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.3.ddl.sqlpp
@@ -16,4 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+CREATE TYPE MyType AS {
+  id : int
+};
+
+CREATE DATASET ds_1(MyType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.4.get.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.4.get.http
index b750c64..09ddf42 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.4.get.http
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+/connector?dataverseName=Metadata&datasetName=Dataset
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.5.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.5.sto.cmd
index b750c64..a5753f0 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.5.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /promote 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.6.post.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.6.post.http
index b750c64..2e8fc63 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.6.post.http
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/partition/master?partition=0&node=asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.7.post.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.7.post.http
index b750c64..e8dca0b 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.7.post.http
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/metadataNode?node=asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.8.pollget.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.8.pollget.http
index b750c64..32e2f78 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.8.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.9.query.sqlpp
similarity index 90%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.9.query.sqlpp
index b750c64..a612cbb 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.9.query.sqlpp
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+SELECT value count(*)
+FROM Metadata.`Dataset`
+WHERE DatasetName = 'ds_1';
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.1.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.1.sto.cmd
index b750c64..1947749 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.1.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /addReplica 2 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.10.post.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.10.post.http
index b750c64..a3ea801 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.10.post.http
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/partition/master?partition=3&node=asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.11.pollget.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.11.pollget.http
index b750c64..32e2f78 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.11.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.12.query.sqlpp
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.12.query.sqlpp
index b750c64..4951292 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.12.query.sqlpp
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+
+use tpch;
+
+select value count(*) from InMemoryLineItem;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.13.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.13.sto.cmd
index b750c64..1e192f4 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.13.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /removeReplica 2 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.14.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.14.sto.cmd
index b750c64..530432f 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.14.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /removeReplica 3 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.2.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.2.sto.cmd
index b750c64..f3810f8 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.2.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /addReplica 3 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.3.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.3.pollget.http
index b750c64..4ea16d7 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.3.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc2 /admin/storage/partition/2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.4.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.4.pollget.http
index b750c64..22558bc 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.4.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc2 /admin/storage/partition/3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.5.ddl.sqlpp
new file mode 100644
index 0000000..4dbf9e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.5.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+create dataset InMemoryLineItem(LineItemType) primary key l_orderkey,l_linenumber;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.6.update.sqlpp
similarity index 77%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.6.update.sqlpp
index b750c64..8962e6d 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.6.update.sqlpp
@@ -16,4 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+
+use tpch;
+
+load dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),
+      (`format`=`delimited-text`),(`delimiter`=`|`));
+
+insert into InMemoryLineItem (select value ls from LineItem ls);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.7.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.7.sto.cmd
index b750c64..389cf68 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.7.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /promote 2 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.8.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.8.sto.cmd
index b750c64..257f26a 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.8.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /promote 3 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.9.post.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.9.post.http
index b750c64..36e1d00 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.9.post.http
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/partition/master?partition=2&node=asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.12.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.12.sto.cmd
index b750c64..71621ac 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.12.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /removeReplica 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/replication.xml b/asterixdb/asterix-app/src/test/resources/runtimets/replication.xml
index c3dfb3c..79c9f5c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/replication.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/replication.xml
@@ -29,5 +29,20 @@
         <output-dir compare="Text">metadata_failover</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="replication">
+      <compilation-unit name="bulkload">
+        <output-dir compare="Text">bulkload</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="replication">
+      <compilation-unit name="mem_component_recovery">
+        <output-dir compare="Text">mem_component_recovery</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="replication">
+      <compilation-unit name="flushed_component">
+        <output-dir compare="Text">flushed_component</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
 </test-suite>
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index 53d6563..85b121e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -22,14 +22,12 @@
     "metadata\.node" : "asterix_nc1",
     "metadata\.registration\.timeout\.secs" : 60,
     "replication\.enabled" : false,
-    "replication\.factor" : 3,
+    "replication\.factor" : 2,
     "replication\.log\.batchsize" : 4096,
     "replication\.log\.buffer\.numpages" : 8,
     "replication\.log\.buffer\.pagesize" : 131072,
-    "replication\.max\.remote\.recovery\.attempts" : 5,
-    "replication\.port" : 2000,
-    "replication\.strategy" : "chained_declustering",
-    "replication\.timeout" : 15,
+    "replication\.strategy" : "none",
+    "replication\.timeout" : 30,
     "txn\.commitprofiler\.enabled" : false,
     "txn\.commitprofiler\.reportinterval" : 5,
     "txn\.job\.recovery\.memorysize" : 67108864,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index 1160758..7017b06 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -22,14 +22,12 @@
     "metadata\.node" : "asterix_nc1",
     "metadata\.registration\.timeout\.secs" : 60,
     "replication\.enabled" : false,
-    "replication\.factor" : 3,
+    "replication\.factor" : 2,
     "replication\.log\.batchsize" : 4096,
     "replication\.log\.buffer\.numpages" : 8,
     "replication\.log\.buffer\.pagesize" : 131072,
-    "replication\.max\.remote\.recovery\.attempts" : 5,
-    "replication\.port" : 2000,
-    "replication\.strategy" : "chained_declustering",
-    "replication\.timeout" : 15,
+    "replication\.strategy" : "none",
+    "replication\.timeout" : 30,
     "txn\.commitprofiler\.enabled" : false,
     "txn\.commitprofiler\.reportinterval" : 5,
     "txn\.job\.recovery\.memorysize" : 67108864,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index d1c9fe0..00e4d67 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -22,14 +22,12 @@
     "metadata\.node" : "asterix_nc1",
     "metadata\.registration\.timeout\.secs" : 60,
     "replication\.enabled" : false,
-    "replication\.factor" : 3,
+    "replication\.factor" : 2,
     "replication\.log\.batchsize" : 4096,
     "replication\.log\.buffer\.numpages" : 8,
     "replication\.log\.buffer\.pagesize" : 131072,
-    "replication\.max\.remote\.recovery\.attempts" : 5,
-    "replication\.port" : 2000,
-    "replication\.strategy" : "chained_declustering",
-    "replication\.timeout" : 15,
+    "replication\.strategy" : "none",
+    "replication\.timeout" : 30,
     "txn\.commitprofiler\.enabled" : false,
     "txn\.commitprofiler\.reportinterval" : 5,
     "txn\.job\.recovery\.memorysize" : 67108864,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.10.adm
new file mode 100644
index 0000000..3a8dfe0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.10.adm
@@ -0,0 +1 @@
+6005
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.3.adm
new file mode 100644
index 0000000..a6afbb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.3.adm
@@ -0,0 +1,7 @@
+[ {
+  "partition" : 2,
+  "replicas" : [ {
+    "location" : "127.0.0.1:2001",
+    "status" : "IN_SYNC"
+  } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.4.adm
new file mode 100644
index 0000000..53648a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.4.adm
@@ -0,0 +1,7 @@
+[ {
+  "partition" : 3,
+  "replicas" : [ {
+    "location" : "127.0.0.1:2001",
+    "status" : "IN_SYNC"
+  } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.7.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.7.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.8.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.8.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.9.adm
similarity index 90%
rename from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.9.adm
index c0697b7..c382703 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.9.adm
@@ -19,7 +19,7 @@
     },
     "2" : {
       "active" : true,
-      "activeNodeId" : "asterix_nc2",
+      "activeNodeId" : "asterix_nc1",
       "iodeviceNum" : 0,
       "nodeId" : "asterix_nc2",
       "partitionId" : 2,
@@ -27,7 +27,7 @@
     },
     "3" : {
       "active" : true,
-      "activeNodeId" : "asterix_nc2",
+      "activeNodeId" : "asterix_nc1",
       "iodeviceNum" : 1,
       "nodeId" : "asterix_nc2",
       "partitionId" : 3,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.2.adm
new file mode 100644
index 0000000..7e92c87
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.2.adm
@@ -0,0 +1,7 @@
+[ {
+  "partition" : 0,
+  "replicas" : [ {
+    "location" : "127.0.0.1:2002",
+    "status" : "IN_SYNC"
+  } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.4.adm
new file mode 100644
index 0000000..f84d77a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.4.adm
@@ -0,0 +1 @@
+{"keys":"DataverseName,DatasetName","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"DatasetRecordType","open":true,"fields":[{"DataverseName":{"type":"AString"}},{"DatasetName":{"type":"AString"}},{"DatatypeDataverseName":{"type":"AString"}},{"DatatypeName":{"type":"AString"}},{"DatasetType":{"type":"AString"}},{"GroupName":{"type":"AString"}},{"CompactionPolicy":{"type":"AString"}},{"CompactionPolicyProperties":{"type":"org.apache.asterix.om.types.AOrderedListType","item-type":{"type":"org.apache.asterix.om.types.ARecordType","name":"DatasetRecordType_CompactionPolicyProperties_Item","open":true,"fields":[{"Name":{"type":"AString"}},{"Value":{"type":"AString"}}]}}},{"InternalDetails":{"type":"org.apache.asterix.om.types.AUnionType","fields":[{"type":"org.apache.asterix.om.types.ARecordType","name":"DatasetRecordType_InternalDetails","open":true,"fields":[{"FileStructure":{"type":"AString"}},{"PartitioningStrategy":{"type":"AString"}},{"PartitioningKey":{"type":"org.apache.asterix.om.types.AOrderedListType","item-type":{"type":"org.apache.asterix.om.types.AOrderedListType","item-type":{"type":"AString"}}}},{"PrimaryKey":{"type":"org.apache.asterix.om.types.AOrderedListType","item-type":{"type":"org.apache.asterix.om.types.AOrderedListType","item-type":{"type":"AString"}}}},{"Autogenerated":{"type":"ABoolean"}}]},{"type":"ANULL"},{"type":"AMISSING"}]}},{"ExternalDetails":{"type":"org.apache.asterix.om.types.AUnionType","fields":[{"type":"org.apache.asterix.om.types.ARecordType","name":"DatasetRecordType_ExternalDetails","open":true,"fields":[{"DatasourceAdapter":{"type":"AString"}},{"Properties":{"type":"org.apache.asterix.om.types.AOrderedListType","item-type":{"type":"org.apache.asterix.om.types.ARecordType","name":"DatasetRecordType_ExternalDetails_Properties_Item","open":true,"fields":[{"Name":{"type":"AString"}},{"Value":{"type":"AString"}}]}}},{"LastRefreshTime":{"type":"ADateTime"}},{"TransactionState":{"type":"AInt32"}}]},{"type":"ANULL"},{"type":"AMISSING"}]}},{"Hints":{"type":"org.apache.asterix.om.types.AUnorderedListType","item-type":{"type":"org.apache.asterix.om.types.ARecordType","name":"DatasetRecordType_Hints_Item","open":true,"fields":[{"Name":{"type":"AString"}},{"Value":{"type":"AString"}}]}}},{"Timestamp":{"type":"AString"}},{"DatasetId":{"type":"AInt32"}},{"PendingOp":{"type":"AInt32"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/Metadata/Dataset/0/Dataset"}]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.6.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.6.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.7.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.7.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.8.adm
similarity index 94%
rename from asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.8.adm
index 15c200c..fa5cfb4 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.8.adm
@@ -11,7 +11,7 @@
     },
     "1" : {
       "active" : true,
-      "activeNodeId" : "asterix_nc2",
+      "activeNodeId" : "asterix_nc1",
       "iodeviceNum" : 1,
       "nodeId" : "asterix_nc1",
       "partitionId" : 1,
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/metadata_node/metadata_node.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.9.adm
similarity index 100%
copy from asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/metadata_node/metadata_node.4.adm
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.9.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.10.adm
new file mode 100644
index 0000000..3a8dfe0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.10.adm
@@ -0,0 +1 @@
+6005
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.3.adm
new file mode 100644
index 0000000..a6afbb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.3.adm
@@ -0,0 +1,7 @@
+[ {
+  "partition" : 2,
+  "replicas" : [ {
+    "location" : "127.0.0.1:2001",
+    "status" : "IN_SYNC"
+  } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.4.adm
new file mode 100644
index 0000000..53648a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.4.adm
@@ -0,0 +1,7 @@
+[ {
+  "partition" : 3,
+  "replicas" : [ {
+    "location" : "127.0.0.1:2001",
+    "status" : "IN_SYNC"
+  } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.7.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.7.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.8.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.8.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.9.adm
similarity index 90%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.9.adm
index c0697b7..c382703 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.9.adm
@@ -19,7 +19,7 @@
     },
     "2" : {
       "active" : true,
-      "activeNodeId" : "asterix_nc2",
+      "activeNodeId" : "asterix_nc1",
       "iodeviceNum" : 0,
       "nodeId" : "asterix_nc2",
       "partitionId" : 2,
@@ -27,7 +27,7 @@
     },
     "3" : {
       "active" : true,
-      "activeNodeId" : "asterix_nc2",
+      "activeNodeId" : "asterix_nc1",
       "iodeviceNum" : 1,
       "nodeId" : "asterix_nc2",
       "partitionId" : 3,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/metadata_failover/metadata_failover.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/metadata_failover/metadata_failover.11.adm
index 56a6051..d0138cb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/metadata_failover/metadata_failover.11.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/metadata_failover/metadata_failover.11.adm
@@ -1 +1,3 @@
-1
\ No newline at end of file
+1
+
+
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
index 28be6fa..8a83c7b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
@@ -24,8 +24,6 @@
 
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.replication.IRemoteRecoveryManager;
-import org.apache.asterix.common.replication.IReplicaResourcesManager;
 import org.apache.asterix.common.replication.IReplicationChannel;
 import org.apache.asterix.common.replication.IReplicationManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
@@ -80,10 +78,6 @@
 
     Object getActiveManager();
 
-    IRemoteRecoveryManager getRemoteRecoveryManager();
-
-    IReplicaResourcesManager getReplicaResourcesManager();
-
     IReplicationManager getReplicationManager();
 
     IReplicationChannel getReplicationChannel();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
index 6e7d58b..3455774 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
@@ -18,43 +18,34 @@
  */
 package org.apache.asterix.common.config;
 
-import java.util.List;
-import org.apache.hyracks.api.config.IApplicationConfig;
+import static org.apache.hyracks.control.common.config.OptionTypes.BOOLEAN;
+import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER;
+import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER_BYTE_UNIT;
+import static org.apache.hyracks.control.common.config.OptionTypes.LONG;
+import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
+
+import java.util.concurrent.TimeUnit;
+
 import org.apache.hyracks.api.config.IOption;
 import org.apache.hyracks.api.config.IOptionType;
 import org.apache.hyracks.api.config.Section;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.util.StorageUtil;
 import org.apache.hyracks.util.StorageUtil.StorageUnit;
 
-import static org.apache.hyracks.control.common.config.OptionTypes.*;
-
 public class ReplicationProperties extends AbstractProperties {
 
     public enum Option implements IOption {
-        REPLICATION_MAX_REMOTE_RECOVERY_ATTEMPTS(
-                INTEGER,
-                5,
-                "The maximum number of times to attempt to recover from a replica on failure before giving up"),
-        REPLICATION_LOG_BUFFER_PAGESIZE(
-                INTEGER_BYTE_UNIT,
-                StorageUtil.getIntSizeInBytes(128, StorageUnit.KILOBYTE),
+        REPLICATION_LOG_BUFFER_PAGESIZE(INTEGER_BYTE_UNIT, StorageUtil.getIntSizeInBytes(128, StorageUnit.KILOBYTE),
                 "The size in bytes of each log buffer page"),
         REPLICATION_LOG_BUFFER_NUMPAGES(INTEGER, 8, "The number of log buffer pages"),
-        REPLICATION_LOG_BATCHSIZE(
-                INTEGER_BYTE_UNIT,
-                StorageUtil.getIntSizeInBytes(4, StorageUnit.KILOBYTE),
+        REPLICATION_LOG_BATCHSIZE(INTEGER_BYTE_UNIT, StorageUtil.getIntSizeInBytes(4, StorageUnit.KILOBYTE),
                 "The size in bytes to replicate in each batch"),
-        REPLICATION_TIMEOUT(
-                INTEGER,
-                REPLICATION_TIME_OUT_DEFAULT,
-                "The time in seconds to timeout when trying to contact a replica, before assuming it is dead"),
-
+        REPLICATION_TIMEOUT(LONG, TimeUnit.SECONDS.toSeconds(30),
+                "The time in seconds to timeout waiting for master or replica to ack"),
         REPLICATION_ENABLED(BOOLEAN, false, "Whether or not data replication is enabled"),
-        REPLICATION_FACTOR(INTEGER, 3, "Number of node controller faults to tolerate with replication"),
-        REPLICATION_STRATEGY(STRING, "chained_declustering", "Replication strategy to choose"),
-        REPLICATION_PORT(INTEGER, 2000, "port on which to run replication related communications"),;
+        REPLICATION_FACTOR(INTEGER, 2, "Number of replicas (backups) to maintain per master replica"),
+        REPLICATION_STRATEGY(STRING, "none", "Replication strategy to choose");
 
         private final IOptionType type;
         private final Object defaultValue;
@@ -85,40 +76,16 @@
         public Object defaultValue() {
             return defaultValue;
         }
-
-        @Override
-        public Object get(IApplicationConfig config) {
-            switch (this) {
-                case REPLICATION_TIMEOUT:
-                    return REPLICATION_TIME_OUT_DEFAULT;
-                default:
-                    return config.getStatic(this);
-            }
-        }
     }
 
     public boolean isReplicationEnabled() {
         return accessor.getBoolean(Option.REPLICATION_ENABLED);
     }
 
-    private static final int REPLICATION_TIME_OUT_DEFAULT = 15;
-
-    public ReplicationProperties(PropertiesAccessor accessor) throws HyracksDataException {
+    public ReplicationProperties(PropertiesAccessor accessor) {
         super(accessor);
     }
 
-    public int getMaxRemoteRecoveryAttempts() {
-        return accessor.getInt(Option.REPLICATION_MAX_REMOTE_RECOVERY_ATTEMPTS);
-    }
-
-    public int getReplicationFactor() {
-        return accessor.getInt(Option.REPLICATION_FACTOR);
-    }
-
-    public List<String> getNodeIds() {
-        return accessor.getNCNames();
-    }
-
     public int getLogBufferPageSize() {
         return accessor.getInt(Option.REPLICATION_LOG_BUFFER_PAGESIZE);
     }
@@ -131,20 +98,23 @@
         return accessor.getInt(Option.REPLICATION_LOG_BATCHSIZE);
     }
 
-    public String getNodeIpFromId(String id) {
-        return accessor.getNCEffectiveConfig(id).getString(NCConfig.Option.PUBLIC_ADDRESS);
+    public String getReplicationAddress() {
+        return accessor.getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS);
+    }
+
+    public int getReplicationPort() {
+        return accessor.getInt(NCConfig.Option.REPLICATION_LISTEN_PORT);
     }
 
     public String getReplicationStrategy() {
         return accessor.getString(Option.REPLICATION_STRATEGY);
     }
 
-    public int getReplicationTimeOut() {
-        return accessor.getInt(Option.REPLICATION_TIMEOUT);
+    public long getReplicationTimeOut() {
+        return accessor.getLong(Option.REPLICATION_TIMEOUT);
     }
 
-    public MetadataProperties getMetadataProperties() {
-        return new MetadataProperties(accessor);
+    public int getReplicationFactor() {
+        return accessor.getInt(Option.REPLICATION_FACTOR);
     }
-
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/AllDatasetsReplicationStrategy.java
similarity index 78%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/AllDatasetsReplicationStrategy.java
index b750c64..9f9294b 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/AllDatasetsReplicationStrategy.java
@@ -16,4 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+package org.apache.asterix.common.replication;
+
+public class AllDatasetsReplicationStrategy implements IReplicationStrategy {
+
+    @Override
+    public boolean isMatch(int datasetId) {
+        return true;
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
deleted file mode 100644
index 6f66599..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
+++ /dev/null
@@ -1,105 +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.common.replication;
-
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.asterix.common.config.ReplicationProperties;
-import org.apache.hyracks.api.config.IConfigManager;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.control.common.config.ConfigManager;
-import org.apache.hyracks.control.common.controllers.NCConfig;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class ChainedDeclusteringReplicationStrategy implements IReplicationStrategy {
-
-    private static final Logger LOGGER = LogManager.getLogger();
-    private int replicationFactor;
-    private ReplicationProperties repProp;
-    private ConfigManager configManager;
-
-    @Override
-    public boolean isMatch(int datasetId) {
-        return true;
-    }
-
-    @Override
-    public Set<Replica> getRemoteReplicas(String nodeId) {
-        Set<Replica> remoteReplicas = new HashSet<>();
-        int numberOfRemoteReplicas = replicationFactor - 1;
-        int nodeIndex = repProp.getNodeIds().indexOf(nodeId);
-
-        if (nodeIndex == -1) {
-            if (LOGGER.isWarnEnabled()) {
-                LOGGER.warn("Could not find node " + nodeId + " in cluster configurations");
-            }
-            return Collections.emptySet();
-        }
-
-        //find nodes to the right of this node
-        while (remoteReplicas.size() != numberOfRemoteReplicas) {
-            String replica = repProp.getNodeIds().get(++nodeIndex % repProp.getNodeIds().size());
-            remoteReplicas.add(new Replica(replica,
-                    configManager.getNodeEffectiveConfig(replica).getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS),
-                    configManager.getNodeEffectiveConfig(replica).getInt(NCConfig.Option.REPLICATION_LISTEN_PORT)));
-        }
-
-        return remoteReplicas;
-    }
-
-    @Override
-    public Set<Replica> getRemoteReplicasAndSelf(String nodeId) {
-        Set<Replica> replicas = getRemoteReplicas(nodeId);
-        replicas.add(new Replica(nodeId,
-                configManager.getNodeEffectiveConfig(nodeId).getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS),
-                configManager.getNodeEffectiveConfig(nodeId).getInt(NCConfig.Option.REPLICATION_LISTEN_PORT)));
-        return replicas;
-
-    }
-
-    @Override
-    public Set<Replica> getRemotePrimaryReplicas(String nodeId) {
-        Set<Replica> clientReplicas = new HashSet<>();
-        final int remotePrimaryReplicasCount = replicationFactor - 1;
-        int nodeIndex = repProp.getNodeIds().indexOf(nodeId);
-
-        //find nodes to the left of this node
-        while (clientReplicas.size() != remotePrimaryReplicasCount) {
-            String replica = repProp.getNodeIds().get(Math.abs(--nodeIndex % repProp.getNodeIds().size()));
-            clientReplicas.add(new Replica(replica,
-                    configManager.getNodeEffectiveConfig(replica).getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS),
-                    configManager.getNodeEffectiveConfig(replica).getInt(NCConfig.Option.REPLICATION_LISTEN_PORT)));
-        }
-
-        return clientReplicas;
-    }
-
-    @Override
-    public ChainedDeclusteringReplicationStrategy from(ReplicationProperties repProp, IConfigManager configManager)
-            throws HyracksDataException {
-        ChainedDeclusteringReplicationStrategy cd = new ChainedDeclusteringReplicationStrategy();
-        cd.repProp = repProp;
-        cd.replicationFactor = repProp.getReplicationFactor();
-        cd.configManager = (ConfigManager) configManager;
-        return cd;
-    }
-}
\ No newline at end of file
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/IFaultToleranceStrategy.java
index e871374..bf4cac9 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/IFaultToleranceStrategy.java
@@ -58,11 +58,11 @@
     /**
      * Constructs a fault tolerance strategy.
      *
-     * @param replicationStrategy
-     * @param messageBroker
-     * @return
+     * @param serviceCtx
+     * @param replicationEnabled
+     * @return the fault tolerance strategy
      */
-    IFaultToleranceStrategy from(ICCServiceContext serviceCtx, IReplicationStrategy replicationStrategy);
+    IFaultToleranceStrategy from(ICCServiceContext serviceCtx, boolean replicationEnabled);
 
     /**
      * Performs the required steps to change the metadata node to {@code node}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INCLifecycleMessage.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INCLifecycleMessage.java
index 372a88a..69432e9 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INCLifecycleMessage.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INCLifecycleMessage.java
@@ -23,17 +23,9 @@
 public interface INCLifecycleMessage extends IMessage {
 
     enum MessageType {
-        REPLAY_LOGS_REQUEST,
-        REPLAY_LOGS_RESPONSE,
-        PREPARE_FAILBACK_REQUEST,
-        PREPARE_FAILBACK_RESPONSE,
-        COMPLETE_FAILBACK_REQUEST,
-        COMPLETE_FAILBACK_RESPONSE,
         REGISTRATION_TASKS_REQUEST,
         REGISTRATION_TASKS_RESPONSE,
         REGISTRATION_TASKS_RESULT,
-        TAKEOVER_PARTITION_REQUEST,
-        TAKEOVER_PARTITION_RESPONSE,
         METADATA_NODE_REQUEST,
         METADATA_NODE_RESPONSE
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IPartitionReplica.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IPartitionReplica.java
index 5a9dc3f..761b2c6 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IPartitionReplica.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IPartitionReplica.java
@@ -44,4 +44,11 @@
      * @return The identifier
      */
     ReplicaIdentifier getIdentifier();
+
+    /**
+     * Notifies that failure {@code failure} occurred on this replica
+     *
+     * @param failure
+     */
+    void notifyFailure(Exception failure);
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java
deleted file mode 100644
index 51b826b..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java
+++ /dev/null
@@ -1,71 +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.common.replication;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public interface IRemoteRecoveryManager {
-
-    /**
-     * Performs the partitions takeover process from the {@code failedNode}
-     *
-     * @param failedNode
-     * @param partitions
-     * @throws IOException
-     * @throws ACIDException
-     */
-    public void takeoverPartitons(Integer[] partitions) throws IOException, ACIDException;
-
-    /**
-     * Construct failback plan and requests LSM disk components from active remote replicas.
-     */
-    public void startFailbackProcess();
-
-    /**
-     * Requests the remaining LSM disk components files from active remote replicas.
-     *
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    public void completeFailbackProcess() throws IOException, InterruptedException;
-
-    /**
-     * Replays all committed jobs logs for {@code partitions}. Optionally, flushes all datasets
-     * to convert the replayed logs into LSM Components.
-     *
-     * @param partitions
-     * @param flush
-     * @throws HyracksDataException
-     */
-    void replayReplicaPartitionLogs(Set<Integer> partitions, boolean flush) throws HyracksDataException;
-
-    /**
-     * Performs the remote recovery plan by requesting data from each specified node
-     * for each partitions specified.
-     *
-     * @param recoveryPlan
-     * @throws HyracksDataException
-     */
-    void doRemoteRecoveryPlan(Map<String, Set<Integer>> recoveryPlan) throws HyracksDataException;
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java
deleted file mode 100644
index 72a7f9d..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java
+++ /dev/null
@@ -1,36 +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.common.replication;
-
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public interface IReplicaResourcesManager {
-
-    /**
-     * @param partitions
-     * @return the minimum LSN of all indexes that belong to {@code partitions}.
-     * @throws HyracksDataException
-     */
-    long getPartitionsMinLSN(Set<Integer> partitions) throws HyracksDataException;
-
-    List<String> getPartitionIndexesFiles(int partition, boolean relativePath) throws HyracksDataException;
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationDestination.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationDestination.java
new file mode 100644
index 0000000..2fe9de8
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationDestination.java
@@ -0,0 +1,60 @@
+/*
+ * 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.common.replication;
+
+import java.net.InetSocketAddress;
+import java.util.Set;
+
+public interface IReplicationDestination {
+
+    /**
+     * Adds {@code replica} to this {@link IReplicationDestination}
+     *
+     * @param replica
+     */
+    void add(IPartitionReplica replica);
+
+    /**
+     * Removes {@code replica} from this {@link IReplicationDestination}
+     *
+     * @param replica
+     */
+    void remove(IPartitionReplica replica);
+
+    /**
+     * Notifies that failure {@code failure} occurred on this {@link IReplicationDestination}
+     *
+     * @param failure
+     */
+    void notifyFailure(Exception failure);
+
+    /**
+     * Gets the list of replicas on this {@link IReplicationDestination}
+     *
+     * @return the list of replicas
+     */
+    Set<IPartitionReplica> getReplicas();
+
+    /**
+     * Gets the location of this {@link IReplicationDestination}
+     *
+     * @return the location
+     */
+    InetSocketAddress getLocation();
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java
index 07076cf..1da2980 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java
@@ -18,122 +18,44 @@
  */
 package org.apache.asterix.common.replication;
 
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Set;
-
 import org.apache.asterix.common.transactions.ILogRecord;
 import org.apache.hyracks.api.replication.IIOReplicationManager;
 
 public interface IReplicationManager extends IIOReplicationManager {
 
     /**
-     * Asynchronously sends a serialized version of the record to remote replicas.
-     *
-     * @param logRecord
-     *            The log record to be replicated,
-     * @throws InterruptedException
-     */
-    public void replicateLog(ILogRecord logRecord) throws InterruptedException;
-
-    /**
-     * Checks whether a log record has been replicated
-     *
-     * @param logRecord
-     *            the log to check for.
-     * @return true, if all ACKs were received from remote replicas.
-     */
-    public boolean hasBeenReplicated(ILogRecord logRecord);
-
-    /**
-     * Requests LSM components files from a remote replica.
-     *
-     * @param remoteReplicaId
-     *            The replica id to send the request to.
-     * @param partitionsToRecover
-     *            Get files that belong to those partitions.
-     * @param existingFiles
-     *            a list of already existing files on the requester
-     * @throws IOException
-     */
-    public void requestReplicaFiles(String remoteReplicaId, Set<Integer> partitionsToRecover, Set<String> existingFiles)
-            throws IOException;
-
-    /**
-     * Requests current maximum LSN from remote replicas.
-     *
-     * @param remoteReplicaIds
-     *            remote replicas to send the request to.
-     * @return The maximum of the received maximum LSNs.
-     * @throws IOException
-     */
-    public long getMaxRemoteLSN(Set<String> remoteReplicaIds) throws IOException;
-
-    /**
-     * @return The number of remote replicas that are in ACTIVE state.
-     */
-    public int getActiveReplicasCount();
-
-    /**
-     * @return The IDs of the remote replicas that are in DEAD state.
-     */
-    public Set<String> getDeadReplicasIds();
-
-    /**
-     * Starts processing of ASYNC replication jobs as well as Txn logs.
-     *
-     * @throws InterruptedException
-     */
-    public void startReplicationThreads() throws InterruptedException;
-
-    /**
-     * Checks and sets each remote replica state.
-     */
-    public void initializeReplicasState();
-
-    /**
-     * Updates remote replica (in-memory) information.
-     *
-     * @param replica
-     *            the replica to update.
-     */
-    public void updateReplicaInfo(Replica replica);
-
-    /**
-     * @return The IDs of the remote replicas that are in ACTIVE state.
-     */
-    public Set<String> getActiveReplicasIds();
-
-    /**
-     * Submits a ReplicaEvent to ReplicationEventsMonitor thread.
-     *
-     * @param event
-     */
-    public void reportReplicaEvent(ReplicaEvent event);
-
-    /**
-     * Sends a request to remote replicas to flush indexes that have LSN less than nonSharpCheckpointTargetLSN
-     *
-     * @param nonSharpCheckpointTargetLSN
-     * @throws IOException
-     */
-    public void requestFlushLaggingReplicaIndexes(long nonSharpCheckpointTargetLSN) throws IOException;
-
-    /**
-     * Transfers the contents of the {@code buffer} to active remote replicas.
-     * The transfer starts from the {@code buffer} current position to its limit.
-     * After the transfer, the {@code buffer} position will be its limit.
-     *
-     * @param buffer
-     */
-    public void replicateTxnLogBatch(ByteBuffer buffer);
-
-    IReplicationStrategy getReplicationStrategy();
-
-    /**
      * Registers {@code replica}. After registration, the replica will be included in all replication events
      *
      * @param replica
      */
     void register(IPartitionReplica replica);
+
+    /**
+     * Unregisters {@code replica} from replication events.
+     *
+     * @param replica
+     */
+    void unregister(IPartitionReplica replica);
+
+    /**
+     * Notifies that failure {@code failure} occurred on {@code dest}
+     *
+     * @param dest
+     * @param failure
+     */
+    void notifyFailure(IReplicationDestination dest, Exception failure);
+
+    /**
+     * Asynchronously sends a serialized version of the record to remote replicas.
+     *
+     * @param logRecord The log record to be replicated
+     * @throws InterruptedException
+     */
+    void replicate(ILogRecord logRecord) throws InterruptedException;
+
+    /**
+     * @return the replication strategy
+     */
+    IReplicationStrategy getReplicationStrategy();
+
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
index 93ccfbe..0ad8be2 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
@@ -18,12 +18,6 @@
  */
 package org.apache.asterix.common.replication;
 
-import java.util.Set;
-
-import org.apache.asterix.common.config.ReplicationProperties;
-import org.apache.hyracks.api.config.IConfigManager;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
 public interface IReplicationStrategy {
 
     /**
@@ -31,32 +25,4 @@
      * @return True, if the dataset should be replicated. Otherwise false.
      */
     boolean isMatch(int datasetId);
-
-    Set<Replica> getRemoteReplicasAndSelf(String nodeId);
-
-    /**
-     * @param nodeId
-     * @return The set of nodes that replicate data on {@code nodeId}.
-     */
-    Set<Replica> getRemotePrimaryReplicas(String nodeId);
-
-    /**
-     * @param node
-     * @return The set of nodes that {@code nodeId} replicates data to.
-     */
-    Set<Replica> getRemoteReplicas(String node);
-
-    /**
-     * @param nodeId
-     * @return true if {@code nodeId} has any remote primary replica or remote replica. Otherwise false.
-     */
-    default boolean isParticipant(String nodeId) {
-        return !getRemoteReplicas(nodeId).isEmpty() || !getRemotePrimaryReplicas(nodeId).isEmpty();
-    }
-
-    /**
-     * @param p
-     * @return A replication strategy based on the passed configurations.
-     */
-    IReplicationStrategy from(ReplicationProperties p, IConfigManager configManager) throws HyracksDataException;
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
index 2262437..5733a60 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
@@ -18,91 +18,12 @@
  */
 package org.apache.asterix.common.replication;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.asterix.common.config.MetadataProperties;
-import org.apache.asterix.common.config.ReplicationProperties;
 import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
-import org.apache.hyracks.api.config.IConfigManager;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.control.common.config.ConfigManager;
-import org.apache.hyracks.control.common.controllers.NCConfig;
 
 public class MetadataOnlyReplicationStrategy implements IReplicationStrategy {
 
-    private String metadataPrimaryReplicaId;
-    private Replica metadataPrimaryReplica;
-    private Set<Replica> metadataNodeReplicas;
-    MetadataProperties metadataProperties;
-
     @Override
     public boolean isMatch(int datasetId) {
         return datasetId < MetadataIndexImmutableProperties.FIRST_AVAILABLE_USER_DATASET_ID && datasetId >= 0;
     }
-
-    @Override
-    public Set<Replica> getRemoteReplicas(String nodeId) {
-        if (nodeId.equals(metadataPrimaryReplicaId)) {
-            return metadataNodeReplicas;
-        }
-        return Collections.emptySet();
-    }
-
-    @Override
-    public Set<Replica> getRemoteReplicasAndSelf(String nodeId) {
-
-        if (nodeId.equals(metadataPrimaryReplicaId)) {
-            Set<Replica> replicasAndSelf = new HashSet<>();
-            replicasAndSelf.addAll(metadataNodeReplicas);
-            replicasAndSelf.add(metadataPrimaryReplica);
-            return replicasAndSelf;
-        }
-        return Collections.emptySet();
-    }
-
-    @Override
-    public Set<Replica> getRemotePrimaryReplicas(String nodeId) {
-        if (metadataNodeReplicas.stream().map(Replica::getId).filter(replicaId -> replicaId.equals(nodeId))
-                .count() != 0) {
-            return new HashSet<>(Arrays.asList(metadataPrimaryReplica));
-        }
-        return Collections.emptySet();
-    }
-
-    @Override
-    public MetadataOnlyReplicationStrategy from(ReplicationProperties p, IConfigManager configManager)
-            throws HyracksDataException {
-        MetadataOnlyReplicationStrategy st = new MetadataOnlyReplicationStrategy();
-        st.metadataProperties = p.getMetadataProperties();
-        st.metadataPrimaryReplicaId = st.metadataProperties.getMetadataNodeName();
-        st.metadataPrimaryReplica = new Replica(st.metadataPrimaryReplicaId,
-                ((ConfigManager) configManager).getNodeEffectiveConfig(st.metadataPrimaryReplicaId)
-                        .getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS),
-                ((ConfigManager) configManager).getNodeEffectiveConfig(st.metadataPrimaryReplicaId)
-                        .getInt(NCConfig.Option.REPLICATION_LISTEN_PORT));
-        final Set<Replica> replicas = new HashSet<>();
-        Set<String> candidateSet = new HashSet<>();
-        candidateSet.addAll(((ConfigManager) (configManager)).getNodeNames());
-        candidateSet.remove(st.metadataPrimaryReplicaId);
-        String[] candidateAry = new String[candidateSet.size()];
-        candidateSet.toArray(candidateAry);
-        for (int i = 0; i < candidateAry.length && i < p.getReplicationFactor(); i++) {
-            replicas.add(new Replica(candidateAry[i],
-                    ((ConfigManager) configManager).getNodeEffectiveConfig(candidateAry[i])
-                            .getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS),
-                    ((ConfigManager) configManager).getNodeEffectiveConfig(candidateAry[i])
-                            .getInt(NCConfig.Option.REPLICATION_LISTEN_PORT)));
-        }
-        st.metadataNodeReplicas = replicas;
-        return st;
-    }
-
-    @Override
-    public boolean isParticipant(String nodeId) {
-        return true;
-    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/NoReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/NoReplicationStrategy.java
index c32ee3c..c3d9ced 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/NoReplicationStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/NoReplicationStrategy.java
@@ -18,42 +18,10 @@
  */
 package org.apache.asterix.common.replication;
 
-import java.util.Collections;
-import java.util.Set;
-
-import org.apache.asterix.common.config.ReplicationProperties;
-import org.apache.hyracks.api.config.IConfigManager;
-import org.apache.hyracks.control.common.config.ConfigManager;
-import org.apache.hyracks.control.common.controllers.NCConfig;
-
 public class NoReplicationStrategy implements IReplicationStrategy {
 
     @Override
     public boolean isMatch(int datasetId) {
         return false;
     }
-
-    @Override
-    public boolean isParticipant(String nodeId) {
-        return false;
-    }
-
-    @Override
-    public Set<Replica> getRemotePrimaryReplicas(String nodeId) {
-        return Collections.emptySet();
-    }
-
-    @Override
-    public Set<Replica> getRemoteReplicas(String node) {
-        return Collections.emptySet();
-    }
-
-    public Set<Replica> getRemoteReplicasAndSelf(String nodeId) {
-        return Collections.emptySet();
-    }
-
-    @Override
-    public NoReplicationStrategy from(ReplicationProperties p, IConfigManager configManager) {
-        return new NoReplicationStrategy();
-    }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/Replica.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/Replica.java
deleted file mode 100644
index 52dbd25..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/Replica.java
+++ /dev/null
@@ -1,112 +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.common.replication;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-public class Replica {
-
-    public enum ReplicaState {
-        ACTIVE,
-        DEAD,
-        UNKNOWN
-    }
-
-    private ReplicaState state = ReplicaState.UNKNOWN;
-    String nodeId;
-    String ipAddr;
-    int port;
-
-    public Replica(String id, String ip, int port) {
-        nodeId = id;
-        ipAddr = ip;
-        this.port = port;
-    }
-
-    public ReplicaState getState() {
-        return state;
-    }
-
-    public void setState(ReplicaState state) {
-        this.state = state;
-    }
-
-    public static Replica create(DataInput input) throws IOException {
-        Replica replica = new Replica(null, null, -1);
-        replica.readFields(input);
-        return replica;
-    }
-
-    public String getId() {
-        return nodeId;
-    }
-
-    public void writeFields(DataOutput output) throws IOException {
-        output.writeUTF(nodeId);
-        output.writeUTF(ipAddr);
-        output.writeInt(port);
-        output.writeInt(state.ordinal());
-    }
-
-    public void readFields(DataInput input) throws IOException {
-        this.nodeId = input.readUTF();
-        this.ipAddr = input.readUTF();
-        this.port = input.readInt();
-        this.state = ReplicaState.values()[input.readInt()];
-    }
-
-    public String getClusterIp() {
-        return ipAddr;
-    }
-
-    public void setClusterIp(String ip) {
-        ipAddr = ip;
-    }
-
-    public int getPort() {
-        return port;
-    }
-
-    public void setPort(int port) {
-        this.port = port;
-    }
-
-    public void serialize(OutputStream out) throws IOException {
-        DataOutputStream dos = new DataOutputStream(out);
-        writeFields(dos);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof Replica)) {
-            return false;
-        }
-        Replica other = (Replica) o;
-        return nodeId.equals(other.getId());
-    }
-
-    @Override
-    public int hashCode() {
-        return nodeId.hashCode();
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java
deleted file mode 100644
index ae02ca9..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java
+++ /dev/null
@@ -1,67 +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.common.replication;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
-
-public class ReplicaEvent {
-
-    Replica replica;
-    ClusterEventType eventType;
-
-    public ReplicaEvent(Replica replica, ClusterEventType eventType) {
-        this.replica = replica;
-        this.eventType = eventType;
-    }
-
-    public Replica getReplica() {
-        return replica;
-    }
-
-    public void setReplica(Replica replica) {
-        this.replica = replica;
-    }
-
-    public ClusterEventType getEventType() {
-        return eventType;
-    }
-
-    public void setEventType(ClusterEventType eventType) {
-        this.eventType = eventType;
-    }
-
-    public void serialize(OutputStream out) throws IOException {
-        DataOutputStream dos = new DataOutputStream(out);
-        replica.writeFields(dos);
-        dos.writeInt(eventType.ordinal());
-    }
-
-    public static ReplicaEvent create(DataInput input) throws IOException {
-        Replica replica = Replica.create(input);
-        ClusterEventType eventType = ClusterEventType.values()[input.readInt()];
-        ReplicaEvent event = new ReplicaEvent(replica, eventType);
-        return event;
-    }
-
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
index e6b6445..8d51a99 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
@@ -21,39 +21,31 @@
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.asterix.common.config.ReplicationProperties;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.hyracks.api.config.IConfigManager;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
 public class ReplicationStrategyFactory {
 
     private static final Map<String, Class<? extends IReplicationStrategy>> BUILT_IN_REPLICATION_STRATEGY =
             new HashMap<>();
 
     static {
-        BUILT_IN_REPLICATION_STRATEGY.put("no_replication", NoReplicationStrategy.class);
-        BUILT_IN_REPLICATION_STRATEGY.put("chained_declustering", ChainedDeclusteringReplicationStrategy.class);
-        BUILT_IN_REPLICATION_STRATEGY.put("metadata_only", MetadataOnlyReplicationStrategy.class);
+        BUILT_IN_REPLICATION_STRATEGY.put("none", NoReplicationStrategy.class);
+        BUILT_IN_REPLICATION_STRATEGY.put("all", AllDatasetsReplicationStrategy.class);
+        BUILT_IN_REPLICATION_STRATEGY.put("metadata", MetadataOnlyReplicationStrategy.class);
     }
 
     private ReplicationStrategyFactory() {
         throw new AssertionError();
     }
 
-    public static IReplicationStrategy create(String name, ReplicationProperties repProp, IConfigManager ncConfig)
-            throws HyracksDataException {
+    public static IReplicationStrategy create(String name) {
         String strategyName = name.toLowerCase();
         if (!BUILT_IN_REPLICATION_STRATEGY.containsKey(strategyName)) {
-            throw new RuntimeDataException(ErrorCode.UNSUPPORTED_REPLICATION_STRATEGY, String.format(
-                    "%s. Available strategies: %s", strategyName, BUILT_IN_REPLICATION_STRATEGY.keySet().toString()));
+            throw new IllegalStateException("Couldn't find strategy with name: " + name);
         }
         Class<? extends IReplicationStrategy> clazz = BUILT_IN_REPLICATION_STRATEGY.get(strategyName);
         try {
-            return clazz.newInstance().from(repProp, ncConfig);
+            return clazz.newInstance();
         } catch (InstantiationException | IllegalAccessException e) {
-            throw new RuntimeDataException(ErrorCode.INSTANTIATION_ERROR, e, clazz.getName());
+            throw new IllegalStateException("Couldn't instantiated replication strategy: " + name, e);
         }
     }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
index 8cc11a8..c7de7a3 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
@@ -91,6 +91,10 @@
         ref.root = tokens[--offset];
     }
 
+    public int getPartitionNum() {
+        return Integer.parseInt(partition.substring(StorageConstants.PARTITION_DIR_PREFIX.length()));
+    }
+
     protected static void parseLegacyPath(ResourceReference ref, String path) {
         // old format: root/partition/dataverse/datasetName_idx_IndexName/fileName
         final String[] tokens = path.split(File.separator);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
index 4090b65..3274849 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
@@ -20,7 +20,6 @@
 
 import java.nio.ByteBuffer;
 
-import org.apache.asterix.common.replication.IReplicationThread;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 
 public interface ILogRecord {
@@ -133,8 +132,6 @@
 
     public void readRemoteLog(ByteBuffer buffer);
 
-    public void setReplicationThread(IReplicationThread replicationThread);
-
     public void setLogSource(byte logSource);
 
     public byte getLogSource();
@@ -150,7 +147,7 @@
     public void setReplicated(boolean replicated);
 
     /**
-     * @return a flag indicating whether the log record should be sent to remote replicas
+     * @return a flag indicating whether the log was replicated
      */
     public boolean isReplicated();
 
@@ -169,4 +166,18 @@
     public void logAppended(long lsn);
 
     public long getPreviousMarkerLSN();
+
+    /**
+     * Sets flag indicating if this log should be replicated or not
+     *
+     * @param replicate
+     */
+    void setReplicate(boolean replicate);
+
+    /**
+     * Gets a flag indicating if this log should be replicated or not
+     *
+     * @return the flag
+     */
+    boolean isReplicate();
 }
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRequester.java
similarity index 72%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRequester.java
index b750c64..8e874a2 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRequester.java
@@ -16,4 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+package org.apache.asterix.common.transactions;
+
+public interface ILogRequester {
+
+    /**
+     * Sends a notification to the log requester that {@code logRecord} has been flushed.
+     *
+     * @param logRecord The log that has been flushed.
+     */
+    void notifyFlushed(ILogRecord logRecord);
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
index dea7a67..bfe7963 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
@@ -119,4 +119,15 @@
      * @throws ACIDException
      */
     void startLocalRecovery(Set<Integer> partitions) throws IOException, ACIDException;
+
+    /**
+     * Replay the commited transactions' logs belonging to {@code partitions}. if {@code flush} is true,
+     * all datasets are flushed after the logs are replayed.
+     *
+     * @param partitions
+     * @param flush
+     * @throws HyracksDataException
+     */
+    void replayReplicaPartitionLogs(Set<Integer> partitions, boolean flush) throws HyracksDataException;
+
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
index 68afb2a..6b38aa1 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
@@ -24,7 +24,6 @@
 import java.util.zip.CRC32;
 
 import org.apache.asterix.common.context.PrimaryIndexOperationTracker;
-import org.apache.asterix.common.replication.IReplicationThread;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.tuples.SimpleTupleReference;
 import org.apache.hyracks.storage.am.common.tuples.SimpleTupleWriter;
@@ -94,18 +93,21 @@
     private final CRC32 checksumGen;
     private int[] PKFields;
     private PrimaryIndexOperationTracker opTracker;
-    private IReplicationThread replicationThread;
+
     /**
      * The fields (numOfFlushedIndexes and nodeId) are used for remote flush logs only
      * to indicate the source of the log and how many indexes were flushed using its LSN.
      */
     private int numOfFlushedIndexes;
     private String nodeId;
-    private boolean replicated = false;
+    private final AtomicBoolean replicated;
+    private boolean replicate = false;
+    private ILogRequester requester;
 
     public LogRecord(ILogMarkerCallback callback) {
         this.callback = callback;
         isFlushed = new AtomicBoolean(false);
+        replicated = new AtomicBoolean(false);
         readPKValue = new PrimaryKeyTupleReference();
         readNewValue = SimpleTupleWriter.INSTANCE.createTupleReference();
         readOldValue = SimpleTupleWriter.INSTANCE.createTupleReference();
@@ -638,15 +640,6 @@
         this.nodeId = nodeId;
     }
 
-    public IReplicationThread getReplicationThread() {
-        return replicationThread;
-    }
-
-    @Override
-    public void setReplicationThread(IReplicationThread replicationThread) {
-        this.replicationThread = replicationThread;
-    }
-
     @Override
     public void setLogSource(byte logSource) {
         this.logSource = logSource;
@@ -684,13 +677,13 @@
     }
 
     @Override
-    public void setReplicated(boolean replicate) {
-        this.replicated = replicate;
+    public void setReplicated(boolean replicated) {
+        this.replicated.set(replicated);
     }
 
     @Override
     public boolean isReplicated() {
-        return replicated;
+        return replicated.get();
     }
 
     @Override
@@ -732,4 +725,22 @@
     public ByteBuffer getMarker() {
         return marker;
     }
+
+    @Override
+    public void setReplicate(boolean replicate) {
+        this.replicate = replicate;
+    }
+
+    @Override
+    public boolean isReplicate() {
+        return replicate;
+    }
+
+    public ILogRequester getRequester() {
+        return requester;
+    }
+
+    public void setRequester(ILogRequester requester) {
+        this.requester = requester;
+    }
 }
diff --git a/asterixdb/asterix-replication/pom.xml b/asterixdb/asterix-replication/pom.xml
index 7072e48..bb983ad 100644
--- a/asterixdb/asterix-replication/pom.xml
+++ b/asterixdb/asterix-replication/pom.xml
@@ -51,10 +51,6 @@
       <artifactId>hyracks-util</artifactId>
     </dependency>
     <dependency>
-      <groupId>commons-io</groupId>
-      <artifactId>commons-io</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-lsm-common</artifactId>
     </dependency>
@@ -72,14 +68,6 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-control-nc</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-control-common</artifactId>
-    </dependency>
-    <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-core</artifactId>
     </dependency>
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicaTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicaTask.java
index 001d41f..807ae5f 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicaTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicaTask.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.replication.api;
 
 import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.replication.IReplicationThread;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface IReplicaTask extends IReplicationMessage {
@@ -31,5 +30,5 @@
      * @param worker
      * @throws HyracksDataException
      */
-    void perform(INcApplicationContext appCtx, IReplicationThread worker) throws HyracksDataException;
+    void perform(INcApplicationContext appCtx, IReplicationWorker worker) throws HyracksDataException;
 }
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationMessage.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationMessage.java
index 2e1cb8a..bc6e87b 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationMessage.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationMessage.java
@@ -20,7 +20,7 @@
 
 import java.io.OutputStream;
 
-import org.apache.asterix.replication.functions.ReplicationProtocol;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface IReplicationMessage {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationThread.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationWorker.java
similarity index 78%
rename from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationThread.java
rename to asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationWorker.java
index 5b9d4fa..8840c3f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationThread.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationWorker.java
@@ -16,21 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.replication;
+package org.apache.asterix.replication.api;
 
 import java.nio.ByteBuffer;
 import java.nio.channels.SocketChannel;
 
 import org.apache.asterix.common.transactions.LogRecord;
 
-public interface IReplicationThread extends Runnable {
-
-    /**
-     * Sends a notification to this thread that logRecord has been flushed.
-     *
-     * @param logRecord The log that has been flushed.
-     */
-    void notifyLogReplicationRequester(LogRecord logRecord);
+public interface IReplicationWorker extends Runnable {
 
     /**
      * @return The replication socket channel.
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/PartitionReplica.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
similarity index 90%
rename from asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/PartitionReplica.java
rename to asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
index b7fa49d..f752745 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/PartitionReplica.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.replication.storage;
+package org.apache.asterix.replication.api;
 
 import static org.apache.asterix.common.replication.IPartitionReplica.PartitionReplicaStatus.CATCHING_UP;
 import static org.apache.asterix.common.replication.IPartitionReplica.PartitionReplicaStatus.DISCONNECTED;
@@ -30,8 +30,8 @@
 import org.apache.asterix.common.exceptions.ReplicationException;
 import org.apache.asterix.common.replication.IPartitionReplica;
 import org.apache.asterix.common.storage.ReplicaIdentifier;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
-import org.apache.asterix.replication.recovery.ReplicaSynchronizer;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.asterix.replication.sync.ReplicaSynchronizer;
 import org.apache.hyracks.util.JSONUtil;
 import org.apache.hyracks.util.StorageUtil;
 import org.apache.hyracks.util.annotations.ThreadSafe;
@@ -70,6 +70,11 @@
         return id;
     }
 
+    @Override
+    public synchronized void notifyFailure(Exception failure) {
+        setStatus(DISCONNECTED);
+    }
+
     public synchronized void sync() {
         if (status == IN_SYNC || status == CATCHING_UP) {
             return;
@@ -81,7 +86,7 @@
                 setStatus(IN_SYNC);
             } catch (Exception e) {
                 LOGGER.error(() -> "Failed to sync replica " + this, e);
-                setStatus(DISCONNECTED);
+                notifyFailure(e);
             } finally {
                 close();
             }
@@ -109,11 +114,11 @@
                 sc = null;
             }
         } catch (IOException e) {
-            throw new ReplicationException(e);
+            LOGGER.warn("Failed to close channel", e);
         }
     }
 
-    public synchronized ByteBuffer gerReusableBuffer() {
+    public synchronized ByteBuffer getReusableBuffer() {
         if (reusbaleBuf == null) {
             reusbaleBuf = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
         }
@@ -154,6 +159,9 @@
     }
 
     private synchronized void setStatus(PartitionReplicaStatus status) {
+        if (this.status == status) {
+            return;
+        }
         LOGGER.info(() -> "Replica " + this + " status changing: " + this.status + " -> " + status);
         this.status = status;
     }
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/ReplicationDestination.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/ReplicationDestination.java
new file mode 100644
index 0000000..a092322
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/ReplicationDestination.java
@@ -0,0 +1,128 @@
+/*
+ * 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.replication.api;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.channels.SocketChannel;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.common.replication.IPartitionReplica;
+import org.apache.asterix.common.replication.IReplicationDestination;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class ReplicationDestination implements IReplicationDestination {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final Set<IPartitionReplica> replicas = new HashSet<>();
+    private final InetSocketAddress location;
+    private SocketChannel logRepChannel;
+
+    private ReplicationDestination(InetSocketAddress location) {
+        this.location = location;
+    }
+
+    public static ReplicationDestination at(InetSocketAddress location) {
+        return new ReplicationDestination(location);
+    }
+
+    @Override
+    public synchronized void add(IPartitionReplica replica) {
+        replicas.add(replica);
+    }
+
+    @Override
+    public synchronized void remove(IPartitionReplica replica) {
+        replicas.remove(replica);
+    }
+
+    @Override
+    public synchronized void notifyFailure(Exception failure) {
+        replicas.forEach(replica -> replica.notifyFailure(failure));
+        closeLogReplicationChannel();
+    }
+
+    @Override
+    public Set<IPartitionReplica> getReplicas() {
+        return new HashSet<>(replicas);
+    }
+
+    public synchronized Optional<IPartitionReplica> getPartitionReplica(int partition) {
+        return replicas.stream().filter(replica -> replica.getIdentifier().getPartition() == partition
+                && replica.getStatus() == IPartitionReplica.PartitionReplicaStatus.IN_SYNC).findAny();
+    }
+
+    public synchronized SocketChannel getLogReplicationChannel() {
+        try {
+            if (logRepChannel == null || !logRepChannel.isOpen() || !logRepChannel.isConnected()) {
+                logRepChannel = SocketChannel.open();
+                logRepChannel.configureBlocking(true);
+                logRepChannel.connect(location);
+            }
+            return logRepChannel;
+        } catch (IOException e) {
+            throw new ReplicationException(e);
+        }
+    }
+
+    private synchronized void closeLogReplicationChannel() {
+        try {
+            if (logRepChannel != null && logRepChannel.isOpen()) {
+                ReplicationProtocol.sendGoodbye(logRepChannel);
+                logRepChannel.close();
+                logRepChannel = null;
+            }
+        } catch (IOException e) {
+            LOGGER.warn("Exception while closing socket", e);
+        }
+    }
+
+    @Override
+    public InetSocketAddress getLocation() {
+        return location;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        ReplicationDestination that = (ReplicationDestination) o;
+        return Objects.equals(location, that.location);
+    }
+
+    @Override
+    public String toString() {
+        return location.toString();
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(location);
+    }
+}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java
deleted file mode 100644
index f5b2378..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java
+++ /dev/null
@@ -1,70 +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.replication.functions;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.HashSet;
-import java.util.Set;
-
-public class ReplicaFilesRequest {
-    private final Set<Integer> partitionIds;
-    private final Set<String> existingFiles;
-
-    public ReplicaFilesRequest(Set<Integer> partitionIds, Set<String> existingFiles) {
-        this.partitionIds = partitionIds;
-        this.existingFiles = existingFiles;
-    }
-
-    public void serialize(OutputStream out) throws IOException {
-        DataOutputStream dos = new DataOutputStream(out);
-        dos.writeInt(partitionIds.size());
-        for (Integer partitionId : partitionIds) {
-            dos.writeInt(partitionId);
-        }
-        dos.writeInt(existingFiles.size());
-        for (String fileName : existingFiles) {
-            dos.writeUTF(fileName);
-        }
-    }
-
-    public static ReplicaFilesRequest create(DataInput input) throws IOException {
-        int size = input.readInt();
-        Set<Integer> partitionIds = new HashSet<>(size);
-        for (int i = 0; i < size; i++) {
-            partitionIds.add(input.readInt());
-        }
-        int filesCount = input.readInt();
-        Set<String> existingFiles = new HashSet<>(filesCount);
-        for (int i = 0; i < filesCount; i++) {
-            existingFiles.add(input.readUTF());
-        }
-        return new ReplicaFilesRequest(partitionIds, existingFiles);
-    }
-
-    public Set<Integer> getPartitionIds() {
-        return partitionIds;
-    }
-
-    public Set<String> getExistingFiles() {
-        return existingFiles;
-    }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaIndexFlushRequest.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaIndexFlushRequest.java
deleted file mode 100644
index a3c269d..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaIndexFlushRequest.java
+++ /dev/null
@@ -1,60 +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.replication.functions;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.HashSet;
-import java.util.Set;
-
-public class ReplicaIndexFlushRequest {
-    Set<Long> laggingRescouresIds;
-
-    public ReplicaIndexFlushRequest(Set<Long> laggingRescouresIds) {
-        this.laggingRescouresIds = laggingRescouresIds;
-    }
-
-    public void serialize(OutputStream out) throws IOException {
-        DataOutputStream dos = new DataOutputStream(out);
-        dos.writeInt(laggingRescouresIds.size());
-        for (Long resourceId : laggingRescouresIds) {
-            dos.writeLong(resourceId);
-        }
-    }
-
-    public static ReplicaIndexFlushRequest create(DataInput input) throws IOException {
-        int numOfResources = input.readInt();
-        Set<Long> laggingRescouresIds = new HashSet<Long>(numOfResources);
-        for (int i = 0; i < numOfResources; i++) {
-            laggingRescouresIds.add(input.readLong());
-        }
-        return new ReplicaIndexFlushRequest(laggingRescouresIds);
-    }
-
-    public Set<Long> getLaggingRescouresIds() {
-        return laggingRescouresIds;
-    }
-
-    public void setLaggingRescouresIds(Set<Long> laggingRescouresIds) {
-        this.laggingRescouresIds = laggingRescouresIds;
-    }
-
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaLogsRequest.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaLogsRequest.java
deleted file mode 100644
index 90df1e7..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaLogsRequest.java
+++ /dev/null
@@ -1,71 +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.replication.functions;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.HashSet;
-import java.util.Set;
-
-public class ReplicaLogsRequest {
-    Set<String> replicaIds;
-    long fromLSN;
-
-    public ReplicaLogsRequest(Set<String> replicaIds, long fromLSN) {
-        this.replicaIds = replicaIds;
-        this.fromLSN = fromLSN;
-    }
-
-    public void serialize(OutputStream out) throws IOException {
-        DataOutputStream dos = new DataOutputStream(out);
-        dos.writeInt(replicaIds.size());
-        for (String replicaId : replicaIds) {
-            dos.writeUTF(replicaId);
-        }
-        dos.writeLong(fromLSN);
-    }
-
-    public static ReplicaLogsRequest create(DataInput input) throws IOException {
-        int size = input.readInt();
-        Set<String> replicaIds = new HashSet<String>(size);
-        for (int i = 0; i < size; i++) {
-            replicaIds.add(input.readUTF());
-        }
-        long fromLSN = input.readLong();
-        return new ReplicaLogsRequest(replicaIds, fromLSN);
-    }
-
-    public Set<String> getReplicaIds() {
-        return replicaIds;
-    }
-
-    public void setReplicaIds(Set<String> replicaIds) {
-        this.replicaIds = replicaIds;
-    }
-
-    public long getFromLSN() {
-        return fromLSN;
-    }
-
-    public void setFromLSN(long fromLSN) {
-        this.fromLSN = fromLSN;
-    }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java
deleted file mode 100644
index 8094548..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java
+++ /dev/null
@@ -1,354 +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.replication.functions;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SocketChannel;
-
-import org.apache.asterix.common.exceptions.ReplicationException;
-import org.apache.asterix.common.replication.ReplicaEvent;
-import org.apache.asterix.replication.api.IReplicationMessage;
-import org.apache.asterix.replication.management.NetworkingUtil;
-import org.apache.asterix.replication.messaging.CheckpointPartitionIndexesTask;
-import org.apache.asterix.replication.messaging.DeleteFileTask;
-import org.apache.asterix.replication.messaging.PartitionResourcesListResponse;
-import org.apache.asterix.replication.messaging.PartitionResourcesListTask;
-import org.apache.asterix.replication.messaging.ReplicateFileTask;
-import org.apache.asterix.replication.storage.LSMComponentProperties;
-import org.apache.asterix.replication.storage.LSMIndexFileProperties;
-import org.apache.asterix.replication.storage.PartitionReplica;
-import org.apache.hyracks.data.std.util.ExtendedByteArrayOutputStream;
-
-public class ReplicationProtocol {
-
-    /**
-     * All replication messages start with ReplicationRequestType (4 bytes), then the length of the request in bytes
-     */
-    public static final String JOB_REPLICATION_ACK = "$";
-
-    public static final  int REPLICATION_REQUEST_TYPE_SIZE = Integer.BYTES;
-    private static final  int REPLICATION_REQUEST_HEADER_SIZE = REPLICATION_REQUEST_TYPE_SIZE + Integer.BYTES;
-
-    /*
-     * ReplicationRequestType:
-     * REPLICATE_LOG: txn log replication
-     * REPLICATE_FILE: replicate a file(s)
-     * DELETE_FILE: delete a file(s)
-     * GET_REPLICA_FILES: used during remote recovery to request lost LSM Components
-     * GET_REPLICA_MAX_LSN: used during remote recovery initialize a log manager LSN
-     * GOODBYE: used to notify replicas that the replication request has been completed
-     * REPLICA_EVENT: used to notify replicas about a remote replica split/merge.
-     * LSM_COMPONENT_PROPERTIES: used to send the properties of an LSM Component before its physical files are sent
-     * ACK: used to notify the requesting replica that the request has been completed successfully
-     * FLUSH_INDEX: request remote replica to flush an LSM component
-     */
-    public enum ReplicationRequestType {
-        REPLICATE_LOG,
-        REPLICATE_FILE,
-        DELETE_FILE,
-        GET_REPLICA_FILES,
-        GET_REPLICA_MAX_LSN,
-        GOODBYE,
-        REPLICA_EVENT,
-        LSM_COMPONENT_PROPERTIES,
-        ACK,
-        FLUSH_INDEX,
-        PARTITION_RESOURCES_REQUEST,
-        PARTITION_RESOURCES_RESPONSE,
-        REPLICATE_RESOURCE_FILE,
-        DELETE_RESOURCE_FILE,
-        CHECKPOINT_PARTITION
-    }
-
-    public static ByteBuffer readRequest(SocketChannel socketChannel, ByteBuffer dataBuffer) throws IOException {
-        //read request size
-        NetworkingUtil.readBytes(socketChannel, dataBuffer, Integer.BYTES);
-        int requestSize = dataBuffer.getInt();
-
-        if (dataBuffer.capacity() < requestSize) {
-            dataBuffer = ByteBuffer.allocate(requestSize);
-        }
-
-        //read request
-        NetworkingUtil.readBytes(socketChannel, dataBuffer, requestSize);
-
-        return dataBuffer;
-    }
-
-    public static ByteBuffer writeLSMComponentPropertiesRequest(LSMComponentProperties lsmCompProp, ByteBuffer buffer)
-            throws IOException {
-        ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
-        try (DataOutputStream oos = new DataOutputStream(outputStream)) {
-            lsmCompProp.serialize(oos);
-            int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
-            if (buffer.capacity() < requestSize) {
-                buffer = ByteBuffer.allocate(requestSize);
-            } else {
-                buffer.clear();
-            }
-            buffer.putInt(ReplicationRequestType.LSM_COMPONENT_PROPERTIES.ordinal());
-            buffer.putInt(oos.size());
-            buffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
-            buffer.flip();
-            return buffer;
-        }
-    }
-
-    public static ReplicationRequestType getRequestType(SocketChannel socketChannel, ByteBuffer byteBuffer)
-            throws IOException {
-        //read replication request type
-        NetworkingUtil.readBytes(socketChannel, byteBuffer, REPLICATION_REQUEST_TYPE_SIZE);
-
-        ReplicationRequestType requestType = ReplicationProtocol.ReplicationRequestType.values()[byteBuffer.getInt()];
-        return requestType;
-    }
-
-    public static LSMComponentProperties readLSMPropertiesRequest(ByteBuffer buffer) throws IOException {
-        ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
-        DataInputStream dis = new DataInputStream(bais);
-        return LSMComponentProperties.create(dis);
-    }
-
-    public static ByteBuffer getGoodbyeBuffer() {
-        ByteBuffer bb = ByteBuffer.allocate(REPLICATION_REQUEST_TYPE_SIZE);
-        bb.putInt(ReplicationRequestType.GOODBYE.ordinal());
-        bb.flip();
-        return bb;
-    }
-
-    public static ByteBuffer getAckBuffer() {
-        ByteBuffer bb = ByteBuffer.allocate(REPLICATION_REQUEST_TYPE_SIZE);
-        bb.putInt(ReplicationRequestType.ACK.ordinal());
-        bb.flip();
-        return bb;
-    }
-
-    public static ByteBuffer writeFileReplicationRequest(ByteBuffer requestBuffer, LSMIndexFileProperties afp,
-            ReplicationRequestType requestType) throws IOException {
-        ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
-        try (DataOutputStream oos = new DataOutputStream(outputStream)) {
-            afp.serialize(oos);
-            int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
-            if (requestBuffer.capacity() < requestSize) {
-                requestBuffer = ByteBuffer.allocate(requestSize);
-            } else {
-                requestBuffer.clear();
-            }
-            requestBuffer.putInt(requestType.ordinal());
-            requestBuffer.putInt(oos.size());
-            requestBuffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
-            requestBuffer.flip();
-            return requestBuffer;
-        }
-    }
-
-    public static LSMIndexFileProperties readFileReplicationRequest(ByteBuffer buffer) throws IOException {
-        ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
-        DataInputStream dis = new DataInputStream(bais);
-        return LSMIndexFileProperties.create(dis);
-    }
-
-    public static ByteBuffer writeReplicaEventRequest(ReplicaEvent event) throws IOException {
-        ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
-        try (DataOutputStream oos = new DataOutputStream(outputStream)) {
-            event.serialize(oos);
-            ByteBuffer buffer = ByteBuffer.allocate(REPLICATION_REQUEST_HEADER_SIZE + oos.size());
-            buffer.putInt(ReplicationRequestType.REPLICA_EVENT.ordinal());
-            buffer.putInt(oos.size());
-            buffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
-            buffer.flip();
-            return buffer;
-        }
-    }
-
-    public static ReplicaEvent readReplicaEventRequest(ByteBuffer buffer) throws IOException {
-        ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
-        DataInputStream dis = new DataInputStream(bais);
-
-        return ReplicaEvent.create(dis);
-    }
-
-    public static ByteBuffer writeGetReplicaFilesRequest(ByteBuffer buffer, ReplicaFilesRequest request)
-            throws IOException {
-        ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
-        try (DataOutputStream oos = new DataOutputStream(outputStream)) {
-            request.serialize(oos);
-
-            int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
-            if (buffer.capacity() < requestSize) {
-                buffer = ByteBuffer.allocate(requestSize);
-            } else {
-                buffer.clear();
-            }
-            buffer.putInt(ReplicationRequestType.GET_REPLICA_FILES.ordinal());
-            buffer.putInt(oos.size());
-            buffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
-            buffer.flip();
-            return buffer;
-        }
-    }
-
-    public static ByteBuffer writeGetReplicaIndexFlushRequest(ByteBuffer buffer, ReplicaIndexFlushRequest request)
-            throws IOException {
-        ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
-        try (DataOutputStream oos = new DataOutputStream(outputStream)) {
-            request.serialize(oos);
-            int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
-            if (buffer.capacity() < requestSize) {
-                buffer = ByteBuffer.allocate(requestSize);
-            } else {
-                buffer.clear();
-            }
-            buffer.putInt(ReplicationRequestType.FLUSH_INDEX.ordinal());
-            buffer.putInt(oos.size());
-            buffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
-            buffer.flip();
-            return buffer;
-        }
-    }
-
-    public static ReplicaFilesRequest readReplicaFileRequest(ByteBuffer buffer) throws IOException {
-        ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
-        DataInputStream dis = new DataInputStream(bais);
-        return ReplicaFilesRequest.create(dis);
-    }
-
-    public static ReplicaIndexFlushRequest readReplicaIndexFlushRequest(ByteBuffer buffer) throws IOException {
-        ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
-        DataInputStream dis = new DataInputStream(bais);
-        return ReplicaIndexFlushRequest.create(dis);
-    }
-
-    public static void writeGetReplicaMaxLSNRequest(ByteBuffer requestBuffer) {
-        requestBuffer.clear();
-        requestBuffer.putInt(ReplicationRequestType.GET_REPLICA_MAX_LSN.ordinal());
-        requestBuffer.flip();
-    }
-
-    public static int getJobIdFromLogAckMessage(String msg) {
-        return Integer.parseInt(msg.substring((msg.indexOf(JOB_REPLICATION_ACK) + 1)));
-    }
-
-    public static String getNodeIdFromLogAckMessage(String msg) {
-        return msg.substring(0, msg.indexOf(JOB_REPLICATION_ACK));
-    }
-
-    /**
-     * Sends a goodbye request to a remote replica indicating the end of a replication request.
-     *
-     * @param socketChannel
-     *            the remote replica socket.
-     * @throws IOException
-     */
-    public static void sendGoodbye(SocketChannel socketChannel) throws IOException {
-        ByteBuffer goodbyeBuffer = ReplicationProtocol.getGoodbyeBuffer();
-        NetworkingUtil.transferBufferToChannel(socketChannel, goodbyeBuffer);
-    }
-
-    public static void sendAck(SocketChannel socketChannel) throws IOException {
-        ByteBuffer ackBuffer = ReplicationProtocol.getAckBuffer();
-        NetworkingUtil.transferBufferToChannel(socketChannel, ackBuffer);
-    }
-
-    public static void sendAck(SocketChannel socketChannel, ByteBuffer buf) {
-        try {
-            buf.clear();
-            buf.putInt(ReplicationRequestType.ACK.ordinal());
-            buf.flip();
-            NetworkingUtil.transferBufferToChannel(socketChannel, buf);
-        } catch (IOException e) {
-            throw new ReplicationException(e);
-        }
-    }
-
-    public static void waitForAck(PartitionReplica replica) throws IOException {
-        final SocketChannel channel = replica.getChannel();
-        final ByteBuffer buf = replica.gerReusableBuffer();
-        ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(channel, buf);
-        if (responseFunction != ReplicationRequestType.ACK) {
-            throw new IllegalStateException("Unexpected response while waiting for ack.");
-        }
-    }
-
-    public static void sendTo(PartitionReplica replica, IReplicationMessage task) {
-        final SocketChannel channel = replica.getChannel();
-        final ByteBuffer buf = replica.gerReusableBuffer();
-        sendTo(channel, task, buf);
-    }
-
-    public static void sendTo(SocketChannel channel, IReplicationMessage task, ByteBuffer buf) {
-        ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
-        try (DataOutputStream oos = new DataOutputStream(outputStream)) {
-            task.serialize(oos);
-            final int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
-            final ByteBuffer requestBuffer = ensureSize(buf, requestSize);
-            requestBuffer.putInt(task.getMessageType().ordinal());
-            requestBuffer.putInt(oos.size());
-            requestBuffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
-            requestBuffer.flip();
-            NetworkingUtil.transferBufferToChannel(channel, requestBuffer);
-        } catch (IOException e) {
-            throw new ReplicationException(e);
-        }
-    }
-
-    public static IReplicationMessage read(SocketChannel socketChannel, ByteBuffer buffer) throws IOException {
-        final ReplicationRequestType type = getRequestType(socketChannel, buffer);
-        return readMessage(type, socketChannel, buffer);
-    }
-
-    public static IReplicationMessage readMessage(ReplicationRequestType type, SocketChannel socketChannel,
-            ByteBuffer buffer) {
-        try {
-            ReplicationProtocol.readRequest(socketChannel, buffer);
-            final ByteArrayInputStream input =
-                    new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
-            try (DataInputStream dis = new DataInputStream(input)) {
-                switch (type) {
-                    case PARTITION_RESOURCES_REQUEST:
-                        return PartitionResourcesListTask.create(dis);
-                    case PARTITION_RESOURCES_RESPONSE:
-                        return PartitionResourcesListResponse.create(dis);
-                    case REPLICATE_RESOURCE_FILE:
-                        return ReplicateFileTask.create(dis);
-                    case DELETE_RESOURCE_FILE:
-                        return DeleteFileTask.create(dis);
-                    case CHECKPOINT_PARTITION:
-                        return CheckpointPartitionIndexesTask.create(dis);
-                    default:
-                        throw new IllegalStateException("Unrecognized replication message");
-                }
-            }
-        } catch (IOException e) {
-            throw new ReplicationException(e);
-        }
-    }
-
-    private static ByteBuffer ensureSize(ByteBuffer buffer, int size) {
-        if (buffer.capacity() < size) {
-            return ByteBuffer.allocate(size);
-        }
-        buffer.clear();
-        return buffer;
-    }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogMapping.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogMapping.java
deleted file mode 100644
index 0444952..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogMapping.java
+++ /dev/null
@@ -1,62 +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.replication.logging;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class RemoteLogMapping {
-
-    private String remoteNodeID;
-    private long remoteLSN;
-    private long localLSN;
-    public AtomicInteger numOfFlushedIndexes = new AtomicInteger();
-
-    public void setRemoteNodeID(String remoteNodeID) {
-        this.remoteNodeID = remoteNodeID;
-    }
-
-    public long getRemoteLSN() {
-        return remoteLSN;
-    }
-
-    public void setRemoteLSN(long remoteLSN) {
-        this.remoteLSN = remoteLSN;
-    }
-
-    public long getLocalLSN() {
-        return localLSN;
-    }
-
-    public void setLocalLSN(long localLSN) {
-        this.localLSN = localLSN;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("Remote Node: " + remoteNodeID);
-        sb.append(" Remote LSN: " + remoteLSN);
-        sb.append(" Local LSN: " + localLSN);
-        return sb.toString();
-    }
-
-    public String getNodeUniqueLSN() {
-        return TxnLogUtil.getNodeUniqueLSN(remoteNodeID, remoteLSN);
-    }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogRecord.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogRecord.java
new file mode 100644
index 0000000..0b496b8
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogRecord.java
@@ -0,0 +1,44 @@
+/*
+ * 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.replication.logging;
+
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.common.transactions.LogRecord;
+
+public class RemoteLogRecord extends LogRecord {
+
+    private long masterLsn;
+    private IReplicationWorker replicationWorker;
+
+    public long getMasterLsn() {
+        return masterLsn;
+    }
+
+    public void setMasterLsn(long masterLsn) {
+        this.masterLsn = masterLsn;
+    }
+
+    public IReplicationWorker getReplicationWorker() {
+        return replicationWorker;
+    }
+
+    public void setReplicationWorker(IReplicationWorker replicationWorker) {
+        this.replicationWorker = replicationWorker;
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java
new file mode 100644
index 0000000..d63496f
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java
@@ -0,0 +1,102 @@
+/*
+ * 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.replication.logging;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.dataflow.DatasetLocalResource;
+import org.apache.asterix.common.storage.DatasetResourceReference;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.transactions.LogType;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.LocalResource;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+class RemoteLogsNotifier implements Runnable {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final PersistentLocalResourceRepository localResourceRep;
+    private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
+    private final LinkedBlockingQueue<RemoteLogRecord> remoteLogsQ;
+    private final INcApplicationContext appCtx;
+
+    public RemoteLogsNotifier(INcApplicationContext appCtx, LinkedBlockingQueue<RemoteLogRecord> remoteLogsQ) {
+        this.appCtx = appCtx;
+        this.remoteLogsQ = remoteLogsQ;
+        localResourceRep = (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+        indexCheckpointManagerProvider = appCtx.getIndexCheckpointManagerProvider();
+    }
+
+    @Override
+    public void run() {
+        final String nodeId = appCtx.getServiceContext().getNodeId();
+        Thread.currentThread().setName(nodeId + RemoteLogsNotifier.class.getSimpleName());
+        while (!Thread.currentThread().isInterrupted()) {
+            try {
+                final RemoteLogRecord logRecord = remoteLogsQ.take();
+                switch (logRecord.getLogType()) {
+                    case LogType.JOB_COMMIT:
+                    case LogType.ABORT:
+                        // send ACK to requester
+                        logRecord.getReplicationWorker().getChannel().socket().getOutputStream()
+                                .write((nodeId + ReplicationProtocol.LOG_REPLICATION_ACK + logRecord.getTxnId() + System
+                                        .lineSeparator()).getBytes());
+                        break;
+                    case LogType.FLUSH:
+                        checkpointReplicaIndexes(logRecord, logRecord.getDatasetId());
+                        break;
+                    default:
+                        throw new IllegalStateException("Unexpected log type: " + logRecord.getLogType());
+                }
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            } catch (IOException e) {
+                LOGGER.error("Failed to process replicated log", e);
+            }
+        }
+    }
+
+    private void checkpointReplicaIndexes(RemoteLogRecord remoteLogMapping, int datasetId) throws HyracksDataException {
+        final Set<Integer> masterPartitions = appCtx.getReplicaManager().getPartitions();
+        final Predicate<LocalResource> replicaIndexesPredicate = lr -> {
+            DatasetLocalResource dls = (DatasetLocalResource) lr.getResource();
+            return dls.getDatasetId() == datasetId && !masterPartitions.contains(dls.getPartition());
+        };
+        final Map<Long, LocalResource> resources = localResourceRep.getResources(replicaIndexesPredicate);
+        final List<DatasetResourceReference> replicaIndexesRef =
+                resources.values().stream().map(DatasetResourceReference::of).collect(Collectors.toList());
+        for (DatasetResourceReference replicaIndexRef : replicaIndexesRef) {
+            final IIndexCheckpointManager indexCheckpointManager = indexCheckpointManagerProvider.get(replicaIndexRef);
+            synchronized (indexCheckpointManager) {
+                indexCheckpointManager.masterFlush(remoteLogMapping.getMasterLsn(), remoteLogMapping.getLSN());
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsProcessor.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsProcessor.java
new file mode 100644
index 0000000..c054e6c
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsProcessor.java
@@ -0,0 +1,88 @@
+/*
+ * 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.replication.logging;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.transactions.ILogManager;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.ILogRequester;
+import org.apache.asterix.common.transactions.LogSource;
+import org.apache.asterix.common.transactions.LogType;
+import org.apache.asterix.common.utils.TransactionUtil;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class RemoteLogsProcessor implements ILogRequester {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final LinkedBlockingQueue<RemoteLogRecord> remoteLogsQ;
+    private final ILogManager logManager;
+
+    public RemoteLogsProcessor(INcApplicationContext appCtx) {
+        logManager = appCtx.getTransactionSubsystem().getLogManager();
+        remoteLogsQ = new LinkedBlockingQueue<>();
+        appCtx.getThreadExecutor().execute(new RemoteLogsNotifier(appCtx, remoteLogsQ));
+    }
+
+    public void process(ByteBuffer logsBatch, RemoteLogRecord reusableLog, IReplicationWorker worker) {
+        while (logsBatch.hasRemaining()) {
+            // get rid of log size
+            logsBatch.getInt();
+            reusableLog.readRemoteLog(logsBatch);
+            reusableLog.setLogSource(LogSource.REMOTE);
+            switch (reusableLog.getLogType()) {
+                case LogType.UPDATE:
+                case LogType.ENTITY_COMMIT:
+                    logManager.log(reusableLog);
+                    break;
+                case LogType.JOB_COMMIT:
+                case LogType.ABORT:
+                    RemoteLogRecord jobTerminationLog = new RemoteLogRecord();
+                    TransactionUtil.formJobTerminateLogRecord(jobTerminationLog, reusableLog.getTxnId(),
+                            reusableLog.getLogType() == LogType.JOB_COMMIT);
+                    jobTerminationLog.setRequester(this);
+                    jobTerminationLog.setReplicationWorker(worker);
+                    jobTerminationLog.setLogSource(LogSource.REMOTE);
+                    logManager.log(jobTerminationLog);
+                    break;
+                case LogType.FLUSH:
+                    RemoteLogRecord flushLog = new RemoteLogRecord();
+                    TransactionUtil
+                            .formFlushLogRecord(flushLog, reusableLog.getDatasetId(), null, reusableLog.getNodeId(),
+                                    reusableLog.getNumOfFlushedIndexes());
+                    flushLog.setRequester(this);
+                    flushLog.setLogSource(LogSource.REMOTE);
+                    flushLog.setMasterLsn(reusableLog.getLSN());
+                    logManager.log(flushLog);
+                    break;
+                default:
+                    LOGGER.error(() -> "Unsupported LogType: " + reusableLog.getLogType());
+            }
+        }
+    }
+
+    @Override
+    public void notifyFlushed(ILogRecord logRecord) {
+        remoteLogsQ.add((RemoteLogRecord) logRecord);
+    }
+}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
index 283f69f..a94f073 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
@@ -22,9 +22,10 @@
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.asterix.common.transactions.ILogRecord;
-import org.apache.asterix.replication.management.ReplicationManager;
+import org.apache.asterix.replication.management.LogReplicationManager;
 
 public class ReplicationLogBuffer {
+
     private final int logBufferSize;
     private final AtomicBoolean full;
     private int appendOffset;
@@ -32,10 +33,10 @@
     private final ByteBuffer appendBuffer;
     private final ByteBuffer replicationBuffer;
     private boolean stop;
-    private ReplicationManager replicationManager;
+    private final LogReplicationManager replicationManager;
     private final int batchSize;
 
-    public ReplicationLogBuffer(ReplicationManager replicationManager, int logBufferSize, int batchSize) {
+    public ReplicationLogBuffer(LogReplicationManager replicationManager, int logBufferSize, int batchSize) {
         this.replicationManager = replicationManager;
         this.logBufferSize = logBufferSize;
         this.batchSize = batchSize;
@@ -116,7 +117,7 @@
     private void transferBuffer(ByteBuffer buffer) {
         if (buffer.remaining() <= batchSize) {
             //the current batch can be sent as it is
-            replicationManager.replicateTxnLogBatch(buffer);
+            replicationManager.transferBatch(buffer);
             return;
         }
         /**
@@ -141,7 +142,7 @@
                 //return to the beginning of the batch position
                 buffer.reset();
             }
-            replicationManager.replicateTxnLogBatch(buffer);
+            replicationManager.transferBatch(buffer);
             //return the original limit to check the new remaining size
             buffer.limit(totalTransferLimit);
         }
@@ -159,7 +160,7 @@
         return logBufferSize;
     }
 
-    public ReplicationManager getReplicationManager() {
+    public LogReplicationManager getReplicationManager() {
         return replicationManager;
     }
 }
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnAck.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnAck.java
new file mode 100644
index 0000000..1b0ac23
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnAck.java
@@ -0,0 +1,49 @@
+/*
+ * 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.replication.logging;
+
+import java.util.Set;
+
+import org.apache.asterix.common.replication.IReplicationDestination;
+import org.apache.asterix.common.transactions.ILogRecord;
+
+public class TxnAck {
+
+    private final Set<IReplicationDestination> pendingAcks;
+    private final ILogRecord logRecord;
+
+    public TxnAck(ILogRecord logRecord, Set<IReplicationDestination> pendingAcks) {
+        this.logRecord = logRecord;
+        this.pendingAcks = pendingAcks;
+    }
+
+    public synchronized void ack(IReplicationDestination dest) {
+        pendingAcks.remove(dest);
+        if (allAcked()) {
+            synchronized (logRecord) {
+                logRecord.setReplicated(true);
+                logRecord.notifyAll();
+            }
+        }
+    }
+
+    public synchronized boolean allAcked() {
+        return pendingAcks.isEmpty();
+    }
+}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnAckTracker.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnAckTracker.java
new file mode 100644
index 0000000..23e6f3c
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnAckTracker.java
@@ -0,0 +1,63 @@
+/*
+ * 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.replication.logging;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.replication.IReplicationDestination;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class TxnAckTracker {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final Map<Long, TxnAck> txnsAcks = new HashMap<>();
+
+    public synchronized void track(ILogRecord logRecord, Set<IReplicationDestination> replicas) {
+        if (replicas.isEmpty()) {
+            logRecord.setReplicated(true);
+            return;
+        }
+        final long txnId = logRecord.getTxnId();
+        //TODO use LSN instead of txnId when all logs have LSN
+        txnsAcks.put(txnId, new TxnAck(logRecord, replicas));
+    }
+
+    public synchronized void ack(long txnId, IReplicationDestination replica) {
+        if (!txnsAcks.containsKey(txnId)) {
+            LOGGER.warn("Received ack for unknown txn {}", txnId);
+            return;
+        }
+        TxnAck txnAcks = txnsAcks.get(txnId);
+        txnAcks.ack(replica);
+        if (txnAcks.allAcked()) {
+            txnsAcks.remove(txnId);
+        }
+    }
+
+    public synchronized void unregister(IReplicationDestination replica) {
+        // assume the ack was received from leaving replicas
+        final HashSet<Long> pendingTxn = new HashSet<>(txnsAcks.keySet());
+        pendingTxn.forEach(txnId -> ack(txnId, replica));
+    }
+}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnLogUtil.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnLogUtil.java
deleted file mode 100644
index f51a64d..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnLogUtil.java
+++ /dev/null
@@ -1,35 +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.replication.logging;
-
-public class TxnLogUtil {
-
-    private TxnLogUtil() {
-        //prevent util class construction
-    }
-
-    /**
-     * @param nodeId
-     * @param LSN
-     * @return Concatenation of nodeId and LSN
-     */
-    public static String getNodeUniqueLSN(String nodeId, long LSN) {
-        return nodeId + LSN;
-    }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java
new file mode 100644
index 0000000..b6f752f
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java
@@ -0,0 +1,184 @@
+/*
+ * 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.replication.management;
+
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.common.replication.IPartitionReplica;
+import org.apache.asterix.common.replication.IReplicationDestination;
+import org.apache.asterix.common.replication.IReplicationManager;
+import org.apache.asterix.common.replication.IReplicationStrategy;
+import org.apache.asterix.common.storage.DatasetResourceReference;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.replication.api.PartitionReplica;
+import org.apache.asterix.replication.api.ReplicationDestination;
+import org.apache.asterix.replication.sync.IndexSynchronizer;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.replication.IReplicationJob;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class IndexReplicationManager {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final IReplicationManager replicationManager;
+    private final Set<ReplicationDestination> destinations = new HashSet<>();
+    private final LinkedBlockingQueue<IReplicationJob> replicationJobsQ = new LinkedBlockingQueue<>();
+    private final IReplicationStrategy replicationStrategy;
+    private final PersistentLocalResourceRepository resourceRepository;
+    private final INcApplicationContext appCtx;
+    private final Object transferLock = new Object();
+    private final Set<ReplicationDestination> failedDest = new HashSet<>();
+
+    public IndexReplicationManager(INcApplicationContext appCtx, IReplicationManager replicationManager) {
+        this.appCtx = appCtx;
+        this.replicationManager = replicationManager;
+        this.resourceRepository = (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+        replicationStrategy = replicationManager.getReplicationStrategy();
+        appCtx.getThreadExecutor().execute(new ReplicationJobsProcessor());
+    }
+
+    public void register(ReplicationDestination dest) {
+        synchronized (transferLock) {
+            LOGGER.info(() -> "register " + dest);
+            destinations.add(dest);
+            failedDest.remove(dest);
+        }
+    }
+
+    public void unregister(IReplicationDestination dest) {
+        synchronized (transferLock) {
+            LOGGER.info(() -> "unregister " + dest);
+            destinations.remove(dest);
+            failedDest.remove(dest);
+        }
+    }
+
+    private void handleFailure(ReplicationDestination dest, Exception e) {
+        synchronized (transferLock) {
+            if (failedDest.contains(dest)) {
+                return;
+            }
+            LOGGER.error("Replica failed", e);
+            if (destinations.contains(dest)) {
+                failedDest.add(dest);
+            }
+            replicationManager.notifyFailure(dest, e);
+        }
+    }
+
+    public void accept(IReplicationJob job) {
+        if (job.getExecutionType() == IReplicationJob.ReplicationExecutionType.ASYNC) {
+            replicationJobsQ.add(job);
+            return;
+        }
+        process(job);
+    }
+
+    private void process(IReplicationJob job) {
+        try {
+            if (skip(job)) {
+                return;
+            }
+            synchronized (transferLock) {
+                if (destinations.isEmpty()) {
+                    return;
+                }
+                final IndexSynchronizer synchronizer = new IndexSynchronizer(job, appCtx);
+                final int indexPartition = getJobPartition(job);
+                for (ReplicationDestination dest : destinations) {
+                    try {
+                        Optional<IPartitionReplica> partitionReplica = dest.getPartitionReplica(indexPartition);
+                        if (!partitionReplica.isPresent()) {
+                            continue;
+                        }
+                        PartitionReplica replica = (PartitionReplica) partitionReplica.get();
+                        synchronizer.sync(replica);
+                    } catch (Exception e) {
+                        handleFailure(dest, e);
+                    }
+                }
+                closeChannels();
+            }
+        } finally {
+            afterReplication(job);
+        }
+    }
+
+    private boolean skip(IReplicationJob job) {
+        try {
+            final DatasetResourceReference indexFileRef =
+                    resourceRepository.getLocalResourceReference(job.getAnyFile());
+            return !replicationStrategy.isMatch(indexFileRef.getDatasetId());
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException("Couldn't find resource for " + job.getAnyFile(), e);
+        }
+    }
+
+    private int getJobPartition(IReplicationJob job) {
+        return ResourceReference.of(job.getAnyFile()).getPartitionNum();
+    }
+
+    private void closeChannels() {
+        if (!replicationJobsQ.isEmpty()) {
+            return;
+        }
+        LOGGER.log(Level.INFO, "No pending replication jobs. Closing connections to replicas");
+        for (ReplicationDestination dest : destinations) {
+            dest.getReplicas().stream().map(PartitionReplica.class::cast).forEach(PartitionReplica::close);
+        }
+    }
+
+    private static void afterReplication(IReplicationJob job) {
+        try {
+            if (job.getOperation() == IReplicationJob.ReplicationOperation.REPLICATE
+                    && job instanceof ILSMIndexReplicationJob) {
+                ((ILSMIndexReplicationJob) job).endReplication();
+            }
+        } catch (HyracksDataException e) {
+            throw new ReplicationException(e);
+        }
+    }
+
+    private class ReplicationJobsProcessor implements Runnable {
+
+        @Override
+        public void run() {
+            Thread.currentThread().setName(ReplicationJobsProcessor.class.getSimpleName());
+            while (!Thread.currentThread().isInterrupted()) {
+                try {
+                    final IReplicationJob job = replicationJobsQ.take();
+                    process(job);
+                } catch (InterruptedException e) {
+                    LOGGER.warn(() -> ReplicationJobsProcessor.class.getSimpleName() + " interrupted.", e);
+                    Thread.currentThread().interrupt();
+                }
+            }
+            LOGGER.warn("{} stopped.", ReplicationJobsProcessor.class.getSimpleName());
+        }
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
new file mode 100644
index 0000000..b28c2f7
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
@@ -0,0 +1,258 @@
+/*
+ * 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.replication.management;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.ByteBuffer;
+import java.nio.channels.AsynchronousCloseException;
+import java.nio.channels.SocketChannel;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.config.ReplicationProperties;
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.common.replication.IReplicationDestination;
+import org.apache.asterix.common.replication.IReplicationManager;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.LogType;
+import org.apache.asterix.replication.api.ReplicationDestination;
+import org.apache.asterix.replication.logging.ReplicationLogBuffer;
+import org.apache.asterix.replication.logging.TxnAckTracker;
+import org.apache.asterix.replication.logging.TxnLogReplicator;
+import org.apache.asterix.replication.messaging.ReplicateLogsTask;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class LogReplicationManager {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final LinkedBlockingQueue<ReplicationLogBuffer> emptyLogBuffersQ;
+    private final LinkedBlockingQueue<ReplicationLogBuffer> pendingFlushLogBuffersQ;
+    private final ByteBuffer txnLogsBatchSizeBuffer = ByteBuffer.allocate(Integer.BYTES);
+    private final Map<ReplicationDestination, SocketChannel> destinations = new HashMap<>();
+    private final IReplicationManager replicationManager;
+    private final Executor executor;
+    private final TxnAckTracker ackTracker = new TxnAckTracker();
+    private final Set<SocketChannel> failedSockets = new HashSet<>();
+    private final Object transferLock = new Object();
+    private final INcApplicationContext appCtx;
+    private final int logPageSize;
+    private final int logBatchSize;
+    private ReplicationLogBuffer currentTxnLogBuffer;
+    private SocketChannel[] destSockets;
+
+    public LogReplicationManager(INcApplicationContext appCtx, IReplicationManager replicationManager) {
+        this.appCtx = appCtx;
+        this.replicationManager = replicationManager;
+        final ReplicationProperties replicationProperties = appCtx.getReplicationProperties();
+        logPageSize = replicationProperties.getLogBufferPageSize();
+        logBatchSize = replicationProperties.getLogBatchSize();
+        executor = appCtx.getThreadExecutor();
+        emptyLogBuffersQ = new LinkedBlockingQueue<>();
+        pendingFlushLogBuffersQ = new LinkedBlockingQueue<>();
+        initBuffers(replicationProperties.getLogBufferNumOfPages());
+        TxnLogReplicator txnlogReplicator = new TxnLogReplicator(emptyLogBuffersQ, pendingFlushLogBuffersQ);
+        ((ExecutorService) executor).submit(txnlogReplicator);
+    }
+
+    private void initBuffers(int buffers) {
+        for (int i = 0; i < buffers; i++) {
+            emptyLogBuffersQ.add(new ReplicationLogBuffer(this, logPageSize, logBatchSize));
+        }
+        try {
+            getAndInitNewPage();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new ReplicationException(e);
+        }
+    }
+
+    public void register(ReplicationDestination dest) {
+        synchronized (transferLock) {
+            synchronized (destinations) {
+                if (destinations.containsKey(dest)) {
+                    return;
+                }
+                LOGGER.info(() -> "register " + dest);
+                SocketChannel socketChannel = dest.getLogReplicationChannel();
+                handshake(dest, socketChannel);
+                destinations.put(dest, socketChannel);
+                failedSockets.remove(socketChannel);
+                destSockets = destinations.values().toArray(new SocketChannel[destinations.size()]);
+            }
+        }
+    }
+
+    public void unregister(IReplicationDestination dest) {
+        synchronized (transferLock) {
+            synchronized (destinations) {
+                if (!destinations.containsKey(dest)) {
+                    return;
+                }
+                LOGGER.info(() -> "unregister " + dest);
+                ackTracker.unregister(dest);
+                SocketChannel destSocket = destinations.remove(dest);
+                failedSockets.remove(destSocket);
+                destSockets = destinations.values().toArray(new SocketChannel[destinations.size()]);
+                endReplication(destSocket);
+            }
+        }
+    }
+
+    public void replicate(ILogRecord logRecord) throws InterruptedException {
+        if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
+            synchronized (destinations) {
+                ackTracker.track(logRecord, new HashSet<>(destinations.keySet()));
+            }
+        }
+        appendToLogBuffer(logRecord);
+    }
+
+    public void transferBatch(final ByteBuffer buffer) {
+        // prepare the batch size buffer
+        txnLogsBatchSizeBuffer.clear();
+        txnLogsBatchSizeBuffer.putInt(buffer.remaining());
+        txnLogsBatchSizeBuffer.flip();
+
+        buffer.mark();
+        synchronized (transferLock) {
+            if (destSockets != null) {
+                for (SocketChannel replicaSocket : destSockets) {
+                    try {
+                        // send batch size then the batch itself
+                        NetworkingUtil.transferBufferToChannel(replicaSocket, txnLogsBatchSizeBuffer);
+                        NetworkingUtil.transferBufferToChannel(replicaSocket, buffer);
+                    } catch (IOException e) {
+                        handleFailure(replicaSocket, e);
+                    } finally {
+                        txnLogsBatchSizeBuffer.position(0);
+                        buffer.reset();
+                    }
+                }
+            }
+        }
+        // move the buffer position to the sent limit
+        buffer.position(buffer.limit());
+    }
+
+    public int getLogPageSize() {
+        return logPageSize;
+    }
+
+    private synchronized void appendToLogBuffer(ILogRecord logRecord) throws InterruptedException {
+        if (!currentTxnLogBuffer.hasSpace(logRecord)) {
+            currentTxnLogBuffer.isFull(true);
+            if (logRecord.getLogSize() > logPageSize) {
+                getAndInitNewLargePage(logRecord.getLogSize());
+            } else {
+                getAndInitNewPage();
+            }
+        }
+        currentTxnLogBuffer.append(logRecord);
+    }
+
+    private void getAndInitNewPage() throws InterruptedException {
+        currentTxnLogBuffer = null;
+        while (currentTxnLogBuffer == null) {
+            currentTxnLogBuffer = emptyLogBuffersQ.take();
+        }
+        currentTxnLogBuffer.reset();
+        pendingFlushLogBuffersQ.add(currentTxnLogBuffer);
+    }
+
+    private void getAndInitNewLargePage(int pageSize) {
+        // for now, alloc a new buffer for each large page
+        currentTxnLogBuffer = new ReplicationLogBuffer(this, pageSize, logBatchSize);
+        pendingFlushLogBuffersQ.add(currentTxnLogBuffer);
+    }
+
+    private void handshake(ReplicationDestination dest, SocketChannel socketChannel) {
+        final String nodeId = appCtx.getServiceContext().getNodeId();
+        final ReplicateLogsTask task = new ReplicateLogsTask(nodeId);
+        ReplicationProtocol.sendTo(socketChannel, task, null);
+        executor.execute(new TxnAckListener(dest, socketChannel));
+    }
+
+    private void endReplication(SocketChannel socketChannel) {
+        if (socketChannel.isConnected()) {
+            // end log replication (by sending a dummy log with a single byte)
+            final ByteBuffer endLogRepBuffer = ReplicationProtocol.getEndLogReplicationBuffer();
+            try {
+                NetworkingUtil.transferBufferToChannel(socketChannel, endLogRepBuffer);
+            } catch (IOException e) {
+                LOGGER.warn("Failed to end txn log", e);
+            }
+        }
+    }
+
+    private synchronized void handleFailure(SocketChannel replicaSocket, IOException e) {
+        if (failedSockets.contains(replicaSocket)) {
+            return;
+        }
+        LOGGER.error("Replica failed", e);
+        failedSockets.add(replicaSocket);
+        Optional<ReplicationDestination> socketDest =
+                destinations.entrySet().stream().filter(entry -> entry.getValue().equals(replicaSocket))
+                        .map(Map.Entry::getKey).findFirst();
+        socketDest.ifPresent(dest -> replicationManager.notifyFailure(dest, e));
+    }
+
+    private class TxnAckListener implements Runnable {
+        private final ReplicationDestination dest;
+        private final SocketChannel replicaSocket;
+
+        TxnAckListener(ReplicationDestination dest, SocketChannel replicaSocket) {
+            this.dest = dest;
+            this.replicaSocket = replicaSocket;
+        }
+
+        @Override
+        public void run() {
+            Thread.currentThread().setName("TxnAckListener (" + dest + ")");
+            LOGGER.info("Started listening on socket: {}", dest);
+            try (BufferedReader incomingResponse = new BufferedReader(
+                    new InputStreamReader(replicaSocket.socket().getInputStream()))) {
+                while (true) {
+                    final String response = incomingResponse.readLine();
+                    if (response == null) {
+                        handleFailure(replicaSocket, new IOException("Unexpected response from replica " + dest));
+                        break;
+                    }
+                    // read ACK
+                    final int txnId = ReplicationProtocol.getTxnIdFromLogAckMessage(response);
+                    ackTracker.ack(txnId, dest);
+                }
+            } catch (AsynchronousCloseException e) {
+                LOGGER.debug(() -> "Stopped listening on socket:" + dest, e);
+            } catch (IOException e) {
+                handleFailure(replicaSocket, e);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaStateChecker.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaStateChecker.java
deleted file mode 100644
index e260de5..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaStateChecker.java
+++ /dev/null
@@ -1,76 +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.replication.management;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.nio.channels.SocketChannel;
-import java.nio.channels.UnresolvedAddressException;
-import java.util.concurrent.Callable;
-
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.replication.Replica.ReplicaState;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
-
-public class ReplicaStateChecker implements Callable<Void> {
-
-    private final int WAIT_TIME = 2000;
-    private final Replica replica;
-    private final int replicationTimeOut;
-    private final ReplicationManager replicationManager;
-    private final boolean suspendReplication;
-
-    public ReplicaStateChecker(Replica replica, int replicationTimeOut, ReplicationManager replicationManager,
-            boolean suspendReplication) {
-        this.replica = replica;
-        this.replicationTimeOut = replicationTimeOut;
-        this.replicationManager = replicationManager;
-        this.suspendReplication = suspendReplication;
-    }
-
-    @Override
-    public Void call() throws Exception {
-        Thread.currentThread().setName("ReplicaConnector Thread");
-
-        long startTime = System.currentTimeMillis();
-        InetSocketAddress replicaAddress = InetSocketAddress.createUnresolved(replica.getClusterIp(),
-                replica.getPort());
-
-        while (true) {
-            try (SocketChannel connection = SocketChannel.open()) {
-                connection.configureBlocking(true);
-                connection.connect(new InetSocketAddress(replicaAddress.getHostString(), replicaAddress.getPort()));
-                ByteBuffer buffer = ReplicationProtocol.getGoodbyeBuffer();
-                connection.write(buffer);
-                replicationManager.updateReplicaState(replica.getId(), ReplicaState.ACTIVE, suspendReplication);
-                return null;
-            } catch (IOException | UnresolvedAddressException e) {
-                Thread.sleep(WAIT_TIME);
-
-                //check if connection to replica timed out
-                if (((System.currentTimeMillis() - startTime) / 1000) >= replicationTimeOut) {
-                    replicationManager.updateReplicaState(replica.getId(), ReplicaState.DEAD, suspendReplication);
-                    return null;
-                }
-            }
-        }
-    }
-
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
index c0863e2..f19f1cd 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
@@ -18,79 +18,23 @@
  */
 package org.apache.asterix.replication.management;
 
-import java.io.File;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.AsynchronousCloseException;
-import java.nio.channels.FileChannel;
 import java.nio.channels.ServerSocketChannel;
 import java.nio.channels.SocketChannel;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
 
-import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.config.ReplicationProperties;
-import org.apache.asterix.common.context.IndexInfo;
-import org.apache.asterix.common.dataflow.DatasetLocalResource;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.replication.IReplicaResourcesManager;
 import org.apache.asterix.common.replication.IReplicationChannel;
-import org.apache.asterix.common.replication.IReplicationManager;
-import org.apache.asterix.common.replication.IReplicationStrategy;
-import org.apache.asterix.common.replication.IReplicationThread;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.replication.ReplicaEvent;
-import org.apache.asterix.common.storage.DatasetResourceReference;
-import org.apache.asterix.common.storage.IIndexCheckpointManager;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.asterix.common.storage.ResourceReference;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
-import org.apache.asterix.common.transactions.ILogManager;
-import org.apache.asterix.common.transactions.LogRecord;
-import org.apache.asterix.common.transactions.LogSource;
-import org.apache.asterix.common.transactions.LogType;
-import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.common.utils.TransactionUtil;
-import org.apache.asterix.replication.functions.ReplicaFilesRequest;
-import org.apache.asterix.replication.functions.ReplicaIndexFlushRequest;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
-import org.apache.asterix.replication.functions.ReplicationProtocol.ReplicationRequestType;
-import org.apache.asterix.replication.logging.RemoteLogMapping;
-import org.apache.asterix.replication.messaging.CheckpointPartitionIndexesTask;
-import org.apache.asterix.replication.messaging.DeleteFileTask;
-import org.apache.asterix.replication.messaging.PartitionResourcesListTask;
-import org.apache.asterix.replication.messaging.ReplicateFileTask;
-import org.apache.asterix.replication.storage.LSMComponentLSNSyncTask;
-import org.apache.asterix.replication.storage.LSMComponentProperties;
-import org.apache.asterix.replication.storage.LSMIndexFileProperties;
-import org.apache.asterix.replication.storage.ReplicaResourcesManager;
-import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
-import org.apache.asterix.transaction.management.service.logging.LogBuffer;
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.control.common.controllers.NCConfig;
-import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.replication.logging.RemoteLogsProcessor;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.asterix.replication.messaging.ReplicationProtocol.ReplicationRequestType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
-import org.apache.hyracks.storage.common.LocalResource;
-import org.apache.hyracks.util.StorageUtil;
-import org.apache.hyracks.util.StorageUtil.StorageUnit;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -101,200 +45,74 @@
 public class ReplicationChannel extends Thread implements IReplicationChannel {
 
     private static final Logger LOGGER = LogManager.getLogger();
-    private static final int LOG_REPLICATION_END_HANKSHAKE_LOG_SIZE = 1;
-    private final ExecutorService replicationThreads;
-    private final String localNodeID;
-    private final ILogManager logManager;
-    private final ReplicaResourcesManager replicaResourcesManager;
     private ServerSocketChannel serverSocketChannel = null;
-    private final IReplicationManager replicationManager;
-    private final ReplicationProperties replicationProperties;
-    private final IAppRuntimeContextProvider appContextProvider;
-    private static final int INTIAL_BUFFER_SIZE = StorageUtil.getIntSizeInBytes(4, StorageUnit.KILOBYTE);
-    private final LinkedBlockingQueue<LSMComponentLSNSyncTask> lsmComponentRemoteLSN2LocalLSNMappingTaskQ;
-    private final LinkedBlockingQueue<LogRecord> pendingNotificationRemoteLogsQ;
-    private final Map<String, LSMComponentProperties> lsmComponentId2PropertiesMap;
-    private final Map<Long, RemoteLogMapping> localLsn2RemoteMapping;
-    private final Map<String, RemoteLogMapping> replicaUniqueLSN2RemoteMapping;
-    private final LSMComponentsSyncService lsmComponentLSNMappingService;
-    private final ReplicationNotifier replicationNotifier;
-    private final Object flushLogslock = new Object();
-    private final IDatasetLifecycleManager dsLifecycleManager;
-    private final PersistentLocalResourceRepository localResourceRep;
-    private final IReplicationStrategy replicationStrategy;
-    private final NCConfig ncConfig;
-    private Set nodeHostedPartitions;
-    private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
     private final INcApplicationContext appCtx;
+    private final RemoteLogsProcessor logsProcessor;
 
-    public ReplicationChannel(String nodeId, ReplicationProperties replicationProperties, ILogManager logManager,
-            IReplicaResourcesManager replicaResoucesManager, IReplicationManager replicationManager,
-            INCServiceContext ncServiceContext, IAppRuntimeContextProvider asterixAppRuntimeContextProvider,
-            IReplicationStrategy replicationStrategy) {
-        this.logManager = logManager;
-        this.localNodeID = nodeId;
-        this.replicaResourcesManager = (ReplicaResourcesManager) replicaResoucesManager;
-        this.replicationManager = replicationManager;
-        this.replicationProperties = replicationProperties;
-        this.appContextProvider = asterixAppRuntimeContextProvider;
-        this.dsLifecycleManager = asterixAppRuntimeContextProvider.getDatasetLifecycleManager();
-        this.localResourceRep = (PersistentLocalResourceRepository) asterixAppRuntimeContextProvider
-                .getLocalResourceRepository();
-        this.replicationStrategy = replicationStrategy;
-        this.ncConfig = ((NodeControllerService) ncServiceContext.getControllerService()).getConfiguration();
-        lsmComponentRemoteLSN2LocalLSNMappingTaskQ = new LinkedBlockingQueue<>();
-        pendingNotificationRemoteLogsQ = new LinkedBlockingQueue<>();
-        lsmComponentId2PropertiesMap = new ConcurrentHashMap<>();
-        replicaUniqueLSN2RemoteMapping = new ConcurrentHashMap<>();
-        localLsn2RemoteMapping = new ConcurrentHashMap<>();
-        lsmComponentLSNMappingService = new LSMComponentsSyncService();
-        replicationNotifier = new ReplicationNotifier();
-        replicationThreads = Executors.newCachedThreadPool(ncServiceContext.getThreadFactory());
-        Map<String, ClusterPartition[]> nodePartitions = asterixAppRuntimeContextProvider.getAppContext()
-                .getMetadataProperties().getNodePartitions();
-        Set<String> nodeReplicationClients = replicationStrategy.getRemotePrimaryReplicas(nodeId).stream()
-                .map(Replica::getId).collect(Collectors.toSet());
-        List<Integer> clientsPartitions = new ArrayList<>();
-        for (String clientId : nodeReplicationClients) {
-            for (ClusterPartition clusterPartition : nodePartitions.get(clientId)) {
-                clientsPartitions.add(clusterPartition.getPartitionId());
-            }
-        }
-        nodeHostedPartitions = new HashSet<>(clientsPartitions.size());
-        nodeHostedPartitions.addAll(clientsPartitions);
-        this.indexCheckpointManagerProvider =
-                ((INcApplicationContext) ncServiceContext.getApplicationContext()).getIndexCheckpointManagerProvider();
-        this.appCtx = (INcApplicationContext) ncServiceContext.getApplicationContext();
+    public ReplicationChannel(INcApplicationContext appCtx) {
+        this.appCtx = appCtx;
+        logsProcessor = new RemoteLogsProcessor(appCtx);
     }
 
     @Override
     public void run() {
-        Thread.currentThread().setName("Replication Channel Thread");
-
-        String nodeIP = replicationProperties.getNodeIpFromId(localNodeID);
-        int dataPort = ncConfig.getReplicationPublicPort();
+        final String nodeId = appCtx.getServiceContext().getNodeId();
+        Thread.currentThread().setName(nodeId + " Replication Channel Thread");
+        final ReplicationProperties replicationProperties = appCtx.getReplicationProperties();
+        final String nodeIP = replicationProperties.getReplicationAddress();
+        final int dataPort = replicationProperties.getReplicationPort();
         try {
             serverSocketChannel = ServerSocketChannel.open();
             serverSocketChannel.configureBlocking(true);
-            InetSocketAddress replicationChannelAddress = new InetSocketAddress(InetAddress.getByName(nodeIP),
-                    dataPort);
+            InetSocketAddress replicationChannelAddress =
+                    new InetSocketAddress(InetAddress.getByName(nodeIP), dataPort);
             serverSocketChannel.socket().bind(replicationChannelAddress);
-            lsmComponentLSNMappingService.start();
-            replicationNotifier.start();
             LOGGER.log(Level.INFO, "opened Replication Channel @ IP Address: " + nodeIP + ":" + dataPort);
-
-            //start accepting replication requests
             while (serverSocketChannel.isOpen()) {
                 SocketChannel socketChannel = serverSocketChannel.accept();
                 socketChannel.configureBlocking(true);
                 //start a new thread to handle the request
-                replicationThreads.execute(new ReplicationThread(socketChannel));
+                appCtx.getThreadExecutor().execute(new ReplicationWorker(socketChannel));
             }
         } catch (AsynchronousCloseException e) {
-            LOGGER.warn("Replication channel closed", e);
+            LOGGER.debug("Replication channel closed", e);
         } catch (IOException e) {
-            throw new IllegalStateException(
-                    "Could not open replication channel @ IP Address: " + nodeIP + ":" + dataPort, e);
+            throw new IllegalStateException("Failed to bind replication channel @ " + nodeIP + ":" + dataPort, e);
         }
     }
 
-    private void updateLSMComponentRemainingFiles(String lsmComponentId) throws IOException {
-        LSMComponentProperties lsmCompProp = lsmComponentId2PropertiesMap.get(lsmComponentId);
-        int remainingFile = lsmCompProp.markFileComplete();
-
-        //clean up when all the LSM component files have been received.
-        if (remainingFile == 0) {
-            if (lsmCompProp.getOpType() == LSMOperationType.FLUSH && lsmCompProp.getReplicaLSN() != null
-                    && replicaUniqueLSN2RemoteMapping.containsKey(lsmCompProp.getNodeUniqueLSN())) {
-                int remainingIndexes = replicaUniqueLSN2RemoteMapping
-                        .get(lsmCompProp.getNodeUniqueLSN()).numOfFlushedIndexes.decrementAndGet();
-                if (remainingIndexes == 0) {
-                    /**
-                     * Note: there is a chance that this will never be removed because some
-                     * index in the dataset was not flushed because it is empty. This could
-                     * be solved by passing only the number of successfully flushed indexes.
-                     */
-                    replicaUniqueLSN2RemoteMapping.remove(lsmCompProp.getNodeUniqueLSN());
-                }
-            }
-
-            //delete mask to indicate that this component is now valid.
-            replicaResourcesManager.markLSMComponentReplicaAsValid(lsmCompProp);
-            lsmComponentId2PropertiesMap.remove(lsmComponentId);
-            LOGGER.log(Level.INFO, "Completed LSMComponent " + lsmComponentId + " Replication.");
-        }
+    public RemoteLogsProcessor getRemoteLogsProcessor() {
+        return logsProcessor;
     }
 
     @Override
     public void close() throws IOException {
-        serverSocketChannel.close();
-        LOGGER.log(Level.INFO, "Replication channel closed.");
+        if (serverSocketChannel != null) {
+            serverSocketChannel.close();
+            LOGGER.info("Replication channel closed.");
+        }
     }
 
-    /**
-     * A replication thread is created per received replication request.
-     */
-    private class ReplicationThread implements IReplicationThread {
+    private class ReplicationWorker implements IReplicationWorker {
         private final SocketChannel socketChannel;
-        private final LogRecord remoteLog;
-        private ByteBuffer inBuffer;
-        private ByteBuffer outBuffer;
+        private final ByteBuffer inBuffer;
+        private final ByteBuffer outBuffer;
 
-        public ReplicationThread(SocketChannel socketChannel) {
+        public ReplicationWorker(SocketChannel socketChannel) {
             this.socketChannel = socketChannel;
-            inBuffer = ByteBuffer.allocate(INTIAL_BUFFER_SIZE);
-            outBuffer = ByteBuffer.allocate(INTIAL_BUFFER_SIZE);
-            remoteLog = new LogRecord();
+            inBuffer = ByteBuffer.allocate(ReplicationProtocol.INITIAL_BUFFER_SIZE);
+            outBuffer = ByteBuffer.allocate(ReplicationProtocol.INITIAL_BUFFER_SIZE);
         }
 
         @Override
         public void run() {
-            Thread.currentThread().setName("Replication Thread");
+            final String oldName = Thread.currentThread().getName();
+            Thread.currentThread().setName("Replication WorkerÙ‹");
             try {
-                ReplicationRequestType replicationFunction = ReplicationProtocol.getRequestType(socketChannel,
-                        inBuffer);
-                while (replicationFunction != ReplicationRequestType.GOODBYE) {
-                    switch (replicationFunction) {
-                        case REPLICATE_LOG:
-                            handleLogReplication();
-                            break;
-                        case LSM_COMPONENT_PROPERTIES:
-                            handleLSMComponentProperties();
-                            break;
-                        case REPLICATE_FILE:
-                            handleReplicateFile();
-                            break;
-                        case DELETE_FILE:
-                            handleDeleteFile();
-                            break;
-                        case REPLICA_EVENT:
-                            handleReplicaEvent();
-                            break;
-                        case GET_REPLICA_MAX_LSN:
-                            handleGetReplicaMaxLSN();
-                            break;
-                        case GET_REPLICA_FILES:
-                            handleGetReplicaFiles();
-                            break;
-                        case FLUSH_INDEX:
-                            handleFlushIndex();
-                            break;
-                        case PARTITION_RESOURCES_REQUEST:
-                            handleGetPartitionResources();
-                            break;
-                        case REPLICATE_RESOURCE_FILE:
-                            handleReplicateResourceFile();
-                            break;
-                        case DELETE_RESOURCE_FILE:
-                            handleDeleteResourceFile();
-                            break;
-                        case CHECKPOINT_PARTITION:
-                            handleCheckpointPartition();
-                            break;
-                        default:
-                            throw new IllegalStateException("Unknown replication request");
-                    }
-                    replicationFunction = ReplicationProtocol.getRequestType(socketChannel, inBuffer);
+                ReplicationRequestType requestType = ReplicationProtocol.getRequestType(socketChannel, inBuffer);
+                while (requestType != ReplicationRequestType.GOODBYE) {
+                    handle(requestType);
+                    requestType = ReplicationProtocol.getRequestType(socketChannel, inBuffer);
                 }
             } catch (Exception e) {
                 LOGGER.warn("Unexpectedly error during replication.", e);
@@ -303,261 +121,10 @@
                     try {
                         socketChannel.close();
                     } catch (IOException e) {
-                        LOGGER.warn("Filed to close replication socket.", e);
+                        LOGGER.warn("Failed to close replication socket.", e);
                     }
                 }
-            }
-        }
-
-        private void handleFlushIndex() throws IOException {
-            inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
-            //read which indexes are requested to be flushed from remote replica
-            ReplicaIndexFlushRequest request = ReplicationProtocol.readReplicaIndexFlushRequest(inBuffer);
-            Set<Long> requestedIndexesToBeFlushed = request.getLaggingRescouresIds();
-
-            /**
-             * check which indexes can be flushed (open indexes) and which cannot be
-             * flushed (closed or have empty memory component).
-             */
-            IDatasetLifecycleManager datasetLifeCycleManager = appContextProvider.getDatasetLifecycleManager();
-            List<IndexInfo> openIndexesInfo = datasetLifeCycleManager.getOpenIndexesInfo();
-            Set<Integer> datasetsToForceFlush = new HashSet<>();
-            for (IndexInfo iInfo : openIndexesInfo) {
-                if (requestedIndexesToBeFlushed.contains(iInfo.getResourceId())) {
-                    AbstractLSMIOOperationCallback ioCallback = (AbstractLSMIOOperationCallback) iInfo.getIndex()
-                            .getIOOperationCallback();
-                    //if an index has a pending flush, then the request to flush it will succeed.
-                    if (ioCallback.hasPendingFlush()) {
-                        //remove index to indicate that it will be flushed
-                        requestedIndexesToBeFlushed.remove(iInfo.getResourceId());
-                    } else if (!((AbstractLSMIndex) iInfo.getIndex()).isCurrentMutableComponentEmpty()) {
-                        /**
-                         * if an index has something to be flushed, then the request to flush it
-                         * will succeed and we need to schedule it to be flushed.
-                         */
-                        datasetsToForceFlush.add(iInfo.getDatasetId());
-                        //remove index to indicate that it will be flushed
-                        requestedIndexesToBeFlushed.remove(iInfo.getResourceId());
-                    }
-                }
-            }
-
-            //schedule flush for datasets requested to be flushed
-            for (int datasetId : datasetsToForceFlush) {
-                datasetLifeCycleManager.flushDataset(datasetId, true);
-            }
-
-            //the remaining indexes in the requested set are those which cannot be flushed.
-            //respond back to the requester that those indexes cannot be flushed
-            ReplicaIndexFlushRequest laggingIndexesResponse = new ReplicaIndexFlushRequest(requestedIndexesToBeFlushed);
-            outBuffer = ReplicationProtocol.writeGetReplicaIndexFlushRequest(outBuffer, laggingIndexesResponse);
-            NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
-        }
-
-        private void handleLSMComponentProperties() throws IOException {
-            inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
-            LSMComponentProperties lsmCompProp = ReplicationProtocol.readLSMPropertiesRequest(inBuffer);
-            //create mask to indicate that this component is not valid yet
-            replicaResourcesManager.createRemoteLSMComponentMask(lsmCompProp);
-            lsmComponentId2PropertiesMap.put(lsmCompProp.getComponentId(), lsmCompProp);
-        }
-
-        private void handleReplicateFile() throws IOException {
-            inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
-            LSMIndexFileProperties afp = ReplicationProtocol.readFileReplicationRequest(inBuffer);
-
-            //get index path
-            String indexPath = replicaResourcesManager.getIndexPath(afp);
-            String replicaFilePath = indexPath + File.separator + afp.getFileName();
-
-            //create file
-            File destFile = new File(replicaFilePath);
-            destFile.createNewFile();
-
-            try (RandomAccessFile fileOutputStream = new RandomAccessFile(destFile, "rw");
-                    FileChannel fileChannel = fileOutputStream.getChannel()) {
-                fileOutputStream.setLength(afp.getFileSize());
-                NetworkingUtil.downloadFile(fileChannel, socketChannel);
-                fileChannel.force(true);
-
-                if (afp.requiresAck()) {
-                    ReplicationProtocol.sendAck(socketChannel);
-                }
-                if (afp.isLSMComponentFile()) {
-                    String componentId = LSMComponentProperties.getLSMComponentID(afp.getFilePath());
-                    final LSMComponentProperties lsmComponentProperties = lsmComponentId2PropertiesMap.get(componentId);
-                    // merge operations do not generate flush logs
-                    if (afp.requiresAck() && lsmComponentProperties.getOpType() == LSMOperationType.FLUSH) {
-                        LSMComponentLSNSyncTask syncTask =
-                                new LSMComponentLSNSyncTask(componentId, destFile.getAbsolutePath());
-                        lsmComponentRemoteLSN2LocalLSNMappingTaskQ.offer(syncTask);
-                    } else {
-                        updateLSMComponentRemainingFiles(componentId);
-                    }
-                } else {
-                    //index metadata file
-                    final ResourceReference indexRef = ResourceReference.of(destFile.getAbsolutePath());
-                    indexCheckpointManagerProvider.get(indexRef).init(logManager.getAppendLSN());
-                }
-            }
-        }
-
-        private void handleGetReplicaMaxLSN() throws IOException {
-            long maxLNS = logManager.getAppendLSN();
-            outBuffer.clear();
-            outBuffer.putLong(maxLNS);
-            outBuffer.flip();
-            NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
-        }
-
-        private void handleGetReplicaFiles() throws IOException {
-            inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
-            ReplicaFilesRequest request = ReplicationProtocol.readReplicaFileRequest(inBuffer);
-
-            LSMIndexFileProperties fileProperties = new LSMIndexFileProperties();
-
-            List<String> filesList;
-            Set<Integer> partitionIds = request.getPartitionIds();
-            Set<String> requesterExistingFiles = request.getExistingFiles();
-            Map<Integer, ClusterPartition> clusterPartitions = appContextProvider.getAppContext()
-                    .getMetadataProperties().getClusterPartitions();
-
-            // Flush replicated datasets to generate the latest LSM components
-            dsLifecycleManager.flushDataset(replicationStrategy);
-            for (Integer partitionId : partitionIds) {
-                ClusterPartition partition = clusterPartitions.get(partitionId);
-                filesList = replicaResourcesManager.getPartitionIndexesFiles(partition.getPartitionId(), false);
-                //start sending files
-                for (String filePath : filesList) {
-                    // Send only files of datasets that are replciated.
-                    DatasetResourceReference indexFileRef = localResourceRep.getLocalResourceReference(filePath);
-                    if (!replicationStrategy.isMatch(indexFileRef.getDatasetId())) {
-                        continue;
-                    }
-                    String relativeFilePath = StoragePathUtil.getIndexFileRelativePath(filePath);
-                    //if the file already exists on the requester, skip it
-                    if (!requesterExistingFiles.contains(relativeFilePath)) {
-                        try (RandomAccessFile fromFile = new RandomAccessFile(filePath, "r");
-                                FileChannel fileChannel = fromFile.getChannel();) {
-                            long fileSize = fileChannel.size();
-                            fileProperties.initialize(filePath, fileSize, partition.getNodeId(), false, false);
-                            outBuffer = ReplicationProtocol.writeFileReplicationRequest(outBuffer, fileProperties,
-                                    ReplicationRequestType.REPLICATE_FILE);
-
-                            //send file info
-                            NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
-
-                            //transfer file
-                            NetworkingUtil.sendFile(fileChannel, socketChannel);
-                        }
-                    }
-                }
-            }
-
-            //send goodbye (end of files)
-            ReplicationProtocol.sendGoodbye(socketChannel);
-        }
-
-        private void handleReplicaEvent() throws IOException {
-            inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
-            ReplicaEvent event = ReplicationProtocol.readReplicaEventRequest(inBuffer);
-            replicationManager.reportReplicaEvent(event);
-        }
-
-        private void handleDeleteFile() throws IOException {
-            inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
-            LSMIndexFileProperties fileProp = ReplicationProtocol.readFileReplicationRequest(inBuffer);
-            replicaResourcesManager.deleteIndexFile(fileProp);
-            if (fileProp.requiresAck()) {
-                ReplicationProtocol.sendAck(socketChannel);
-            }
-        }
-
-        private void handleLogReplication() throws IOException, ACIDException {
-            //set initial buffer size to a log buffer page size
-            inBuffer = ByteBuffer.allocate(logManager.getLogPageSize());
-            while (true) {
-                //read a batch of logs
-                inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
-                //check if it is end of handshake (a single byte log)
-                if (inBuffer.remaining() == LOG_REPLICATION_END_HANKSHAKE_LOG_SIZE) {
-                    break;
-                }
-
-                processLogsBatch(inBuffer);
-            }
-        }
-
-        private void processLogsBatch(ByteBuffer buffer) throws ACIDException {
-            while (buffer.hasRemaining()) {
-                //get rid of log size
-                inBuffer.getInt();
-                //Deserialize log
-                remoteLog.readRemoteLog(inBuffer);
-                remoteLog.setLogSource(LogSource.REMOTE);
-
-                switch (remoteLog.getLogType()) {
-                    case LogType.UPDATE:
-                    case LogType.ENTITY_COMMIT:
-                        logManager.log(remoteLog);
-                        break;
-                    case LogType.JOB_COMMIT:
-                    case LogType.ABORT:
-                        LogRecord jobTerminationLog = new LogRecord();
-                        TransactionUtil.formJobTerminateLogRecord(jobTerminationLog, remoteLog.getTxnId(),
-                                remoteLog.getLogType() == LogType.JOB_COMMIT);
-                        jobTerminationLog.setReplicationThread(this);
-                        jobTerminationLog.setLogSource(LogSource.REMOTE);
-                        logManager.log(jobTerminationLog);
-                        break;
-                    case LogType.FLUSH:
-                        //store mapping information for flush logs to use them in incoming LSM components.
-                        RemoteLogMapping flushLogMap = new RemoteLogMapping();
-                        LogRecord flushLog = new LogRecord();
-                        TransactionUtil.formFlushLogRecord(flushLog, remoteLog.getDatasetId(), null,
-                                remoteLog.getNodeId(), remoteLog.getNumOfFlushedIndexes());
-                        flushLog.setReplicationThread(this);
-                        flushLog.setLogSource(LogSource.REMOTE);
-                        flushLogMap.setRemoteNodeID(remoteLog.getNodeId());
-                        flushLogMap.setRemoteLSN(remoteLog.getLSN());
-                        synchronized (localLsn2RemoteMapping) {
-                            logManager.log(flushLog);
-                            //the log LSN value is updated by logManager.log(.) to a local value
-                            flushLogMap.setLocalLSN(flushLog.getLSN());
-                            flushLogMap.numOfFlushedIndexes.set(flushLog.getNumOfFlushedIndexes());
-                            replicaUniqueLSN2RemoteMapping.put(flushLogMap.getNodeUniqueLSN(), flushLogMap);
-                            localLsn2RemoteMapping.put(flushLog.getLSN(), flushLogMap);
-                        }
-                        synchronized (flushLogslock) {
-                            flushLogslock.notify();
-                        }
-                        break;
-                    default:
-                        LOGGER.error("Unsupported LogType: " + remoteLog.getLogType());
-                }
-            }
-        }
-
-        /**
-         * this method is called sequentially by {@link LogBuffer#notifyReplicationTermination()}
-         * for JOB_COMMIT, JOB_ABORT, and FLUSH log types.
-         */
-        @Override
-        public void notifyLogReplicationRequester(LogRecord logRecord) {
-            switch (logRecord.getLogType()) {
-                case LogType.JOB_COMMIT:
-                case LogType.ABORT:
-                    pendingNotificationRemoteLogsQ.offer(logRecord);
-                    break;
-                case LogType.FLUSH:
-                    final RemoteLogMapping remoteLogMapping;
-                    synchronized (localLsn2RemoteMapping) {
-                        remoteLogMapping = localLsn2RemoteMapping.remove(logRecord.getLSN());
-                    }
-                    checkpointReplicaIndexes(remoteLogMapping, logRecord.getDatasetId());
-                    break;
-                default:
-                    throw new IllegalStateException("Unexpected log type: " + logRecord.getLogType());
+                Thread.currentThread().setName(oldName);
             }
         }
 
@@ -571,115 +138,10 @@
             return outBuffer;
         }
 
-        private void checkpointReplicaIndexes(RemoteLogMapping remoteLogMapping, int datasetId) {
-            try {
-                Predicate<LocalResource> replicaIndexesPredicate = lr -> {
-                    DatasetLocalResource dls = (DatasetLocalResource) lr.getResource();
-                    return dls.getDatasetId() == datasetId && !localResourceRep.getActivePartitions()
-                            .contains(dls.getPartition());
-                };
-                final Map<Long, LocalResource> resources = localResourceRep.getResources(replicaIndexesPredicate);
-                final List<DatasetResourceReference> replicaIndexesRef =
-                        resources.values().stream().map(DatasetResourceReference::of).collect(Collectors.toList());
-                for (DatasetResourceReference replicaIndexRef : replicaIndexesRef) {
-                    final IIndexCheckpointManager indexCheckpointManager =
-                            indexCheckpointManagerProvider.get(replicaIndexRef);
-                    synchronized (indexCheckpointManager) {
-                        indexCheckpointManager
-                                .masterFlush(remoteLogMapping.getRemoteLSN(), remoteLogMapping.getLocalLSN());
-                    }
-                }
-            } catch (Exception e) {
-                LOGGER.error("Failed to checkpoint replica indexes", e);
-            }
-        }
-
-        private void handleGetPartitionResources() throws IOException {
-            final PartitionResourcesListTask task = (PartitionResourcesListTask) ReplicationProtocol
-                    .readMessage(ReplicationRequestType.PARTITION_RESOURCES_REQUEST, socketChannel, inBuffer);
+        private void handle(ReplicationRequestType requestType) throws HyracksDataException {
+            final IReplicaTask task =
+                    (IReplicaTask) ReplicationProtocol.readMessage(requestType, socketChannel, inBuffer);
             task.perform(appCtx, this);
         }
-
-        private void handleReplicateResourceFile() throws HyracksDataException {
-            ReplicateFileTask task = (ReplicateFileTask) ReplicationProtocol
-                    .readMessage(ReplicationRequestType.REPLICATE_RESOURCE_FILE, socketChannel, inBuffer);
-            task.perform(appCtx, this);
-        }
-
-        private void handleDeleteResourceFile() throws HyracksDataException {
-            DeleteFileTask task = (DeleteFileTask) ReplicationProtocol
-                    .readMessage(ReplicationRequestType.DELETE_RESOURCE_FILE, socketChannel, inBuffer);
-            task.perform(appCtx, this);
-        }
-
-        private void handleCheckpointPartition() throws HyracksDataException {
-            CheckpointPartitionIndexesTask task = (CheckpointPartitionIndexesTask) ReplicationProtocol
-                    .readMessage(ReplicationRequestType.CHECKPOINT_PARTITION, socketChannel, inBuffer);
-            task.perform(appCtx, this);
-        }
-    }
-
-    /**
-     * This thread is responsible for sending JOB_COMMIT/ABORT ACKs to replication clients.
-     */
-    private class ReplicationNotifier extends Thread {
-        @Override
-        public void run() {
-            Thread.currentThread().setName("ReplicationNotifier Thread");
-            while (true) {
-                try {
-                    LogRecord logRecord = pendingNotificationRemoteLogsQ.take();
-                    //send ACK to requester
-                    logRecord.getReplicationThread().getChannel().socket().getOutputStream()
-                            .write((localNodeID + ReplicationProtocol.JOB_REPLICATION_ACK + logRecord.getTxnId()
-                                    + System.lineSeparator()).getBytes());
-                } catch (InterruptedException e) {
-                    Thread.currentThread().interrupt();
-                } catch (IOException e) {
-                    LOGGER.warn("Failed to send job replication ACK", e);
-                }
-            }
-        }
-    }
-
-    /**
-     * This thread is responsible for synchronizing the LSN of
-     * the received LSM components to a local LSN.
-     */
-    private class LSMComponentsSyncService extends Thread {
-
-        @Override
-        public void run() {
-            Thread.currentThread().setName("LSMComponentsSyncService Thread");
-
-            while (true) {
-                try {
-                    LSMComponentLSNSyncTask syncTask = lsmComponentRemoteLSN2LocalLSNMappingTaskQ.take();
-                    LSMComponentProperties lsmCompProp = lsmComponentId2PropertiesMap.get(syncTask.getComponentId());
-                    syncLSMComponentFlushLSN(lsmCompProp, syncTask);
-                    updateLSMComponentRemainingFiles(lsmCompProp.getComponentId());
-                } catch (InterruptedException e) {
-                    Thread.currentThread().interrupt();
-                } catch (Exception e) {
-                    LOGGER.error("Unexpected exception during LSN synchronization", e);
-                }
-            }
-        }
-
-        private void syncLSMComponentFlushLSN(LSMComponentProperties lsmCompProp, LSMComponentLSNSyncTask syncTask)
-                throws InterruptedException, IOException {
-            final String componentFilePath = syncTask.getComponentFilePath();
-            final ResourceReference indexRef = ResourceReference.of(componentFilePath);
-            final IIndexCheckpointManager indexCheckpointManager = indexCheckpointManagerProvider.get(indexRef);
-            synchronized (indexCheckpointManager) {
-                long masterLsn = lsmCompProp.getOriginalLSN();
-                // wait until the lsn mapping is flushed to disk
-                while (!indexCheckpointManager.isFlushed(masterLsn)) {
-                    indexCheckpointManager.wait();
-                }
-                indexCheckpointManager
-                        .replicated(AbstractLSMIndexFileManager.getComponentEndTime(indexRef.getName()), masterLsn);
-            }
-        }
     }
 }
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
index 6445345..727a379 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
@@ -18,1342 +18,120 @@
  */
 package org.apache.asterix.replication.management;
 
-import java.io.BufferedReader;
-import java.io.File;
 import java.io.IOException;
-import java.io.InputStreamReader;
 import java.io.OutputStream;
-import java.io.RandomAccessFile;
 import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.nio.channels.AsynchronousCloseException;
-import java.nio.channels.FileChannel;
-import java.nio.channels.SocketChannel;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Optional;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.stream.Collectors;
 
-import org.apache.asterix.common.cluster.ClusterPartition;
+import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.config.ReplicationProperties;
 import org.apache.asterix.common.replication.IPartitionReplica;
-import org.apache.asterix.common.replication.IReplicaResourcesManager;
+import org.apache.asterix.common.replication.IReplicationDestination;
 import org.apache.asterix.common.replication.IReplicationManager;
 import org.apache.asterix.common.replication.IReplicationStrategy;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.replication.Replica.ReplicaState;
-import org.apache.asterix.common.replication.ReplicaEvent;
-import org.apache.asterix.common.replication.ReplicationJob;
 import org.apache.asterix.common.replication.ReplicationStrategyFactory;
-import org.apache.asterix.common.storage.DatasetResourceReference;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.asterix.common.storage.ResourceReference;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
-import org.apache.asterix.common.transactions.ILogManager;
 import org.apache.asterix.common.transactions.ILogRecord;
-import org.apache.asterix.common.transactions.LogType;
-import org.apache.asterix.replication.functions.ReplicaFilesRequest;
-import org.apache.asterix.replication.functions.ReplicaIndexFlushRequest;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
-import org.apache.asterix.replication.functions.ReplicationProtocol.ReplicationRequestType;
-import org.apache.asterix.replication.logging.ReplicationLogBuffer;
-import org.apache.asterix.replication.logging.TxnLogReplicator;
-import org.apache.asterix.replication.storage.LSMComponentProperties;
-import org.apache.asterix.replication.storage.LSMIndexFileProperties;
-import org.apache.asterix.replication.storage.ReplicaResourcesManager;
-import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
-import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.config.IApplicationConfig;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.asterix.replication.api.ReplicationDestination;
 import org.apache.hyracks.api.replication.IReplicationJob;
-import org.apache.hyracks.api.replication.IReplicationJob.ReplicationExecutionType;
-import org.apache.hyracks.api.replication.IReplicationJob.ReplicationJobType;
-import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
-import org.apache.hyracks.control.common.controllers.NCConfig;
-import org.apache.hyracks.control.nc.NodeControllerService;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
-import org.apache.hyracks.util.StorageUtil;
-import org.apache.hyracks.util.StorageUtil.StorageUnit;
-import org.apache.logging.log4j.Level;
+import org.apache.hyracks.util.annotations.ThreadSafe;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-/**
- * This class is used to process replication jobs and maintain remote replicas states
- */
+@ThreadSafe
 public class ReplicationManager implements IReplicationManager {
 
     private static final Logger LOGGER = LogManager.getLogger();
-    private static final int INITIAL_REPLICATION_FACTOR = 1;
-    private static final int MAX_JOB_COMMIT_ACK_WAIT = 10000;
-    private final String nodeId;
-    private ExecutorService replicationListenerThreads;
-    private final Map<Long, Set<String>> txnCommitAcks;
-    private final Map<Long, ILogRecord> replicationTxnsPendingAcks;
-    private ByteBuffer dataBuffer;
-    private final LinkedBlockingQueue<IReplicationJob> replicationJobsQ;
-    private final LinkedBlockingQueue<ReplicaEvent> replicaEventsQ;
-
-    private int replicationFactor = 1;
-    private final ReplicaResourcesManager replicaResourcesManager;
-    private final ILogManager logManager;
-    private final IAppRuntimeContextProvider asterixAppRuntimeContextProvider;
+    private final Map<InetSocketAddress, ReplicationDestination> dests = new HashMap<>();
     private final ReplicationProperties replicationProperties;
-    private final Map<String, Replica> replicas;
-    private final Map<String, Set<Integer>> replica2PartitionsMap;
+    private final IReplicationStrategy strategy;
+    private final INcApplicationContext appCtx;
+    private final LogReplicationManager logReplicationManager;
+    private final IndexReplicationManager lsnIndexReplicationManager;
 
-    private final AtomicBoolean replicationSuspended;
-    private AtomicBoolean terminateJobsReplication;
-    private AtomicBoolean jobsReplicationSuspended;
-    private static final int INITIAL_BUFFER_SIZE = StorageUtil.getIntSizeInBytes(4, StorageUnit.KILOBYTE);
-    private final Set<String> shuttingDownReplicaIds;
-    //replication threads
-    private ReplicationJobsProccessor replicationJobsProcessor;
-    private final ReplicasEventsMonitor replicationMonitor;
-    //dummy job used to stop ReplicationJobsProccessor thread.
-    private static final IReplicationJob REPLICATION_JOB_POISON_PILL = new ReplicationJob(ReplicationJobType.METADATA,
-            ReplicationOperation.REPLICATE, ReplicationExecutionType.ASYNC, null);
-    //used to identify the correct IP address when the node has multiple network interfaces
-    private String hostIPAddressFirstOctet = null;
-
-    private LinkedBlockingQueue<ReplicationLogBuffer> emptyLogBuffersQ;
-    private LinkedBlockingQueue<ReplicationLogBuffer> pendingFlushLogBuffersQ;
-    protected ReplicationLogBuffer currentTxnLogBuffer;
-    private TxnLogReplicator txnlogReplicator;
-    private Future<? extends Object> txnLogReplicatorTask;
-    private SocketChannel[] logsRepSockets;
-    private final ByteBuffer txnLogsBatchSizeBuffer = ByteBuffer.allocate(Integer.BYTES);
-    private IReplicationStrategy replicationStrategy;
-    private final PersistentLocalResourceRepository localResourceRepo;
-    private NCConfig ncConfig;
-    private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
-
-    //TODO this class needs to be refactored by moving its private classes to separate files
-    //and possibly using MessageBroker to send/receive remote replicas events.
-    public ReplicationManager(String nodeId, ReplicationProperties replicationProperties,
-            IReplicaResourcesManager remoteResoucesManager, ILogManager logManager,
-            IAppRuntimeContextProvider asterixAppRuntimeContextProvider, INCServiceContext ncServiceContext) {
-        this.nodeId = nodeId;
-        this.ncConfig = ((NodeControllerService) ncServiceContext.getControllerService()).getConfiguration();
+    public ReplicationManager(INcApplicationContext appCtx, ReplicationProperties replicationProperties) {
         this.replicationProperties = replicationProperties;
-        try {
-            replicationStrategy = ReplicationStrategyFactory.create(replicationProperties.getReplicationStrategy(),
-                    replicationProperties, ncConfig.getConfigManager());
-        } catch (HyracksDataException e) {
-            LOGGER.log(Level.WARN, "Couldn't initialize replication strategy", e);
-        }
-        this.replicaResourcesManager = (ReplicaResourcesManager) remoteResoucesManager;
-        this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
-        this.logManager = logManager;
-        localResourceRepo =
-                (PersistentLocalResourceRepository) asterixAppRuntimeContextProvider.getLocalResourceRepository();
-        this.hostIPAddressFirstOctet = ncConfig.getPublicAddress().substring(0, 3);
-        this.indexCheckpointManagerProvider =
-                asterixAppRuntimeContextProvider.getAppContext().getIndexCheckpointManagerProvider();
-        replicas = new HashMap<>();
-        replicationJobsQ = new LinkedBlockingQueue<>();
-        replicaEventsQ = new LinkedBlockingQueue<>();
-        terminateJobsReplication = new AtomicBoolean(false);
-        jobsReplicationSuspended = new AtomicBoolean(true);
-        replicationSuspended = new AtomicBoolean(true);
-        txnCommitAcks = new ConcurrentHashMap<>();
-        replicationTxnsPendingAcks = new ConcurrentHashMap<>();
-        shuttingDownReplicaIds = new HashSet<>();
-        dataBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
-        replicationMonitor = new ReplicasEventsMonitor();
-        //add list of replicas from configurations (To be read from another source e.g. Zookeeper)
-        Set<Replica> replicaNodes = replicationStrategy.getRemoteReplicas(nodeId);
-
-        //Used as async listeners from replicas
-        replicationListenerThreads = Executors.newCachedThreadPool();
-        replicationJobsProcessor = new ReplicationJobsProccessor();
-
-        Map<String, ClusterPartition[]> nodePartitions =
-                asterixAppRuntimeContextProvider.getAppContext().getMetadataProperties().getNodePartitions();
-        replica2PartitionsMap = new HashMap<>(replicaNodes.size());
-        for (Replica replica : replicaNodes) {
-            replicas.put(replica.getId(), replica);
-            //for each remote replica, get the list of replication clients
-            Set<Replica> nodeReplicationClients = replicationStrategy.getRemotePrimaryReplicas(replica.getId());
-            //get the partitions of each client
-            List<Integer> clientPartitions = new ArrayList<>();
-            for (Replica client : nodeReplicationClients) {
-                for (ClusterPartition clusterPartition : nodePartitions.get(client.getId())) {
-                    clientPartitions.add(clusterPartition.getPartitionId());
-                }
-            }
-            Set<Integer> clientPartitonsSet = new HashSet<>(clientPartitions.size());
-            clientPartitonsSet.addAll(clientPartitions);
-            replica2PartitionsMap.put(replica.getId(), clientPartitonsSet);
-        }
-        int numLogBuffers = replicationProperties.getLogBufferNumOfPages();
-        emptyLogBuffersQ = new LinkedBlockingQueue<>(numLogBuffers);
-        pendingFlushLogBuffersQ = new LinkedBlockingQueue<>(numLogBuffers);
-
-        int logBufferSize = replicationProperties.getLogBufferPageSize();
-        for (int i = 0; i < numLogBuffers; i++) {
-            emptyLogBuffersQ
-                    .offer(new ReplicationLogBuffer(this, logBufferSize, replicationProperties.getLogBatchSize()));
-        }
-    }
-
-    @Override
-    public void submitJob(IReplicationJob job) throws IOException {
-        if (job.getExecutionType() == ReplicationExecutionType.ASYNC) {
-            replicationJobsQ.offer(job);
-        } else {
-            //wait until replication is resumed
-            while (replicationSuspended.get()) {
-                synchronized (replicationSuspended) {
-                    try {
-                        replicationSuspended.wait();
-                    } catch (InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                    }
-                }
-            }
-            processJob(job, null, null);
-        }
-    }
-
-    @Override
-    public void replicateLog(ILogRecord logRecord) throws InterruptedException {
-        if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
-            //if replication is suspended, wait until it is resumed.
-            while (replicationSuspended.get()) {
-                synchronized (replicationSuspended) {
-                    replicationSuspended.wait();
-                }
-            }
-            Set<String> replicaIds = Collections.synchronizedSet(new HashSet<String>());
-            replicaIds.add(nodeId);
-            txnCommitAcks.put(logRecord.getTxnId(), replicaIds);
-        }
-
-        appendToLogBuffer(logRecord);
-    }
-
-    protected void getAndInitNewLargePage(int pageSize) {
-        // for now, alloc a new buffer for each large page
-        // TODO: consider pooling large pages
-        currentTxnLogBuffer = new ReplicationLogBuffer(this, pageSize, replicationProperties.getLogBufferPageSize());
-        pendingFlushLogBuffersQ.offer(currentTxnLogBuffer);
-    }
-
-    protected void getAndInitNewPage() throws InterruptedException {
-        currentTxnLogBuffer = null;
-        while (currentTxnLogBuffer == null) {
-            currentTxnLogBuffer = emptyLogBuffersQ.take();
-        }
-        currentTxnLogBuffer.reset();
-        pendingFlushLogBuffersQ.offer(currentTxnLogBuffer);
-    }
-
-    private synchronized void appendToLogBuffer(ILogRecord logRecord) throws InterruptedException {
-        if (!currentTxnLogBuffer.hasSpace(logRecord)) {
-            currentTxnLogBuffer.isFull(true);
-            if (logRecord.getLogSize() > getLogPageSize()) {
-                getAndInitNewLargePage(logRecord.getLogSize());
-            } else {
-                getAndInitNewPage();
-            }
-        }
-        currentTxnLogBuffer.append(logRecord);
-    }
-
-    /**
-     * Processes the replication job based on its specifications
-     *
-     * @param job
-     *            The replication job
-     * @param replicasSockets
-     *            The remote replicas sockets to send the request to.
-     * @param requestBuffer
-     *            The buffer to use to send the request.
-     * @throws IOException
-     */
-    private void processJob(IReplicationJob job, Map<String, SocketChannel> replicasSockets, ByteBuffer requestBuffer)
-            throws IOException {
-        try {
-
-            //all of the job's files belong to a single storage partition.
-            //get any of them to determine the partition from the file path.
-            String jobFile = job.getJobFiles().iterator().next();
-            DatasetResourceReference indexFileRef = localResourceRepo.getLocalResourceReference(jobFile);
-            if (!replicationStrategy.isMatch(indexFileRef.getDatasetId())) {
-                return;
-            }
-            int jobPartitionId = indexFileRef.getPartitionId();
-
-            ByteBuffer responseBuffer = null;
-            LSMIndexFileProperties asterixFileProperties = new LSMIndexFileProperties();
-            if (requestBuffer == null) {
-                requestBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
-            }
-
-            boolean isLSMComponentFile = job.getJobType() == ReplicationJobType.LSM_COMPONENT;
-            try {
-                //if there isn't already a connection, establish a new one
-                if (replicasSockets == null) {
-                    replicasSockets = getActiveRemoteReplicasSockets();
-                }
-
-                int remainingFiles = job.getJobFiles().size();
-                if (job.getOperation() == ReplicationOperation.REPLICATE) {
-                    //if the replication job is an LSM_COMPONENT, its properties are sent first, then its files.
-                    ILSMIndexReplicationJob LSMComponentJob = null;
-                    if (job.getJobType() == ReplicationJobType.LSM_COMPONENT) {
-                        //send LSMComponent properties
-                        LSMComponentJob = (ILSMIndexReplicationJob) job;
-                        LSMComponentProperties lsmCompProp = new LSMComponentProperties(LSMComponentJob, nodeId);
-                        requestBuffer = ReplicationProtocol.writeLSMComponentPropertiesRequest(lsmCompProp, //NOSONAR
-                                requestBuffer);
-                        sendRequest(replicasSockets, requestBuffer);
-                    }
-
-                    for (String filePath : job.getJobFiles()) {
-                        remainingFiles--;
-                        Path path = Paths.get(filePath);
-                        if (Files.notExists(path)) {
-                            LOGGER.log(Level.ERROR, "File deleted before replication: " + filePath);
-                            continue;
-                        }
-
-                        LOGGER.log(Level.INFO, "Replicating file: " + filePath);
-                        //open file for reading
-                        try (RandomAccessFile fromFile = new RandomAccessFile(filePath, "r");
-                                FileChannel fileChannel = fromFile.getChannel();) {
-
-                            long fileSize = fileChannel.size();
-                            asterixFileProperties.initialize(filePath, fileSize, nodeId, isLSMComponentFile,
-                                    remainingFiles == 0);
-                            requestBuffer = ReplicationProtocol.writeFileReplicationRequest(requestBuffer,
-                                    asterixFileProperties, ReplicationRequestType.REPLICATE_FILE);
-                            Iterator<Map.Entry<String, SocketChannel>> iterator = replicasSockets.entrySet().iterator();
-                            while (iterator.hasNext()) {
-                                Map.Entry<String, SocketChannel> entry = iterator.next();
-                                //if the remote replica is not interested in this partition, skip it.
-                                if (!replica2PartitionsMap.get(entry.getKey()).contains(jobPartitionId)) {
-                                    continue;
-                                }
-                                SocketChannel socketChannel = entry.getValue();
-                                //transfer request header & file
-                                try {
-                                    NetworkingUtil.transferBufferToChannel(socketChannel, requestBuffer);
-                                    NetworkingUtil.sendFile(fileChannel, socketChannel);
-                                    if (asterixFileProperties.requiresAck()) {
-                                        ReplicationRequestType responseType =
-                                                waitForResponse(socketChannel, responseBuffer);
-                                        if (responseType != ReplicationRequestType.ACK) {
-                                            throw new IOException(
-                                                    "Could not receive ACK from replica " + entry.getKey());
-                                        }
-                                    }
-                                } catch (IOException e) {
-                                    handleReplicationFailure(socketChannel, e);
-                                    iterator.remove();
-                                } finally {
-                                    requestBuffer.position(0);
-                                }
-                            }
-                        }
-                    }
-                } else if (job.getOperation() == ReplicationOperation.DELETE) {
-                    for (String filePath : job.getJobFiles()) {
-                        remainingFiles--;
-                        asterixFileProperties.initialize(filePath, -1, nodeId, isLSMComponentFile, remainingFiles == 0);
-                        ReplicationProtocol.writeFileReplicationRequest(requestBuffer, asterixFileProperties,
-                                ReplicationRequestType.DELETE_FILE);
-
-                        Iterator<Map.Entry<String, SocketChannel>> iterator = replicasSockets.entrySet().iterator();
-                        while (iterator.hasNext()) {
-                            Map.Entry<String, SocketChannel> entry = iterator.next();
-                            //if the remote replica is not interested in this partition, skip it.
-                            if (!replica2PartitionsMap.get(entry.getKey()).contains(jobPartitionId)) {
-                                continue;
-                            }
-                            SocketChannel socketChannel = entry.getValue();
-                            try {
-                                sendRequest(replicasSockets, requestBuffer);
-                                if (asterixFileProperties.requiresAck()) {
-                                    waitForResponse(socketChannel, responseBuffer);
-                                }
-                            } catch (IOException e) {
-                                handleReplicationFailure(socketChannel, e);
-                                iterator.remove();
-                            } finally {
-                                requestBuffer.position(0);
-                            }
-                        }
-                    }
-                }
-            } finally {
-                //if sync, close sockets with replicas since they wont be reused
-                if (job.getExecutionType() == ReplicationExecutionType.SYNC) {
-                    closeReplicaSockets(replicasSockets);
-                }
-            }
-        } finally {
-            exitReplicatedLSMComponent(job);
-        }
-    }
-
-    private static void exitReplicatedLSMComponent(IReplicationJob job) throws HyracksDataException {
-        if (job.getOperation() == ReplicationOperation.REPLICATE && job instanceof ILSMIndexReplicationJob) {
-            //exit the replicated LSM components
-            ILSMIndexReplicationJob aJob = (ILSMIndexReplicationJob) job;
-            aJob.endReplication();
-        }
-    }
-
-    /**
-     * Waits and reads a response from a remote replica
-     *
-     * @param socketChannel
-     *            The socket to read the response from
-     * @param responseBuffer
-     *            The response buffer to read the response to.
-     * @return The response type.
-     * @throws IOException
-     */
-    private static ReplicationRequestType waitForResponse(SocketChannel socketChannel, ByteBuffer responseBuffer)
-            throws IOException {
-        if (responseBuffer == null) {
-            responseBuffer = ByteBuffer.allocate(ReplicationProtocol.REPLICATION_REQUEST_TYPE_SIZE);
-        } else {
-            responseBuffer.clear();
-        }
-
-        //read response from remote replicas
-        ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(socketChannel, responseBuffer);
-        return responseFunction;
-    }
-
-    @Override
-    public boolean isReplicationEnabled() {
-        return replicationStrategy.isParticipant(nodeId);
-    }
-
-    @Override
-    public synchronized void updateReplicaInfo(Replica replicaNode) {
-        Replica replica = replicas.get(replicaNode.getId());
-        //should not update the info of an active replica
-        if (replica.getState() == ReplicaState.ACTIVE) {
-            return;
-        }
-        replica.setClusterIp(replicaNode.getClusterIp());
-    }
-
-    /**
-     * Suspends processing replication jobs/logs.
-     *
-     * @param force
-     *            a flag indicates if replication should be suspended right away or when the pending jobs are completed.
-     */
-    private void suspendReplication(boolean force) {
-        //suspend replication jobs processing
-        if (replicationJobsProcessor != null && replicationJobsProcessor.isAlive()) {
-            if (force) {
-                terminateJobsReplication.set(true);
-            }
-            replicationJobsQ.offer(REPLICATION_JOB_POISON_PILL);
-
-            //wait until the jobs are suspended
-            synchronized (jobsReplicationSuspended) {
-                while (!jobsReplicationSuspended.get()) {
-                    try {
-                        jobsReplicationSuspended.wait();
-                    } catch (InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                    }
-                }
-            }
-        }
-
-        //suspend logs replication
-        if (txnlogReplicator != null) {
-            endTxnLogReplicationHandshake();
-        }
-    }
-
-    /**
-     * Opens a new connection with Active remote replicas and starts a listen thread per connection.
-     */
-    private void establishTxnLogReplicationHandshake() {
-        Map<String, SocketChannel> activeRemoteReplicasSockets = getActiveRemoteReplicasSockets();
-        logsRepSockets = new SocketChannel[activeRemoteReplicasSockets.size()];
-        int i = 0;
-        //start a listener thread per connection
-        for (Entry<String, SocketChannel> entry : activeRemoteReplicasSockets.entrySet()) {
-            logsRepSockets[i] = entry.getValue();
-            replicationListenerThreads
-                    .execute(new TxnLogsReplicationResponseListener(entry.getKey(), entry.getValue()));
-            i++;
-        }
-
-        /*
-         * establish log replication handshake
-         */
-        ByteBuffer handshakeBuffer = ByteBuffer.allocate(ReplicationProtocol.REPLICATION_REQUEST_TYPE_SIZE)
-                .putInt(ReplicationProtocol.ReplicationRequestType.REPLICATE_LOG.ordinal());
-        handshakeBuffer.flip();
-        //send handshake request
-        for (SocketChannel replicaSocket : logsRepSockets) {
-            try {
-                NetworkingUtil.transferBufferToChannel(replicaSocket, handshakeBuffer);
-            } catch (IOException e) {
-                handleReplicationFailure(replicaSocket, e);
-            } finally {
-                handshakeBuffer.position(0);
-            }
-        }
-    }
-
-    private void handleReplicationFailure(SocketChannel socketChannel, Throwable t) {
-        if (LOGGER.isWarnEnabled()) {
-            LOGGER.log(Level.WARN, "Could not complete replication request.", t);
-        }
-        if (socketChannel.isOpen()) {
-            try {
-                socketChannel.close();
-            } catch (IOException e) {
-                LOGGER.log(Level.WARN, "Could not close socket.", e);
-            }
-        }
-        reportFailedReplica(getReplicaIdBySocket(socketChannel));
-    }
-
-    /**
-     * Stops TxnLogReplicator and closes the sockets used to replicate logs.
-     */
-    private void endTxnLogReplicationHandshake() {
-        LOGGER.info("Terminating TxnLogReplicator thread ...");
-        txnlogReplicator.terminate();
-        try {
-            txnLogReplicatorTask.get();
-        } catch (ExecutionException | InterruptedException e) {
-            LOGGER.error("TxnLogReplicator thread terminated abnormally", e);
-        }
-        LOGGER.info("TxnLogReplicator thread was terminated.");
-
-        /*
-         * End log replication handshake (by sending a dummy log with a single byte)
-         */
-        ByteBuffer endLogRepHandshake = ByteBuffer.allocate(Integer.SIZE + 1).putInt(1).put((byte) 0);
-        endLogRepHandshake.flip();
-        for (SocketChannel replicaSocket : logsRepSockets) {
-            try {
-                NetworkingUtil.transferBufferToChannel(replicaSocket, endLogRepHandshake);
-            } catch (IOException e) {
-                handleReplicationFailure(replicaSocket, e);
-            } finally {
-                endLogRepHandshake.position(0);
-            }
-        }
-
-        //wait for any ACK to arrive before closing sockets.
-        if (logsRepSockets != null) {
-            synchronized (txnCommitAcks) {
-                try {
-                    long waitStartTime = System.currentTimeMillis();
-                    while (!txnCommitAcks.isEmpty()) {
-                        txnCommitAcks.wait(1000);
-                        long waitDuration = System.currentTimeMillis() - waitStartTime;
-                        if (waitDuration > MAX_JOB_COMMIT_ACK_WAIT) {
-                            LOGGER.log(Level.ERROR,
-                                    "Timeout before receving all job ACKs from replicas. Pending txns ("
-                                            + txnCommitAcks.keySet().toString() + ")");
-                            break;
-                        }
-                    }
-                } catch (InterruptedException e) {
-                    LOGGER.error("Interrupted while waiting for jobs ACK", e);
-                    Thread.currentThread().interrupt();
-                }
-            }
-        }
-
-        /*
-         * Close log replication sockets
-         */
-        ByteBuffer goodbyeBuffer = ReplicationProtocol.getGoodbyeBuffer();
-        for (SocketChannel replicaSocket : logsRepSockets) {
-            try {
-                //send goodbye to remote replica
-                NetworkingUtil.transferBufferToChannel(replicaSocket, goodbyeBuffer);
-                replicaSocket.close();
-            } catch (IOException e) {
-                handleReplicationFailure(replicaSocket, e);
-            } finally {
-                goodbyeBuffer.position(0);
-            }
-        }
-        logsRepSockets = null;
-    }
-
-    /**
-     * Sends a shutdown event to remote replicas notifying them
-     * no more logs/files will be sent from this local replica.
-     *
-     * @throws IOException
-     */
-    private void sendShutdownNotifiction() throws IOException {
-        Replica replica = new Replica(nodeId, NetworkingUtil.getHostAddress(hostIPAddressFirstOctet),
-                ncConfig.getReplicationPublicPort());
-        ReplicaEvent event = new ReplicaEvent(replica, ClusterEventType.NODE_SHUTTING_DOWN);
-        ByteBuffer buffer = ReplicationProtocol.writeReplicaEventRequest(event);
-        Map<String, SocketChannel> replicaSockets = getActiveRemoteReplicasSockets();
-        sendRequest(replicaSockets, buffer);
-        closeReplicaSockets(replicaSockets);
-    }
-
-    /**
-     * Sends a request to remote replicas
-     *
-     * @param replicaSockets
-     *            The sockets to send the request to.
-     * @param requestBuffer
-     *            The buffer that contains the request.
-     */
-    private void sendRequest(Map<String, SocketChannel> replicaSockets, ByteBuffer requestBuffer) {
-        Iterator<Map.Entry<String, SocketChannel>> iterator = replicaSockets.entrySet().iterator();
-        while (iterator.hasNext()) {
-            Entry<String, SocketChannel> replicaSocket = iterator.next();
-            SocketChannel clientSocket = replicaSocket.getValue();
-            try {
-                NetworkingUtil.transferBufferToChannel(clientSocket, requestBuffer);
-            } catch (IOException e) {
-                handleReplicationFailure(clientSocket, e);
-                iterator.remove();
-            } finally {
-                requestBuffer.position(0);
-            }
-        }
-    }
-
-    /**
-     * Closes the passed replication sockets by sending GOODBYE request to remote replicas.
-     *
-     * @param replicaSockets
-     */
-    private void closeReplicaSockets(Map<String, SocketChannel> replicaSockets) {
-        //send goodbye
-        ByteBuffer goodbyeBuffer = ReplicationProtocol.getGoodbyeBuffer();
-        sendRequest(replicaSockets, goodbyeBuffer);
-
-        Iterator<Map.Entry<String, SocketChannel>> iterator = replicaSockets.entrySet().iterator();
-        while (iterator.hasNext()) {
-            Entry<String, SocketChannel> replicaSocket = iterator.next();
-            SocketChannel clientSocket = replicaSocket.getValue();
-            if (clientSocket.isOpen()) {
-                try {
-                    clientSocket.close();
-                } catch (IOException e) {
-                    handleReplicationFailure(clientSocket, e);
-                }
-            }
-        }
-    }
-
-    @Override
-    public void initializeReplicasState() {
-        for (Replica replica : replicas.values()) {
-            checkReplicaState(replica.getId(), false, false);
-        }
-    }
-
-    /**
-     * Checks the state of a remote replica by trying to ping it.
-     *
-     * @param replicaId
-     *            The replica to check the state for.
-     * @param async
-     *            a flag indicating whether to wait for the result or not.
-     * @param suspendReplication
-     *            a flag indicating whether to suspend replication on replica state change or not.
-     */
-    private void checkReplicaState(String replicaId, boolean async, boolean suspendReplication) {
-        Replica replica = replicas.get(replicaId);
-
-        ReplicaStateChecker connector = new ReplicaStateChecker(replica, replicationProperties.getReplicationTimeOut(),
-                this, suspendReplication);
-        Future<? extends Object> ft = asterixAppRuntimeContextProvider.getThreadExecutor().submit(connector);
-
-        if (!async) {
-            //wait until task is done
-            while (!ft.isDone()) {
-                try {
-                    Thread.sleep(1000);
-                } catch (InterruptedException e) {
-                    Thread.currentThread().interrupt();
-                }
-            }
-        }
-    }
-
-    /**
-     * Updates the state of a remote replica.
-     *
-     * @param replicaId
-     *            The replica id to update.
-     * @param newState
-     *            The new state of the replica.
-     * @param suspendReplication
-     *            a flag indicating whether to suspend replication on state change or not.
-     * @throws InterruptedException
-     */
-    public synchronized void updateReplicaState(String replicaId, ReplicaState newState, boolean suspendReplication)
-            throws InterruptedException {
-        Replica replica = replicas.get(replicaId);
-
-        if (replica.getState() == newState) {
-            return;
-        }
-
-        if (suspendReplication) {
-            //prevent new jobs/logs from coming in
-            replicationSuspended.set(true);
-
-            if (newState == ReplicaState.DEAD) {
-                //assume the dead replica ACK has been received for all pending jobs
-                synchronized (txnCommitAcks) {
-                    for (Long txnId : txnCommitAcks.keySet()) {
-                        addAckToJob(txnId, replicaId);
-                    }
-                }
-            }
-
-            //force replication threads to stop in order to change the replication factor
-            suspendReplication(true);
-        }
-
-        replica.setState(newState);
-
-        if (newState == ReplicaState.ACTIVE) {
-            replicationFactor++;
-        } else if (newState == ReplicaState.DEAD && replicationFactor > INITIAL_REPLICATION_FACTOR) {
-            replicationFactor--;
-        }
-
-        if (LOGGER.isWarnEnabled()) {
-            LOGGER.warn("Replica " + replicaId + " state changed to: " + newState.name()
-                    + ". Replication factor changed to: " + replicationFactor);
-        }
-
-        if (suspendReplication) {
-            startReplicationThreads();
-        }
-    }
-
-    /**
-     * When an ACK for a JOB_COMMIT is received, it is added to the corresponding job.
-     *
-     * @param txnId
-     * @param replicaId
-     *            The remote replica id the ACK received from.
-     */
-    private void addAckToJob(long txnId, String replicaId) {
-        synchronized (txnCommitAcks) {
-            //add ACK to the job
-            if (txnCommitAcks.containsKey(txnId)) {
-                Set<String> replicaIds = txnCommitAcks.get(txnId);
-                replicaIds.add(replicaId);
-            } else {
-                if (LOGGER.isWarnEnabled()) {
-                    LOGGER.warn("Invalid job replication ACK received for txnId(" + txnId + ")");
-                }
-                return;
-            }
-
-            //if got ACKs from all remote replicas, notify pending jobs if any
-
-            if (txnCommitAcks.get(txnId).size() == replicationFactor && replicationTxnsPendingAcks.containsKey(txnId)) {
-                ILogRecord pendingLog = replicationTxnsPendingAcks.get(txnId);
-                synchronized (pendingLog) {
-                    pendingLog.notifyAll();
-                }
-            }
-        }
-    }
-
-    @Override
-    public boolean hasBeenReplicated(ILogRecord logRecord) {
-        long txnId = logRecord.getTxnId();
-        if (txnCommitAcks.containsKey(txnId)) {
-            synchronized (txnCommitAcks) {
-                //check if all ACKs have been received
-                if (txnCommitAcks.get(txnId).size() == replicationFactor) {
-                    txnCommitAcks.remove(txnId);
-
-                    //remove from pending jobs if exists
-                    replicationTxnsPendingAcks.remove(txnId);
-
-                    //notify any threads waiting for all jobs to finish
-                    if (txnCommitAcks.size() == 0) {
-                        txnCommitAcks.notifyAll();
-                    }
-                    return true;
-                } else {
-                    replicationTxnsPendingAcks.putIfAbsent(txnId, logRecord);
-                    return false;
-                }
-            }
-        }
-        //presume replicated
-        return true;
-    }
-
-    private Map<String, SocketChannel> getActiveRemoteReplicasSockets() {
-        Map<String, SocketChannel> replicaNodesSockets = new HashMap<>();
-        for (Replica replica : replicas.values()) {
-            if (replica.getState() == ReplicaState.ACTIVE) {
-                try {
-                    SocketChannel sc = getReplicaSocket(replica.getId());
-                    replicaNodesSockets.put(replica.getId(), sc);
-                } catch (IOException e) {
-                    if (LOGGER.isWarnEnabled()) {
-                        LOGGER.log(Level.WARN, "Could not get replica socket", e);
-                    }
-                    reportFailedReplica(replica.getId());
-                }
-            }
-        }
-        return replicaNodesSockets;
-    }
-
-    /**
-     * Establishes a connection with a remote replica.
-     *
-     * @param replicaId
-     *            The replica to connect to.
-     * @return The socket of the remote replica
-     * @throws IOException
-     */
-    private SocketChannel getReplicaSocket(String replicaId) throws IOException {
-        SocketChannel sc = SocketChannel.open();
-        sc.configureBlocking(true);
-        IApplicationConfig config = ncConfig.getConfigManager().getNodeEffectiveConfig(replicaId);
-        sc.connect(new InetSocketAddress(config.getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS),
-                config.getInt(NCConfig.Option.REPLICATION_LISTEN_PORT)));
-        return sc;
-    }
-
-    @Override
-    public Set<String> getDeadReplicasIds() {
-        Set<String> replicasIds = new HashSet<>();
-        for (Replica replica : replicas.values()) {
-            if (replica.getState() == ReplicaState.DEAD) {
-                replicasIds.add(replica.getId());
-            }
-        }
-        return replicasIds;
-    }
-
-    @Override
-    public Set<String> getActiveReplicasIds() {
-        Set<String> replicasIds = new HashSet<>();
-        for (Replica replica : replicas.values()) {
-            if (replica.getState() == ReplicaState.ACTIVE) {
-                replicasIds.add(replica.getId());
-            }
-        }
-        return replicasIds;
-    }
-
-    @Override
-    public int getActiveReplicasCount() {
-        return getActiveReplicasIds().size();
-    }
-
-    @Override
-    public void start() {
-        //do nothing
-    }
-
-    @Override
-    public void dumpState(OutputStream os) throws IOException {
-        //do nothing
-    }
-
-    /**
-     * Called during NC shutdown to notify remote replicas about the shutdown
-     * and wait for remote replicas shutdown notification then closes the local
-     * replication channel.
-     */
-    @Override
-    public void stop(boolean dumpState, OutputStream ouputStream) throws IOException {
-        //stop replication thread afters all jobs/logs have been processed
-        suspendReplication(false);
-
-        /*
-         * If this node has any remote replicas, it needs to inform them
-         * that it is shutting down.
-         */
-        if (!replicationStrategy.getRemoteReplicas(nodeId).isEmpty()) {
-            //send shutdown event to remote replicas
-            sendShutdownNotifiction();
-        }
-
-        /*
-         * If this node has any remote primary replicas, then it needs to wait
-         * until all of them send the shutdown notification.
-         */
-        // find active remote primary replicas
-        Set<String> activeRemotePrimaryReplicas = replicationStrategy.getRemotePrimaryReplicas(nodeId).stream()
-                .map(Replica::getId).filter(getActiveReplicasIds()::contains).collect(Collectors.toSet());
-
-        if (!activeRemotePrimaryReplicas.isEmpty()) {
-            //wait until all shutdown events come from all remote primary replicas
-            synchronized (shuttingDownReplicaIds) {
-                while (!shuttingDownReplicaIds.containsAll(activeRemotePrimaryReplicas)) {
-                    try {
-                        shuttingDownReplicaIds.wait();
-                    } catch (InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                    }
-                }
-            }
-        }
-
-        LOGGER.log(Level.INFO, "Got shutdown notification from all remote replicas");
-        //close replication channel
-        asterixAppRuntimeContextProvider.getAppContext().getReplicationChannel().close();
-
-        LOGGER.log(Level.INFO, "Replication manager stopped.");
-    }
-
-    @Override
-    public void reportReplicaEvent(ReplicaEvent event) {
-        replicaEventsQ.offer(event);
-    }
-
-    /**
-     * Suspends replications and sends a remote replica failure event to ReplicasEventsMonitor.
-     *
-     * @param replicaId
-     *            the failed replica id.
-     */
-    public void reportFailedReplica(String replicaId) {
-        Replica replica = replicas.get(replicaId);
-        if (replica == null) {
-            return;
-        }
-        if (replica.getState() == ReplicaState.DEAD) {
-            return;
-        }
-
-        //need to stop processing any new logs or jobs
-        terminateJobsReplication.set(true);
-
-        ReplicaEvent event = new ReplicaEvent(replica, ClusterEventType.NODE_FAILURE);
-        reportReplicaEvent(event);
-    }
-
-    private String getReplicaIdBySocket(SocketChannel socketChannel) {
-        InetSocketAddress socketAddress = NetworkingUtil.getSocketAddress(socketChannel);
-        for (Replica replica : replicas.values()) {
-            if (replica.getClusterIp().equals(socketAddress.getHostName())
-                    && ncConfig.getReplicationPublicPort() == socketAddress.getPort()) {
-                return replica.getId();
-            }
-        }
-        return null;
-    }
-
-    @Override
-    public void startReplicationThreads() throws InterruptedException {
-        replicationJobsProcessor = new ReplicationJobsProccessor();
-
-        //start/continue processing jobs/logs
-        if (logsRepSockets == null) {
-            establishTxnLogReplicationHandshake();
-            getAndInitNewPage();
-            txnlogReplicator = new TxnLogReplicator(emptyLogBuffersQ, pendingFlushLogBuffersQ);
-            txnLogReplicatorTask = asterixAppRuntimeContextProvider.getThreadExecutor().submit(txnlogReplicator);
-        }
-
-        replicationJobsProcessor.start();
-
-        if (!replicationMonitor.isAlive()) {
-            replicationMonitor.start();
-        }
-
-        //notify any waiting threads that replication has been resumed
-        synchronized (replicationSuspended) {
-            LOGGER.log(Level.INFO, "Replication started/resumed");
-            replicationSuspended.set(false);
-            replicationSuspended.notifyAll();
-        }
-    }
-
-    @Override
-    public void requestFlushLaggingReplicaIndexes(long nonSharpCheckpointTargetLSN) throws IOException {
-        long startLSN = logManager.getAppendLSN();
-        Set<String> replicaIds = getActiveReplicasIds();
-        if (replicaIds.isEmpty()) {
-            return;
-        }
-        ByteBuffer requestBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
-        for (String replicaId : replicaIds) {
-            //1. identify replica indexes with LSN less than nonSharpCheckpointTargetLSN.
-            Map<Long, DatasetResourceReference> laggingIndexes =
-                    replicaResourcesManager.getLaggingReplicaIndexesId2PathMap(replicaId, nonSharpCheckpointTargetLSN);
-
-            if (!laggingIndexes.isEmpty()) {
-                //2. send request to remote replicas that have lagging indexes.
-                ReplicaIndexFlushRequest laggingIndexesResponse = null;
-                try (SocketChannel socketChannel = getReplicaSocket(replicaId)) {
-                    ReplicaIndexFlushRequest laggingIndexesRequest =
-                            new ReplicaIndexFlushRequest(laggingIndexes.keySet());
-                    requestBuffer =
-                            ReplicationProtocol.writeGetReplicaIndexFlushRequest(requestBuffer, laggingIndexesRequest);
-                    NetworkingUtil.transferBufferToChannel(socketChannel, requestBuffer);
-
-                    //3. remote replicas will respond with indexes that were not flushed.
-                    ReplicationRequestType responseFunction = waitForResponse(socketChannel, requestBuffer);
-
-                    if (responseFunction == ReplicationRequestType.FLUSH_INDEX) {
-                        requestBuffer = ReplicationProtocol.readRequest(socketChannel, requestBuffer);
-                        //returning the indexes that were not flushed
-                        laggingIndexesResponse = ReplicationProtocol.readReplicaIndexFlushRequest(requestBuffer);
-                    }
-                    //send goodbye
-                    ReplicationProtocol.sendGoodbye(socketChannel);
-                }
-
-                /*
-                 * 4. update checkpoints for indexes that were not flushed
-                 * to the current append LSN to indicate no operations happened
-                 * since the checkpoint start.
-                 */
-                if (laggingIndexesResponse != null) {
-                    for (Long resouceId : laggingIndexesResponse.getLaggingRescouresIds()) {
-                        final DatasetResourceReference datasetResourceReference = laggingIndexes.get(resouceId);
-                        indexCheckpointManagerProvider.get(datasetResourceReference).advanceLowWatermark(startLSN);
-                    }
-                }
-            }
-        }
-    }
-
-    //Recovery Method
-    @Override
-    public long getMaxRemoteLSN(Set<String> remoteReplicas) throws IOException {
-        long maxRemoteLSN = 0;
-
-        ReplicationProtocol.writeGetReplicaMaxLSNRequest(dataBuffer);
-        Map<String, SocketChannel> replicaSockets = new HashMap<>();
-        try {
-            for (String replicaId : remoteReplicas) {
-                replicaSockets.put(replicaId, getReplicaSocket(replicaId));
-            }
-
-            //send request
-            Iterator<Map.Entry<String, SocketChannel>> iterator = replicaSockets.entrySet().iterator();
-            while (iterator.hasNext()) {
-                Entry<String, SocketChannel> replicaSocket = iterator.next();
-                SocketChannel clientSocket = replicaSocket.getValue();
-                NetworkingUtil.transferBufferToChannel(clientSocket, dataBuffer);
-                dataBuffer.position(0);
-            }
-
-            iterator = replicaSockets.entrySet().iterator();
-            while (iterator.hasNext()) {
-                Entry<String, SocketChannel> replicaSocket = iterator.next();
-                SocketChannel clientSocket = replicaSocket.getValue();
-                //read response
-                NetworkingUtil.readBytes(clientSocket, dataBuffer, Long.BYTES);
-                maxRemoteLSN = Math.max(maxRemoteLSN, dataBuffer.getLong());
-            }
-        } finally {
-            closeReplicaSockets(replicaSockets);
-        }
-
-        return maxRemoteLSN;
-    }
-
-    //Recovery Method
-    @Override
-    public void requestReplicaFiles(String selectedReplicaId, Set<Integer> partitionsToRecover,
-            Set<String> existingFiles) throws IOException {
-        ReplicaFilesRequest request = new ReplicaFilesRequest(partitionsToRecover, existingFiles);
-        dataBuffer = ReplicationProtocol.writeGetReplicaFilesRequest(dataBuffer, request);
-
-        try (SocketChannel socketChannel = getReplicaSocket(selectedReplicaId)) {
-
-            //transfer request
-            NetworkingUtil.transferBufferToChannel(socketChannel, dataBuffer);
-
-            String indexPath;
-            String destFilePath;
-            ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(socketChannel, dataBuffer);
-            LSMIndexFileProperties fileProperties;
-            while (responseFunction != ReplicationRequestType.GOODBYE) {
-                dataBuffer = ReplicationProtocol.readRequest(socketChannel, dataBuffer);
-
-                fileProperties = ReplicationProtocol.readFileReplicationRequest(dataBuffer);
-
-                //get index path
-                indexPath = replicaResourcesManager.getIndexPath(fileProperties);
-                destFilePath = indexPath + File.separator + fileProperties.getFileName();
-
-                //create file
-                File destFile = new File(destFilePath);
-                destFile.createNewFile();
-
-                try (RandomAccessFile fileOutputStream = new RandomAccessFile(destFile, "rw");
-                        FileChannel fileChannel = fileOutputStream.getChannel();) {
-                    fileOutputStream.setLength(fileProperties.getFileSize());
-
-                    NetworkingUtil.downloadFile(fileChannel, socketChannel);
-                    fileChannel.force(true);
-                }
-
-                //we need to create initial map for .metadata files that belong to remote replicas
-                if (!fileProperties.isLSMComponentFile() && !fileProperties.getNodeId().equals(nodeId)) {
-                    final ResourceReference indexRef = ResourceReference.of(destFile.getAbsolutePath());
-                    indexCheckpointManagerProvider.get(indexRef).init(logManager.getAppendLSN());
-                }
-                responseFunction = ReplicationProtocol.getRequestType(socketChannel, dataBuffer);
-            }
-
-            //send goodbye
-            ReplicationProtocol.sendGoodbye(socketChannel);
-        }
-    }
-
-    public int getLogPageSize() {
-        return replicationProperties.getLogBufferPageSize();
-    }
-
-    @Override
-    public void replicateTxnLogBatch(final ByteBuffer buffer) {
-        //if replication is suspended, wait until it is resumed
-        try {
-            while (replicationSuspended.get()) {
-                synchronized (replicationSuspended) {
-                    replicationSuspended.wait();
-                }
-            }
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-        }
-        //prepare the batch size buffer
-        txnLogsBatchSizeBuffer.clear();
-        txnLogsBatchSizeBuffer.putInt(buffer.remaining());
-        txnLogsBatchSizeBuffer.flip();
-
-        buffer.mark();
-        for (SocketChannel replicaSocket : logsRepSockets) {
-            try {
-                //send batch size
-                NetworkingUtil.transferBufferToChannel(replicaSocket, txnLogsBatchSizeBuffer);
-                //send log
-                NetworkingUtil.transferBufferToChannel(replicaSocket, buffer);
-            } catch (IOException e) {
-                handleReplicationFailure(replicaSocket, e);
-            } finally {
-                txnLogsBatchSizeBuffer.position(0);
-                buffer.reset();
-            }
-        }
-        //move the bufeer position to the sent limit
-        buffer.position(buffer.limit());
+        this.appCtx = appCtx;
+        strategy = ReplicationStrategyFactory.create(replicationProperties.getReplicationStrategy());
+        logReplicationManager = new LogReplicationManager(appCtx, this);
+        lsnIndexReplicationManager = new IndexReplicationManager(appCtx, this);
     }
 
     @Override
     public void register(IPartitionReplica replica) {
-        // find the replica node based on ip and replication port
-        Optional<Replica> replicaNode = replicationStrategy.getRemoteReplicasAndSelf(nodeId).stream()
-                .filter(node -> node.getClusterIp().equals(replica.getIdentifier().getLocation().getHostString())
-                        && node.getPort() == replica.getIdentifier().getLocation().getPort())
-                .findAny();
-        if (!replicaNode.isPresent()) {
-            throw new IllegalStateException("Couldn't find node for replica: " + replica);
+        synchronized (dests) {
+            final InetSocketAddress location = replica.getIdentifier().getLocation();
+            final ReplicationDestination replicationDest = dests.computeIfAbsent(location, ReplicationDestination::at);
+            replicationDest.add(replica);
+            logReplicationManager.register(replicationDest);
+            lsnIndexReplicationManager.register(replicationDest);
         }
-        Replica replicaRef = replicaNode.get();
-        final String replicaId = replicaRef.getId();
-        replicas.putIfAbsent(replicaId, replicaRef);
-        replica2PartitionsMap.computeIfAbsent(replicaId, k -> new HashSet<>());
-        replica2PartitionsMap.get(replicaId).add(replica.getIdentifier().getPartition());
-        updateReplicaInfo(replicaRef);
-        checkReplicaState(replicaId, false, true);
     }
 
-    //supporting classes
-    /**
-     * This class is responsible for processing replica events.
-     */
-    private class ReplicasEventsMonitor extends Thread {
-        ReplicaEvent event;
-
-        @Override
-        public void run() {
-            while (true) {
-                try {
-                    event = replicaEventsQ.take();
-
-                    switch (event.getEventType()) {
-                        case NODE_FAILURE:
-                            handleReplicaFailure(event.getReplica().getId());
-                            break;
-                        case NODE_JOIN:
-                            updateReplicaInfo(event.getReplica());
-                            checkReplicaState(event.getReplica().getId(), false, true);
-                            break;
-                        case NODE_SHUTTING_DOWN:
-                            handleShutdownEvent(event.getReplica().getId());
-                            break;
-                        default:
-                            break;
-                    }
-                } catch (InterruptedException e) {
-                    Thread.currentThread().interrupt();
-                }
-            }
-        }
-
-        public void handleReplicaFailure(String replicaId) throws InterruptedException {
-            Replica replica = replicas.get(replicaId);
-
-            if (replica.getState() == ReplicaState.DEAD) {
+    @Override
+    public void unregister(IPartitionReplica replica) {
+        synchronized (dests) {
+            final InetSocketAddress location = replica.getIdentifier().getLocation();
+            final ReplicationDestination dest = dests.get(location);
+            if (dest == null) {
+                LOGGER.warn(() -> "Asked to unregister unknown replica " + replica);
                 return;
             }
-
-            updateReplicaState(replicaId, ReplicaState.DEAD, true);
-
-            //delete any invalid LSMComponents for this replica
-            replicaResourcesManager.cleanInvalidLSMComponents(replicaId);
-        }
-
-        public void handleShutdownEvent(String replicaId) {
-            synchronized (shuttingDownReplicaIds) {
-                shuttingDownReplicaIds.add(replicaId);
-                shuttingDownReplicaIds.notifyAll();
-            }
-        }
-    }
-
-    /**
-     * This class process is responsible for processing ASYNC replication job.
-     */
-    private class ReplicationJobsProccessor extends Thread {
-        Map<String, SocketChannel> replicaSockets;
-        ByteBuffer reusableBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
-
-        @Override
-        public void run() {
-            Thread.currentThread().setName("ReplicationJobsProccessor Thread");
-            terminateJobsReplication.set(false);
-            jobsReplicationSuspended.set(false);
-
-            while (true) {
-                try {
-                    if (terminateJobsReplication.get()) {
-                        closeSockets();
-                        break;
-                    }
-
-                    IReplicationJob job = replicationJobsQ.take();
-                    if (job == REPLICATION_JOB_POISON_PILL) {
-                        terminateJobsReplication.set(true);
-                        continue;
-                    }
-
-                    //if there isn't already a connection, establish a new one
-                    if (replicaSockets == null) {
-                        replicaSockets = getActiveRemoteReplicasSockets();
-                    }
-                    processJob(job, replicaSockets, reusableBuffer);
-
-                    //if no more jobs to process, close sockets
-                    if (replicationJobsQ.isEmpty()) {
-                        LOGGER.log(Level.INFO, "No pending replication jobs. Closing connections to replicas");
-                        closeSockets();
-                    }
-                } catch (InterruptedException e) {
-                    Thread.currentThread().interrupt();
-                } catch (IOException e) {
-                    LOGGER.warn("Couldn't complete processing replication job", e);
-                }
-            }
-
-            synchronized (jobsReplicationSuspended) {
-                jobsReplicationSuspended.set(true);
-                jobsReplicationSuspended.notifyAll();
-            }
-            LOGGER.log(Level.INFO, "ReplicationJobsProccessor stopped. ");
-        }
-
-        private void closeSockets() {
-            if (replicaSockets != null) {
-                closeReplicaSockets(replicaSockets);
-                replicaSockets.clear();
-                replicaSockets = null;
-            }
-        }
-    }
-
-    /**
-     * This class is responsible for listening on sockets that belong to TxnLogsReplicator.
-     */
-    private class TxnLogsReplicationResponseListener implements Runnable {
-        final SocketChannel replicaSocket;
-        final String replicaId;
-
-        public TxnLogsReplicationResponseListener(String replicaId, SocketChannel replicaSocket) {
-            this.replicaId = replicaId;
-            this.replicaSocket = replicaSocket;
-        }
-
-        @Override
-        public void run() {
-            Thread.currentThread().setName("TxnLogs Replication Listener Thread");
-            LOGGER.log(Level.INFO, "Started listening on socket: " + replicaSocket.socket().getRemoteSocketAddress());
-
-            try (BufferedReader incomingResponse =
-                    new BufferedReader(new InputStreamReader(replicaSocket.socket().getInputStream()))) {
-                while (true) {
-                    String responseLine = incomingResponse.readLine();
-                    if (responseLine == null) {
-                        break;
-                    }
-                    //read ACK for job commit log
-                    String ackFrom = ReplicationProtocol.getNodeIdFromLogAckMessage(responseLine);
-                    int jobId = ReplicationProtocol.getJobIdFromLogAckMessage(responseLine);
-                    addAckToJob(jobId, ackFrom);
-                }
-            } catch (AsynchronousCloseException e) {
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.log(Level.INFO, "Replication listener stopped for remote replica: " + replicaId, e);
-                }
-            } catch (IOException e) {
-                handleReplicationFailure(replicaSocket, e);
+            LOGGER.info(() -> "unregister " + replica);
+            dest.remove(replica);
+            if (dest.getReplicas().isEmpty()) {
+                LOGGER.info(() -> "Removing destination with no replicas " + dest);
+                logReplicationManager.unregister(dest);
+                lsnIndexReplicationManager.unregister(dest);
+                dests.remove(location);
             }
         }
     }
 
     @Override
-    public IReplicationStrategy getReplicationStrategy() {
-        return replicationStrategy;
+    public void notifyFailure(IReplicationDestination dest, Exception failure) {
+        LOGGER.info(() -> "processing failure for " + dest);
+        appCtx.getThreadExecutor().execute(() -> {
+            logReplicationManager.unregister(dest);
+            lsnIndexReplicationManager.unregister(dest);
+            dest.notifyFailure(failure);
+        });
     }
-}
+
+    @Override
+    public void replicate(ILogRecord logRecord) throws InterruptedException {
+        logReplicationManager.replicate(logRecord);
+    }
+
+    @Override
+    public IReplicationStrategy getReplicationStrategy() {
+        return strategy;
+    }
+
+    @Override
+    public void submitJob(IReplicationJob job) {
+        lsnIndexReplicationManager.accept(job);
+    }
+
+    @Override
+    public boolean isReplicationEnabled() {
+        return replicationProperties.isReplicationEnabled();
+    }
+
+    @Override
+    public void start() {
+        // no op
+    }
+
+    @Override
+    public void dumpState(OutputStream os) {
+        // no op
+    }
+
+    @Override
+    public void stop(boolean dumpState, OutputStream ouputStream) throws IOException {
+        LOGGER.info("Closing replication channel");
+        appCtx.getReplicationChannel().close();
+        LOGGER.info("Replication manager stopped");
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
index 2c1937b..84922cd 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
@@ -25,12 +25,11 @@
 import java.util.Collection;
 
 import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.replication.IReplicationThread;
+import org.apache.asterix.replication.api.IReplicationWorker;
 import org.apache.asterix.common.storage.DatasetResourceReference;
 import org.apache.asterix.common.storage.IIndexCheckpointManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.asterix.replication.api.IReplicaTask;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.common.LocalResource;
@@ -47,7 +46,7 @@
     }
 
     @Override
-    public void perform(INcApplicationContext appCtx, IReplicationThread worker) throws HyracksDataException {
+    public void perform(INcApplicationContext appCtx, IReplicationWorker worker) throws HyracksDataException {
         final IIndexCheckpointManagerProvider indexCheckpointManagerProvider =
                 appCtx.getIndexCheckpointManagerProvider();
         PersistentLocalResourceRepository resRepo =
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ComponentMaskTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ComponentMaskTask.java
new file mode 100644
index 0000000..26c9577
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ComponentMaskTask.java
@@ -0,0 +1,92 @@
+/*
+ * 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.replication.messaging;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.common.utils.StorageConstants;
+import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IIOManager;
+
+/**
+ * A task to create a mask file for an incoming lsm component from master
+ */
+public class ComponentMaskTask implements IReplicaTask {
+
+    private static final String COMPONENT_MASK_FILE_PREFIX = StorageConstants.MASK_FILE_PREFIX + "C_";
+    private final String file;
+    private final String componentId;
+
+    public ComponentMaskTask(String file, String componentId) {
+        this.file = file;
+        this.componentId = componentId;
+    }
+
+    @Override
+    public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
+        try {
+            // create mask
+            final Path maskPath = getComponentMaskPath(appCtx, file);
+            Files.createFile(maskPath);
+            ReplicationProtocol.sendAck(worker.getChannel(), worker.getReusableBuffer());
+        } catch (IOException e) {
+            throw new ReplicationException(e);
+        }
+    }
+
+    public static Path getComponentMaskPath(INcApplicationContext appCtx, String file) throws IOException {
+        final IIOManager ioManager = appCtx.getIoManager();
+        final FileReference localPath = ioManager.resolve(file);
+        final Path resourceDir = Files.createDirectories(localPath.getFile().getParentFile().toPath());
+        return Paths.get(resourceDir.toString(), COMPONENT_MASK_FILE_PREFIX + localPath.getFile().getName());
+    }
+
+    @Override
+    public ReplicationProtocol.ReplicationRequestType getMessageType() {
+        return ReplicationProtocol.ReplicationRequestType.LSM_COMPONENT_MASK;
+    }
+
+    @Override
+    public void serialize(OutputStream out) throws HyracksDataException {
+        try {
+            final DataOutputStream dos = new DataOutputStream(out);
+            dos.writeUTF(file);
+            dos.writeUTF(componentId);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    public static ComponentMaskTask create(DataInput input) throws IOException {
+        String indexFile = input.readUTF();
+        String componentId = input.readUTF();
+        return new ComponentMaskTask(indexFile, componentId);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DeleteFileTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DeleteFileTask.java
index d4de3b7..1b5470d 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DeleteFileTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DeleteFileTask.java
@@ -27,9 +27,8 @@
 
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.exceptions.ReplicationException;
-import org.apache.asterix.common.replication.IReplicationThread;
+import org.apache.asterix.replication.api.IReplicationWorker;
 import org.apache.asterix.replication.api.IReplicaTask;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.logging.log4j.LogManager;
@@ -48,7 +47,7 @@
     }
 
     @Override
-    public void perform(INcApplicationContext appCtx, IReplicationThread worker) {
+    public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
         try {
             final IIOManager ioManager = appCtx.getIoManager();
             final File localFile = ioManager.resolve(file).getFile();
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DropIndexTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DropIndexTask.java
new file mode 100644
index 0000000..b7f0985
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DropIndexTask.java
@@ -0,0 +1,84 @@
+/*
+ * 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.replication.messaging;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.util.IoUtil;
+
+/**
+ * A task to drop an index that was dropped on master
+ */
+public class DropIndexTask implements IReplicaTask {
+
+    private static final Logger LOGGER = Logger.getLogger(DeleteFileTask.class.getName());
+    private final String file;
+
+    public DropIndexTask(String file) {
+        this.file = file;
+    }
+
+    @Override
+    public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
+        try {
+            final IIOManager ioManager = appCtx.getIoManager();
+            final File indexFile = ioManager.resolve(file).getFile();
+            if (indexFile.exists()) {
+                File indexDir = indexFile.getParentFile();
+                IoUtil.deleteDirectory(indexDir);
+                LOGGER.info(() -> "Deleted index: " + indexFile.getAbsolutePath());
+            } else {
+                LOGGER.warning(() -> "Requested to delete a non-existing index: " + indexFile.getAbsolutePath());
+            }
+            ReplicationProtocol.sendAck(worker.getChannel(), worker.getReusableBuffer());
+        } catch (IOException e) {
+            throw new ReplicationException(e);
+        }
+    }
+
+    @Override
+    public ReplicationProtocol.ReplicationRequestType getMessageType() {
+        return ReplicationProtocol.ReplicationRequestType.REPLICATE_RESOURCE_FILE;
+    }
+
+    @Override
+    public void serialize(OutputStream out) throws HyracksDataException {
+        try {
+            DataOutputStream dos = new DataOutputStream(out);
+            dos.writeUTF(file);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    public static DropIndexTask create(DataInput input) throws IOException {
+        return new DropIndexTask(input.readUTF());
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
new file mode 100644
index 0000000..b581321
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
@@ -0,0 +1,110 @@
+/*
+ * 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.replication.messaging;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+
+/**
+ * A task to mark a replicated LSM component as valid
+ */
+public class MarkComponentValidTask implements IReplicaTask {
+
+    private final long masterLsn;
+    private final String file;
+
+    public MarkComponentValidTask(String file, long masterLsn) {
+        this.file = file;
+        this.masterLsn = masterLsn;
+    }
+
+    @Override
+    public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
+        try {
+            if (masterLsn > 0) {
+                ensureComponentLsnFlushed(appCtx);
+            }
+            // delete mask
+            final Path maskPath = ComponentMaskTask.getComponentMaskPath(appCtx, file);
+            Files.delete(maskPath);
+            ReplicationProtocol.sendAck(worker.getChannel(), worker.getReusableBuffer());
+        } catch (IOException | InterruptedException e) {
+            throw new ReplicationException(e);
+        }
+    }
+
+    private void ensureComponentLsnFlushed(INcApplicationContext appCtx)
+            throws HyracksDataException, InterruptedException {
+        final ResourceReference indexRef = ResourceReference.of(file);
+        final IIndexCheckpointManagerProvider checkpointManagerProvider = appCtx.getIndexCheckpointManagerProvider();
+        final IIndexCheckpointManager indexCheckpointManager = checkpointManagerProvider.get(indexRef);
+        long replicationTimeOut = TimeUnit.SECONDS.toMillis(appCtx.getReplicationProperties().getReplicationTimeOut());
+        final long startTime = System.nanoTime();
+        synchronized (indexCheckpointManager) {
+            // wait until the lsn mapping is flushed to disk
+            while (!indexCheckpointManager.isFlushed(masterLsn)) {
+                if (replicationTimeOut <= 0) {
+                    throw new ReplicationException(new TimeoutException("Couldn't receive flush lsn from master"));
+                }
+                indexCheckpointManager.wait(replicationTimeOut);
+                replicationTimeOut -= TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
+            }
+            final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(indexRef.getName());
+            indexCheckpointManager.replicated(componentEndTime, masterLsn);
+        }
+    }
+
+    @Override
+    public ReplicationProtocol.ReplicationRequestType getMessageType() {
+        return ReplicationProtocol.ReplicationRequestType.MARK_COMPONENT_VALID;
+    }
+
+    @Override
+    public void serialize(OutputStream out) throws HyracksDataException {
+        try {
+            final DataOutputStream dos = new DataOutputStream(out);
+            dos.writeUTF(file);
+            dos.writeLong(masterLsn);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    public static MarkComponentValidTask create(DataInput input) throws IOException {
+        final String indexFile = input.readUTF();
+        final long lsn = input.readLong();
+        return new MarkComponentValidTask(indexFile, lsn);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListResponse.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListResponse.java
index 85b7bb9..561a6bf 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListResponse.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListResponse.java
@@ -26,7 +26,6 @@
 import java.util.List;
 
 import org.apache.asterix.replication.api.IReplicationMessage;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class PartitionResourcesListResponse implements IReplicationMessage {
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java
index b2b8ac6..b972f32 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java
@@ -26,10 +26,10 @@
 import java.util.stream.Collectors;
 
 import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.replication.IReplicationThread;
+import org.apache.asterix.replication.api.IReplicationWorker;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.replication.api.IReplicaTask;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
@@ -44,11 +44,12 @@
     }
 
     @Override
-    public void perform(INcApplicationContext appCtx, IReplicationThread worker) throws HyracksDataException {
+    public void perform(INcApplicationContext appCtx, IReplicationWorker worker) throws HyracksDataException {
         //TODO delete any invalid files with masks
-        final List<String> partitionResources =
-                appCtx.getReplicaResourcesManager().getPartitionIndexesFiles(partition, false).stream()
-                        .map(StoragePathUtil::getFileRelativePath).collect(Collectors.toList());
+        final PersistentLocalResourceRepository localResourceRepository =
+                (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+        final List<String> partitionResources = localResourceRepository.getPartitionIndexesFiles(partition).stream()
+                .map(StoragePathUtil::getFileRelativePath).collect(Collectors.toList());
         final PartitionResourcesListResponse response =
                 new PartitionResourcesListResponse(partition, partitionResources);
         ReplicationProtocol.sendTo(worker.getChannel(), response, worker.getReusableBuffer());
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
index 45d8971..99c7256 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
@@ -30,10 +30,12 @@
 
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.exceptions.ReplicationException;
-import org.apache.asterix.common.replication.IReplicationThread;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.storage.ResourceReference;
 import org.apache.asterix.common.utils.StorageConstants;
 import org.apache.asterix.replication.api.IReplicaTask;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
 import org.apache.asterix.replication.management.NetworkingUtil;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
@@ -49,14 +51,16 @@
     private static final Logger LOGGER = LogManager.getLogger();
     private final String file;
     private final long size;
+    private final boolean indexMetadata;
 
-    public ReplicateFileTask(String file, long size) {
+    public ReplicateFileTask(String file, long size, boolean indexMetadata) {
         this.file = file;
         this.size = size;
+        this.indexMetadata = indexMetadata;
     }
 
     @Override
-    public void perform(INcApplicationContext appCtx, IReplicationThread worker) throws HyracksDataException {
+    public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
         try {
             final IIOManager ioManager = appCtx.getIoManager();
             // resolve path
@@ -76,6 +80,9 @@
                 NetworkingUtil.downloadFile(fileChannel, worker.getChannel());
                 fileChannel.force(true);
             }
+            if (indexMetadata) {
+                initIndexCheckpoint(appCtx);
+            }
             //delete mask
             Files.delete(maskPath);
             LOGGER.info(() -> "Replicated file: " + localPath);
@@ -85,6 +92,16 @@
         }
     }
 
+    private void initIndexCheckpoint(INcApplicationContext appCtx) throws HyracksDataException {
+        final ResourceReference indexRef = ResourceReference.of(file);
+        final IIndexCheckpointManagerProvider checkpointManagerProvider = appCtx.getIndexCheckpointManagerProvider();
+        final IIndexCheckpointManager indexCheckpointManager = checkpointManagerProvider.get(indexRef);
+        final long currentLSN = appCtx.getTransactionSubsystem().getLogManager().getAppendLSN();
+        indexCheckpointManager.delete();
+        indexCheckpointManager.init(currentLSN);
+        LOGGER.info(() -> "Checkpoint index: " + indexRef);
+    }
+
     @Override
     public ReplicationProtocol.ReplicationRequestType getMessageType() {
         return ReplicationProtocol.ReplicationRequestType.REPLICATE_RESOURCE_FILE;
@@ -96,6 +113,7 @@
             DataOutputStream dos = new DataOutputStream(out);
             dos.writeUTF(file);
             dos.writeLong(size);
+            dos.writeBoolean(indexMetadata);
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
@@ -104,6 +122,7 @@
     public static ReplicateFileTask create(DataInput input) throws IOException {
         final String s = input.readUTF();
         final long i = input.readLong();
-        return new ReplicateFileTask(s, i);
+        final boolean isMetadata = input.readBoolean();
+        return new ReplicateFileTask(s, i, isMetadata);
     }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateLogsTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateLogsTask.java
new file mode 100644
index 0000000..b71f4b8
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateLogsTask.java
@@ -0,0 +1,92 @@
+/*
+ * 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.replication.messaging;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.common.transactions.ILogManager;
+import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.asterix.replication.logging.RemoteLogRecord;
+import org.apache.asterix.replication.logging.RemoteLogsProcessor;
+import org.apache.asterix.replication.management.ReplicationChannel;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * A task to replicate transaction logs from master replica
+ */
+public class ReplicateLogsTask implements IReplicaTask {
+
+    public static final int END_REPLICATION_LOG_SIZE = 1;
+    private final String nodeId;
+
+    public ReplicateLogsTask(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    @Override
+    public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
+        final ReplicationChannel replicationChannel = (ReplicationChannel) appCtx.getReplicationChannel();
+        final RemoteLogsProcessor logsProcessor = replicationChannel.getRemoteLogsProcessor();
+        final ILogManager logManager = appCtx.getTransactionSubsystem().getLogManager();
+        final RemoteLogRecord reusableLog = new RemoteLogRecord();
+        final SocketChannel channel = worker.getChannel();
+        ByteBuffer logsBuffer = ByteBuffer.allocate(logManager.getLogPageSize());
+        try {
+            while (true) {
+                // read a batch of logs
+                logsBuffer = ReplicationProtocol.readRequest(channel, logsBuffer);
+                // check if it is end of handshake
+                if (logsBuffer.remaining() == END_REPLICATION_LOG_SIZE) {
+                    break;
+                }
+                logsProcessor.process(logsBuffer, reusableLog, worker);
+            }
+        } catch (IOException e) {
+            throw new ReplicationException(e);
+        }
+    }
+
+    @Override
+    public ReplicationProtocol.ReplicationRequestType getMessageType() {
+        return ReplicationProtocol.ReplicationRequestType.REPLICATE_LOGS;
+    }
+
+    @Override
+    public void serialize(OutputStream out) throws HyracksDataException {
+        try {
+            DataOutputStream dos = new DataOutputStream(out);
+            dos.writeUTF(nodeId);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    public static ReplicateLogsTask create(DataInput input) throws IOException {
+        final String node = input.readUTF();
+        return new ReplicateLogsTask(node);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicationProtocol.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicationProtocol.java
new file mode 100644
index 0000000..280a2d4
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicationProtocol.java
@@ -0,0 +1,200 @@
+/*
+ * 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.replication.messaging;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Stream;
+
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.replication.api.IReplicationMessage;
+import org.apache.asterix.replication.api.PartitionReplica;
+import org.apache.asterix.replication.management.NetworkingUtil;
+import org.apache.hyracks.data.std.util.ExtendedByteArrayOutputStream;
+import org.apache.hyracks.util.StorageUtil;
+
+public class ReplicationProtocol {
+
+    /**
+     * All replication messages start with ReplicationRequestType (4 bytes), then the length of the request in bytes
+     */
+    public static final String LOG_REPLICATION_ACK = "$";
+    public static final int INITIAL_BUFFER_SIZE = StorageUtil.getIntSizeInBytes(4, StorageUtil.StorageUnit.KILOBYTE);
+    private static final int REPLICATION_REQUEST_TYPE_SIZE = Integer.BYTES;
+    private static final int REPLICATION_REQUEST_HEADER_SIZE = REPLICATION_REQUEST_TYPE_SIZE + Integer.BYTES;
+
+    public enum ReplicationRequestType {
+        GOODBYE,
+        ACK,
+        PARTITION_RESOURCES_REQUEST,
+        PARTITION_RESOURCES_RESPONSE,
+        REPLICATE_RESOURCE_FILE,
+        DELETE_RESOURCE_FILE,
+        CHECKPOINT_PARTITION,
+        LSM_COMPONENT_MASK,
+        MARK_COMPONENT_VALID,
+        DROP_INDEX,
+        REPLICATE_LOGS
+    }
+
+    private static final Map<Integer, ReplicationRequestType> TYPES = new HashMap<>();
+
+    static {
+        Stream.of(ReplicationRequestType.values()).forEach(type -> TYPES.put(type.ordinal(), type));
+    }
+
+    public static ByteBuffer readRequest(SocketChannel socketChannel, ByteBuffer dataBuffer) throws IOException {
+        // read request size
+        NetworkingUtil.readBytes(socketChannel, dataBuffer, Integer.BYTES);
+        final int requestSize = dataBuffer.getInt();
+        final ByteBuffer buf = ensureSize(dataBuffer, requestSize);
+        // read request
+        NetworkingUtil.readBytes(socketChannel, buf, requestSize);
+        return dataBuffer;
+    }
+
+    public static ReplicationRequestType getRequestType(SocketChannel socketChannel, ByteBuffer byteBuffer)
+            throws IOException {
+        // read replication request type
+        NetworkingUtil.readBytes(socketChannel, byteBuffer, REPLICATION_REQUEST_TYPE_SIZE);
+        return TYPES.get(byteBuffer.getInt());
+    }
+
+    private static ByteBuffer getGoodbyeBuffer() {
+        ByteBuffer bb = ByteBuffer.allocate(REPLICATION_REQUEST_TYPE_SIZE);
+        bb.putInt(ReplicationRequestType.GOODBYE.ordinal());
+        bb.flip();
+        return bb;
+    }
+
+    public static int getTxnIdFromLogAckMessage(String msg) {
+        return Integer.parseInt(msg.substring(msg.indexOf(LOG_REPLICATION_ACK) + 1));
+    }
+
+    public static void sendGoodbye(SocketChannel socketChannel) throws IOException {
+        ByteBuffer goodbyeBuffer = ReplicationProtocol.getGoodbyeBuffer();
+        NetworkingUtil.transferBufferToChannel(socketChannel, goodbyeBuffer);
+    }
+
+    public static void sendAck(SocketChannel socketChannel, ByteBuffer buf) {
+        try {
+            buf.clear();
+            buf.putInt(ReplicationRequestType.ACK.ordinal());
+            buf.flip();
+            NetworkingUtil.transferBufferToChannel(socketChannel, buf);
+        } catch (IOException e) {
+            throw new ReplicationException(e);
+        }
+    }
+
+    public static void waitForAck(PartitionReplica replica) throws IOException {
+        final SocketChannel channel = replica.getChannel();
+        final ByteBuffer buf = replica.getReusableBuffer();
+        ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(channel, buf);
+        if (responseFunction != ReplicationRequestType.ACK) {
+            throw new IllegalStateException("Unexpected response while waiting for ack.");
+        }
+    }
+
+    public static void sendTo(PartitionReplica replica, IReplicationMessage task) {
+        final SocketChannel channel = replica.getChannel();
+        final ByteBuffer buf = replica.getReusableBuffer();
+        sendTo(channel, task, buf);
+    }
+
+    public static void sendTo(SocketChannel channel, IReplicationMessage task, ByteBuffer buf) {
+        ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
+        try (DataOutputStream oos = new DataOutputStream(outputStream)) {
+            task.serialize(oos);
+            final int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
+            final ByteBuffer requestBuffer = ensureSize(buf, requestSize);
+            requestBuffer.putInt(task.getMessageType().ordinal());
+            requestBuffer.putInt(oos.size());
+            requestBuffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
+            requestBuffer.flip();
+            NetworkingUtil.transferBufferToChannel(channel, requestBuffer);
+        } catch (IOException e) {
+            throw new ReplicationException(e);
+        }
+    }
+
+    public static IReplicationMessage read(SocketChannel socketChannel, ByteBuffer buffer) throws IOException {
+        final ReplicationRequestType type = getRequestType(socketChannel, buffer);
+        return readMessage(type, socketChannel, buffer);
+    }
+
+    public static IReplicationMessage readMessage(ReplicationRequestType type, SocketChannel socketChannel,
+            ByteBuffer buffer) {
+        try {
+            ReplicationProtocol.readRequest(socketChannel, buffer);
+            final ByteArrayInputStream input =
+                    new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
+            try (DataInputStream dis = new DataInputStream(input)) {
+                switch (type) {
+                    case PARTITION_RESOURCES_REQUEST:
+                        return PartitionResourcesListTask.create(dis);
+                    case PARTITION_RESOURCES_RESPONSE:
+                        return PartitionResourcesListResponse.create(dis);
+                    case REPLICATE_RESOURCE_FILE:
+                        return ReplicateFileTask.create(dis);
+                    case DELETE_RESOURCE_FILE:
+                        return DeleteFileTask.create(dis);
+                    case CHECKPOINT_PARTITION:
+                        return CheckpointPartitionIndexesTask.create(dis);
+                    case LSM_COMPONENT_MASK:
+                        return ComponentMaskTask.create(dis);
+                    case DROP_INDEX:
+                        return DropIndexTask.create(dis);
+                    case MARK_COMPONENT_VALID:
+                        return MarkComponentValidTask.create(dis);
+                    case REPLICATE_LOGS:
+                        return ReplicateLogsTask.create(dis);
+                    default:
+                        throw new IllegalStateException("Unrecognized replication message");
+                }
+            }
+        } catch (IOException e) {
+            throw new ReplicationException(e);
+        }
+    }
+
+    public static ByteBuffer getEndLogReplicationBuffer() {
+        final int logsBatchSize = 1;
+        final ByteBuffer endLogRepBuffer =
+                ByteBuffer.allocate(Integer.BYTES + ReplicateLogsTask.END_REPLICATION_LOG_SIZE);
+        endLogRepBuffer.putInt(logsBatchSize);
+        endLogRepBuffer.put((byte) 0);
+        endLogRepBuffer.flip();
+        return endLogRepBuffer;
+    }
+
+    private static ByteBuffer ensureSize(ByteBuffer buffer, int size) {
+        if (buffer == null || buffer.capacity() < size) {
+            return ByteBuffer.allocate(size);
+        }
+        buffer.clear();
+        return buffer;
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java
deleted file mode 100644
index 5d044b4..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java
+++ /dev/null
@@ -1,320 +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.replication.recovery;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.asterix.common.api.IDatasetLifecycleManager;
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.config.ReplicationProperties;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.replication.IRemoteRecoveryManager;
-import org.apache.asterix.common.replication.IReplicationManager;
-import org.apache.asterix.common.replication.IReplicationStrategy;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.transactions.ILogManager;
-import org.apache.asterix.common.transactions.IRecoveryManager;
-import org.apache.asterix.replication.storage.ReplicaResourcesManager;
-import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class RemoteRecoveryManager implements IRemoteRecoveryManager {
-
-    private final IReplicationManager replicationManager;
-    private static final Logger LOGGER = LogManager.getLogger();
-    private final INcApplicationContext runtimeContext;
-    private final ReplicationProperties replicationProperties;
-    private Map<String, Set<String>> failbackRecoveryReplicas;
-    private IReplicationStrategy replicationStrategy;
-
-    public RemoteRecoveryManager(IReplicationManager replicationManager, INcApplicationContext runtimeContext,
-            ReplicationProperties replicationProperties) {
-        this.replicationManager = replicationManager;
-        this.runtimeContext = runtimeContext;
-        this.replicationProperties = replicationProperties;
-        this.replicationStrategy = replicationManager.getReplicationStrategy();
-    }
-
-    private Map<String, Set<String>> constructRemoteRecoveryPlan() {
-        //1. identify which replicas reside in this node
-        String localNodeId = runtimeContext.getTransactionSubsystem().getId();
-
-        Set<Replica> replicas = replicationStrategy.getRemoteReplicasAndSelf(localNodeId);
-        Map<String, Set<String>> recoveryCandidates = new HashMap<>();
-        Map<String, Integer> candidatesScore = new HashMap<>();
-
-        //2. identify which nodes has backup per lost node data
-        for (Replica node : replicas) {
-            Set<Replica> locations = replicationStrategy.getRemoteReplicasAndSelf(node.getId());
-
-            //since the local node just started, remove it from candidates
-            locations.remove(new Replica(localNodeId, "", -1));
-
-            //remove any dead replicas
-            Set<String> deadReplicas = replicationManager.getDeadReplicasIds();
-            for (String deadReplica : deadReplicas) {
-                locations.remove(new Replica(deadReplica, "", -1));
-            }
-
-            //no active replicas to recover from
-            if (locations.isEmpty()) {
-                throw new IllegalStateException("Could not find any ACTIVE replica to recover " + node + " data.");
-            }
-
-            for (Replica locationRep : locations) {
-                String location = locationRep.getId();
-                if (candidatesScore.containsKey(location)) {
-                    candidatesScore.put(location, candidatesScore.get(location) + 1);
-                } else {
-                    candidatesScore.put(location, 1);
-                }
-            }
-            recoveryCandidates.put(node.getId(), locations.stream().map(Replica::getId).collect(Collectors.toSet()));
-        }
-
-        Map<String, Set<String>> recoveryList = new HashMap<>();
-
-        //3. find best candidate to recover from per lost replica data
-        recoveryCandidates.forEach((key, value) -> {
-            int winnerScore = -1;
-            String winner = "";
-            for (String node : value) {
-
-                int nodeScore = candidatesScore.get(node);
-
-                if (nodeScore > winnerScore) {
-                    winnerScore = nodeScore;
-                    winner = node;
-                }
-            }
-
-            if (recoveryList.containsKey(winner)) {
-                recoveryList.get(winner).add(key);
-            } else {
-                Set<String> nodesToRecover = new HashSet<>();
-                nodesToRecover.add(key);
-                recoveryList.put(winner, nodesToRecover);
-            }
-
-        });
-
-        return recoveryList;
-    }
-
-    @Override
-    public void replayReplicaPartitionLogs(Set<Integer> partitions, boolean flush) throws HyracksDataException {
-        ILogManager logManager = runtimeContext.getTransactionSubsystem().getLogManager();
-        long minLSN = runtimeContext.getReplicaResourcesManager().getPartitionsMinLSN(partitions);
-        long readableSmallestLSN = logManager.getReadableSmallestLSN();
-        if (minLSN < readableSmallestLSN) {
-            minLSN = readableSmallestLSN;
-        }
-
-        //replay logs > minLSN that belong to these partitions
-        IRecoveryManager recoveryManager = runtimeContext.getTransactionSubsystem().getRecoveryManager();
-        try {
-            recoveryManager.replayPartitionsLogs(partitions, logManager.getLogReader(true), minLSN);
-            if (flush) {
-                runtimeContext.getDatasetLifecycleManager().flushAllDatasets();
-            }
-        } catch (IOException | ACIDException e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
-    @Override
-    public void takeoverPartitons(Integer[] partitions) throws IOException, ACIDException {
-        /*
-         * TODO even though the takeover is always expected to succeed,
-         * in case of any failure during the takeover, the CC should be
-         * notified that the takeover failed.
-         */
-        Set<Integer> partitionsToTakeover = new HashSet<>(Arrays.asList(partitions));
-        replayReplicaPartitionLogs(partitionsToTakeover, false);
-
-        //mark these partitions as active in this node
-        PersistentLocalResourceRepository resourceRepository = (PersistentLocalResourceRepository) runtimeContext
-                .getLocalResourceRepository();
-        for (Integer patitionId : partitions) {
-            resourceRepository.addActivePartition(patitionId);
-        }
-    }
-
-    @Override
-    public void startFailbackProcess() {
-        int maxRecoveryAttempts = replicationProperties.getMaxRemoteRecoveryAttempts();
-        PersistentLocalResourceRepository resourceRepository = (PersistentLocalResourceRepository) runtimeContext
-                .getLocalResourceRepository();
-        IDatasetLifecycleManager datasetLifeCycleManager = runtimeContext.getDatasetLifecycleManager();
-        Map<String, ClusterPartition[]> nodePartitions = runtimeContext.getMetadataProperties().getNodePartitions();
-
-        while (true) {
-            //start recovery steps
-            try {
-                if (maxRecoveryAttempts <= 0) {
-                    //to avoid infinite loop in case of unexpected behavior.
-                    throw new IllegalStateException("Failed to perform remote recovery.");
-                }
-
-                /*** Prepare for Recovery ***/
-                //1. check remote replicas states
-                replicationManager.initializeReplicasState();
-                int activeReplicasCount = replicationManager.getActiveReplicasCount();
-
-                if (activeReplicasCount == 0) {
-                    throw new IllegalStateException("no ACTIVE remote replica(s) exists to perform remote recovery");
-                }
-
-                //2. clean any memory data that could've existed from previous failed recovery attempt
-                datasetLifeCycleManager.closeAllDatasets();
-
-                //3. remove any existing storage data and initialize storage metadata
-                resourceRepository.deleteStorageData();
-
-                //4. select remote replicas to recover from per lost replica data
-                failbackRecoveryReplicas = constructRemoteRecoveryPlan();
-
-                /*** Start Recovery Per Lost Replica ***/
-                for (Entry<String, Set<String>> remoteReplica : failbackRecoveryReplicas.entrySet()) {
-                    String replicaId = remoteReplica.getKey();
-                    Set<String> ncsToRecoverFor = remoteReplica.getValue();
-                    Set<Integer> partitionsIds = new HashSet<>();
-                    for (String node : ncsToRecoverFor) {
-                        partitionsIds.addAll((Arrays.asList(nodePartitions.get(node))).stream()
-                                .map(ClusterPartition::getPartitionId).collect(Collectors.toList()));
-                    }
-
-                    //1. Request indexes metadata and LSM components
-                    replicationManager.requestReplicaFiles(replicaId, partitionsIds, new HashSet<String>());
-                }
-                break;
-            } catch (IOException e) {
-                LOGGER.warn("Failed during remote recovery. Attempting again...", e);
-                maxRecoveryAttempts--;
-            }
-        }
-    }
-
-    @Override
-    public void completeFailbackProcess() throws IOException, InterruptedException {
-        ILogManager logManager = runtimeContext.getTransactionSubsystem().getLogManager();
-        ReplicaResourcesManager replicaResourcesManager = (ReplicaResourcesManager) runtimeContext
-                .getReplicaResourcesManager();
-        Map<String, ClusterPartition[]> nodePartitions = runtimeContext.getMetadataProperties().getNodePartitions();
-
-        /*
-         * for each lost partition, get the remaining files from replicas
-         * to complete the failback process.
-         */
-        try {
-            for (Entry<String, Set<String>> remoteReplica : failbackRecoveryReplicas.entrySet()) {
-                String replicaId = remoteReplica.getKey();
-                Set<String> NCsDataToRecover = remoteReplica.getValue();
-                Set<String> existingFiles = new HashSet<>();
-                Set<Integer> partitionsToRecover = new HashSet<>();
-                for (String nodeId : NCsDataToRecover) {
-                    //get partitions that will be recovered from this node
-                    ClusterPartition[] replicaPartitions = nodePartitions.get(nodeId);
-                    for (ClusterPartition partition : replicaPartitions) {
-                        existingFiles.addAll(
-                                replicaResourcesManager.getPartitionIndexesFiles(partition.getPartitionId(), true));
-                        partitionsToRecover.add(partition.getPartitionId());
-                    }
-                }
-
-                //Request remaining indexes files
-                replicationManager.requestReplicaFiles(replicaId, partitionsToRecover, existingFiles);
-            }
-        } catch (IOException e) {
-            /*
-             * in case of failure during failback completion process we need to construct a new plan
-             * and get all the files from the start since the remote replicas will change in the new plan.
-             */
-            LOGGER.warn("Failed during completing failback. Restarting failback process...", e);
-            startFailbackProcess();
-        }
-
-        //get max LSN from selected remote replicas
-        long maxRemoteLSN = replicationManager.getMaxRemoteLSN(failbackRecoveryReplicas.keySet());
-
-        //6. force LogManager to start from a partition > maxLSN in selected remote replicas
-        logManager.renewLogFilesAndStartFromLSN(maxRemoteLSN);
-
-        //start replication service after failback completed
-        runtimeContext.getReplicationChannel().start();
-        runtimeContext.getReplicationManager().startReplicationThreads();
-
-        failbackRecoveryReplicas = null;
-    }
-
-    //TODO refactor common code between remote recovery and failback process
-    @Override
-    public void doRemoteRecoveryPlan(Map<String, Set<Integer>> recoveryPlan) throws HyracksDataException {
-        int maxRecoveryAttempts = replicationProperties.getMaxRemoteRecoveryAttempts();
-        PersistentLocalResourceRepository resourceRepository = (PersistentLocalResourceRepository) runtimeContext
-                .getLocalResourceRepository();
-        IDatasetLifecycleManager datasetLifeCycleManager = runtimeContext.getDatasetLifecycleManager();
-        ILogManager logManager = runtimeContext.getTransactionSubsystem().getLogManager();
-        while (true) {
-            //start recovery steps
-            try {
-                if (maxRecoveryAttempts <= 0) {
-                    //to avoid infinite loop in case of unexpected behavior.
-                    throw new IllegalStateException("Failed to perform remote recovery.");
-                }
-
-                /*** Prepare for Recovery ***/
-                //1. clean any memory data that could've existed from previous failed recovery attempt
-                datasetLifeCycleManager.closeAllDatasets();
-
-                //2. remove any existing storage data and initialize storage metadata
-                resourceRepository.deleteStorageData();
-
-                /*** Start Recovery Per Lost Replica ***/
-                for (Entry<String, Set<Integer>> remoteReplica : recoveryPlan.entrySet()) {
-                    String replicaId = remoteReplica.getKey();
-                    Set<Integer> partitionsToRecover = remoteReplica.getValue();
-
-                    //Request indexes metadata and LSM components
-                    replicationManager.requestReplicaFiles(replicaId, partitionsToRecover, new HashSet<String>());
-                }
-
-                //get max LSN from selected remote replicas
-                long maxRemoteLSN = replicationManager.getMaxRemoteLSN(recoveryPlan.keySet());
-
-                //6. force LogManager to start from a partition > maxLSN in selected remote replicas
-                logManager.renewLogFilesAndStartFromLSN(maxRemoteLSN);
-                break;
-            } catch (IOException e) {
-                LOGGER.warn("Failed during remote recovery. Attempting again...", e);
-                maxRecoveryAttempts--;
-            }
-        }
-    }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentLSNSyncTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentLSNSyncTask.java
deleted file mode 100644
index 08c0ec7..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentLSNSyncTask.java
+++ /dev/null
@@ -1,39 +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.replication.storage;
-
-public class LSMComponentLSNSyncTask {
-
-    private String componentFilePath;
-    private String componentId;
-
-    public LSMComponentLSNSyncTask(String componentId, String componentFilePath) {
-        this.componentId = componentId;
-        this.componentFilePath = componentFilePath;
-    }
-
-    public String getComponentFilePath() {
-        return componentFilePath;
-    }
-
-    public String getComponentId() {
-        return componentId;
-    }
-
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java
deleted file mode 100644
index bf987d0..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java
+++ /dev/null
@@ -1,159 +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.replication.storage;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.file.Paths;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.storage.ResourceReference;
-import org.apache.asterix.replication.logging.TxnLogUtil;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
-import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
-
-public class LSMComponentProperties {
-
-    private AtomicInteger numberOfFiles;
-    private String componentId;
-    private long lsnOffset;
-    private long originalLSN;
-    private String nodeId;
-    private Long replicaLSN;
-    private String maskPath = null;
-    private String replicaPath = null;
-    private LSMOperationType opType;
-
-    public LSMComponentProperties(ILSMIndexReplicationJob job, String nodeId) {
-        this.nodeId = nodeId;
-        componentId = LSMComponentProperties.getLSMComponentID((String) job.getJobFiles().toArray()[0]);
-        numberOfFiles = new AtomicInteger(job.getJobFiles().size());
-        opType = job.getLSMOpType();
-        originalLSN = opType == LSMOperationType.FLUSH ?
-                LSMComponentProperties.getLSMComponentLSN((AbstractLSMIndex) job.getLSMIndex(),
-                        job.getLSMIndexOperationContext()) : 0;
-    }
-
-    public LSMComponentProperties() {
-    }
-
-    public static long getLSMComponentLSN(AbstractLSMIndex lsmIndex, ILSMIndexOperationContext ctx) {
-        long componentLSN = -1;
-        try {
-            componentLSN = ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
-                    .getComponentLSN(ctx.getComponentsToBeReplicated());
-        } catch (HyracksDataException e) {
-            e.printStackTrace();
-        }
-        if (componentLSN < 0) {
-            componentLSN = 0;
-        }
-        return componentLSN;
-    }
-
-    public void serialize(OutputStream out) throws IOException {
-        DataOutputStream dos = new DataOutputStream(out);
-        dos.writeUTF(componentId);
-        dos.writeUTF(nodeId);
-        dos.writeInt(numberOfFiles.get());
-        dos.writeLong(originalLSN);
-        dos.writeLong(lsnOffset);
-        dos.writeInt(opType.ordinal());
-    }
-
-    public static LSMComponentProperties create(DataInput input) throws IOException {
-        LSMComponentProperties lsmCompProp = new LSMComponentProperties();
-        lsmCompProp.componentId = input.readUTF();
-        lsmCompProp.nodeId = input.readUTF();
-        lsmCompProp.numberOfFiles = new AtomicInteger(input.readInt());
-        lsmCompProp.originalLSN = input.readLong();
-        lsmCompProp.lsnOffset = input.readLong();
-        lsmCompProp.opType = LSMOperationType.values()[input.readInt()];
-        return lsmCompProp;
-    }
-
-    public String getMaskPath(ReplicaResourcesManager resourceManager) throws HyracksDataException {
-        if (maskPath == null) {
-            LSMIndexFileProperties afp = new LSMIndexFileProperties(this);
-            maskPath = getReplicaComponentPath(resourceManager) + File.separator + afp.getFileName()
-                    + ReplicaResourcesManager.LSM_COMPONENT_MASK_SUFFIX;
-        }
-        return maskPath;
-    }
-
-    public String getReplicaComponentPath(ReplicaResourcesManager resourceManager) throws HyracksDataException {
-        if (replicaPath == null) {
-            LSMIndexFileProperties afp = new LSMIndexFileProperties(this);
-            replicaPath = resourceManager.getIndexPath(afp);
-        }
-        return replicaPath;
-    }
-
-    /***
-     * @param filePath
-     *            any file of the LSM component
-     * @return a unique id based on the timestamp of the component
-     */
-    public static String getLSMComponentID(String filePath) {
-        final ResourceReference ref = ResourceReference.of(filePath);
-        final String fileUniqueTimestamp =
-                ref.getName().substring(0, ref.getName().lastIndexOf(AbstractLSMIndexFileManager.DELIMITER));
-        return Paths.get(ref.getRelativePath().toString(), fileUniqueTimestamp).toString();
-    }
-
-    public String getComponentId() {
-        return componentId;
-    }
-
-    public long getOriginalLSN() {
-        return originalLSN;
-    }
-
-    public String getNodeId() {
-        return nodeId;
-    }
-
-    public int markFileComplete() {
-        return numberOfFiles.decrementAndGet();
-    }
-
-    public Long getReplicaLSN() {
-        return replicaLSN;
-    }
-
-    public void setReplicaLSN(Long replicaLSN) {
-        this.replicaLSN = replicaLSN;
-    }
-
-    public LSMOperationType getOpType() {
-        return opType;
-    }
-
-    public String getNodeUniqueLSN() {
-        return TxnLogUtil.getNodeUniqueLSN(nodeId, originalLSN);
-    }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java
deleted file mode 100644
index 2ebf2cb..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java
+++ /dev/null
@@ -1,109 +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.replication.storage;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.file.Paths;
-
-public class LSMIndexFileProperties {
-
-    private long fileSize;
-    private String nodeId;
-    private boolean lsmComponentFile;
-    private String filePath;
-    private boolean requiresAck = false;
-
-    public LSMIndexFileProperties() {
-    }
-
-    public LSMIndexFileProperties(String filePath, long fileSize, String nodeId, boolean lsmComponentFile,
-            boolean requiresAck) {
-        initialize(filePath, fileSize, nodeId, lsmComponentFile, requiresAck);
-    }
-
-    public LSMIndexFileProperties(LSMComponentProperties lsmComponentProperties) {
-        initialize(lsmComponentProperties.getComponentId(), -1, lsmComponentProperties.getNodeId(), false, false);
-    }
-
-    public void initialize(String filePath, long fileSize, String nodeId, boolean lsmComponentFile,
-            boolean requiresAck) {
-        this.filePath = filePath;
-        this.fileSize = fileSize;
-        this.nodeId = nodeId;
-        this.lsmComponentFile = lsmComponentFile;
-        this.requiresAck = requiresAck;
-    }
-
-    public void serialize(OutputStream out) throws IOException {
-        DataOutputStream dos = new DataOutputStream(out);
-        dos.writeUTF(nodeId);
-        dos.writeUTF(filePath);
-        dos.writeLong(fileSize);
-        dos.writeBoolean(lsmComponentFile);
-        dos.writeBoolean(requiresAck);
-    }
-
-    public static LSMIndexFileProperties create(DataInput input) throws IOException {
-        String nodeId = input.readUTF();
-        String filePath = input.readUTF();
-        long fileSize = input.readLong();
-        boolean lsmComponentFile = input.readBoolean();
-        boolean requiresAck = input.readBoolean();
-        LSMIndexFileProperties fileProp =
-                new LSMIndexFileProperties(filePath, fileSize, nodeId, lsmComponentFile, requiresAck);
-        return fileProp;
-    }
-
-    public String getFilePath() {
-        return filePath;
-    }
-
-    public long getFileSize() {
-        return fileSize;
-    }
-
-    public String getNodeId() {
-        return nodeId;
-    }
-
-    public boolean isLSMComponentFile() {
-        return lsmComponentFile;
-    }
-
-    public boolean requiresAck() {
-        return requiresAck;
-    }
-
-    public String getFileName() {
-        return Paths.get(filePath).toFile().getName();
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("File Path: " + filePath + "  ");
-        sb.append("File Size: " + fileSize + "  ");
-        sb.append("Node ID: " + nodeId + "  ");
-        sb.append("isLSMComponentFile : " + lsmComponentFile + "  ");
-        return sb.toString();
-    }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java
deleted file mode 100644
index 398f97d..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java
+++ /dev/null
@@ -1,226 +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.replication.storage;
-
-import java.io.File;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.config.MetadataProperties;
-import org.apache.asterix.common.dataflow.DatasetLocalResource;
-import org.apache.asterix.common.replication.IReplicaResourcesManager;
-import org.apache.asterix.common.storage.DatasetResourceReference;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.asterix.common.utils.StorageConstants;
-import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
-import org.apache.commons.io.FileUtils;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.storage.common.ILocalResourceRepository;
-import org.apache.hyracks.storage.common.LocalResource;
-
-public class ReplicaResourcesManager implements IReplicaResourcesManager {
-    public static final String LSM_COMPONENT_MASK_SUFFIX = "_mask";
-    private final PersistentLocalResourceRepository localRepository;
-    private final Map<String, ClusterPartition[]> nodePartitions;
-    private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
-
-    public ReplicaResourcesManager(ILocalResourceRepository localRepository, MetadataProperties metadataProperties,
-            IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
-        this.localRepository = (PersistentLocalResourceRepository) localRepository;
-        this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
-        nodePartitions = metadataProperties.getNodePartitions();
-    }
-
-    public void deleteIndexFile(LSMIndexFileProperties afp) throws HyracksDataException {
-        String indexPath = getIndexPath(afp);
-        if (indexPath != null) {
-            if (afp.isLSMComponentFile()) {
-                //delete index file
-                String indexFilePath = indexPath + File.separator + afp.getFileName();
-                File destFile = new File(indexFilePath);
-                FileUtils.deleteQuietly(destFile);
-            } else {
-                //delete index directory
-                FileUtils.deleteQuietly(new File(indexPath));
-            }
-        }
-    }
-
-    public String getIndexPath(LSMIndexFileProperties fileProperties) throws HyracksDataException {
-        final FileReference indexPath = localRepository.getIndexPath(Paths.get(fileProperties.getFilePath()));
-        if (!indexPath.getFile().exists()) {
-            indexPath.getFile().mkdirs();
-        }
-        return indexPath.toString();
-    }
-
-    public void createRemoteLSMComponentMask(LSMComponentProperties lsmComponentProperties) throws IOException {
-        String maskPath = lsmComponentProperties.getMaskPath(this);
-        Path path = Paths.get(maskPath);
-        if (!Files.exists(path)) {
-            File maskFile = new File(maskPath);
-            maskFile.createNewFile();
-        }
-    }
-
-    public void markLSMComponentReplicaAsValid(LSMComponentProperties lsmComponentProperties) throws IOException {
-        //remove mask to mark component as valid
-        String maskPath = lsmComponentProperties.getMaskPath(this);
-        Path path = Paths.get(maskPath);
-        Files.deleteIfExists(path);
-    }
-
-    public Set<File> getReplicaIndexes(String replicaId) throws HyracksDataException {
-        Set<File> remoteIndexesPaths = new HashSet<File>();
-        ClusterPartition[] partitions = nodePartitions.get(replicaId);
-        for (ClusterPartition partition : partitions) {
-            remoteIndexesPaths.addAll(localRepository.getPartitionIndexes(partition.getPartitionId()));
-        }
-        return remoteIndexesPaths;
-    }
-
-    @Override
-    public long getPartitionsMinLSN(Set<Integer> partitions) throws HyracksDataException {
-        long minRemoteLSN = Long.MAX_VALUE;
-        for (Integer partition : partitions) {
-            final List<DatasetResourceReference> partitionResources = localRepository.getResources(resource -> {
-                DatasetLocalResource dsResource = (DatasetLocalResource) resource.getResource();
-                return dsResource.getPartition() == partition;
-            }).values().stream().map(DatasetResourceReference::of).collect(Collectors.toList());
-            for (DatasetResourceReference indexRef : partitionResources) {
-                long remoteIndexMaxLSN = indexCheckpointManagerProvider.get(indexRef).getLowWatermark();
-                minRemoteLSN = Math.min(minRemoteLSN, remoteIndexMaxLSN);
-            }
-        }
-        return minRemoteLSN;
-    }
-
-    public Map<Long, DatasetResourceReference> getLaggingReplicaIndexesId2PathMap(String replicaId, long targetLSN)
-            throws HyracksDataException {
-        Map<Long, DatasetResourceReference> laggingReplicaIndexes = new HashMap<>();
-        final List<Integer> replicaPartitions =
-                Arrays.stream(nodePartitions.get(replicaId)).map(ClusterPartition::getPartitionId)
-                        .collect(Collectors.toList());
-        for (int patition : replicaPartitions) {
-            final Map<Long, LocalResource> partitionResources = localRepository.getPartitionResources(patition);
-            final List<DatasetResourceReference> indexesRefs =
-                    partitionResources.values().stream().map(DatasetResourceReference::of).collect(Collectors.toList());
-            for (DatasetResourceReference ref : indexesRefs) {
-                if (indexCheckpointManagerProvider.get(ref).getLowWatermark() < targetLSN) {
-                    laggingReplicaIndexes.put(ref.getResourceId(), ref);
-                }
-            }
-        }
-        return laggingReplicaIndexes;
-    }
-
-    public void cleanInvalidLSMComponents(String replicaId) {
-        //for every index in replica
-        Set<File> remoteIndexes = null;
-        try {
-            remoteIndexes = getReplicaIndexes(replicaId);
-        } catch (HyracksDataException e) {
-            throw new IllegalStateException(e);
-        }
-        for (File remoteIndexFile : remoteIndexes) {
-            //search for any mask
-            File[] masks = remoteIndexFile.listFiles(LSM_COMPONENTS_MASKS_FILTER);
-
-            for (File mask : masks) {
-                //delete all files belonging to this mask
-                deleteLSMComponentFilesForMask(mask);
-                //delete the mask itself
-                mask.delete();
-            }
-        }
-    }
-
-    private static void deleteLSMComponentFilesForMask(File maskFile) {
-        String lsmComponentTimeStamp = maskFile.getName().substring(0,
-                maskFile.getName().length() - LSM_COMPONENT_MASK_SUFFIX.length());
-        File indexFolder = maskFile.getParentFile();
-        File[] lsmComponentsFiles = indexFolder.listFiles(LSM_COMPONENTS_NON_MASKS_FILTER);
-        for (File lsmComponentFile : lsmComponentsFiles) {
-            if (lsmComponentFile.getName().contains(lsmComponentTimeStamp)) {
-                //match based on time stamp
-                lsmComponentFile.delete();
-            }
-        }
-    }
-
-    /**
-     * @param partition
-     * @return Absolute paths to all partition files
-     */
-    @Override
-    public List<String> getPartitionIndexesFiles(int partition, boolean relativePath) throws HyracksDataException {
-        List<String> partitionFiles = new ArrayList<String>();
-        Set<File> partitionIndexes = localRepository.getPartitionIndexes(partition);
-        for (File indexDir : partitionIndexes) {
-            if (indexDir.isDirectory()) {
-                File[] indexFiles = indexDir.listFiles(LSM_INDEX_FILES_FILTER);
-                if (indexFiles != null) {
-                    for (File file : indexFiles) {
-                        if (!relativePath) {
-                            partitionFiles.add(file.getAbsolutePath());
-                        } else {
-                            partitionFiles.add(StoragePathUtil.getIndexFileRelativePath(file.getAbsolutePath()));
-                        }
-                    }
-                }
-            }
-        }
-        return partitionFiles;
-    }
-
-    private static final FilenameFilter LSM_COMPONENTS_MASKS_FILTER = new FilenameFilter() {
-        @Override
-        public boolean accept(File dir, String name) {
-            return name.endsWith(LSM_COMPONENT_MASK_SUFFIX);
-        }
-    };
-
-    private static final FilenameFilter LSM_COMPONENTS_NON_MASKS_FILTER = new FilenameFilter() {
-        @Override
-        public boolean accept(File dir, String name) {
-            return !name.endsWith(LSM_COMPONENT_MASK_SUFFIX);
-        }
-    };
-
-    private static final FilenameFilter LSM_INDEX_FILES_FILTER = new FilenameFilter() {
-        @Override
-        public boolean accept(File dir, String name) {
-            return name.equalsIgnoreCase(StorageConstants.METADATA_FILE_NAME) || !name.startsWith(".");
-        }
-    };
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/FileSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/FileSynchronizer.java
similarity index 89%
rename from asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/FileSynchronizer.java
rename to asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/FileSynchronizer.java
index 8aa4487..e1649b3 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/FileSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/FileSynchronizer.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.replication.recovery;
+package org.apache.asterix.replication.sync;
 
 import java.io.IOException;
 import java.io.RandomAccessFile;
@@ -25,11 +25,11 @@
 
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.exceptions.ReplicationException;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
 import org.apache.asterix.replication.management.NetworkingUtil;
 import org.apache.asterix.replication.messaging.DeleteFileTask;
 import org.apache.asterix.replication.messaging.ReplicateFileTask;
-import org.apache.asterix.replication.storage.PartitionReplica;
+import org.apache.asterix.replication.api.PartitionReplica;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
 
@@ -44,11 +44,15 @@
     }
 
     public void replicate(String file) {
+        replicate(file, false);
+    }
+
+    public void replicate(String file, boolean metadata) {
         try {
             final IIOManager ioManager = appCtx.getIoManager();
             final SocketChannel channel = replica.getChannel();
             final FileReference filePath = ioManager.resolve(file);
-            ReplicateFileTask task = new ReplicateFileTask(file, filePath.getFile().length());
+            ReplicateFileTask task = new ReplicateFileTask(file, filePath.getFile().length(), metadata);
             ReplicationProtocol.sendTo(replica, task);
             // send the file itself
             try (RandomAccessFile fromFile = new RandomAccessFile(filePath.getFile(),
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
new file mode 100644
index 0000000..74f38e2
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
@@ -0,0 +1,140 @@
+/*
+ * 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.replication.sync;
+
+import static org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation.DELETE;
+import static org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation.REPLICATE;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.common.utils.StoragePathUtil;
+import org.apache.asterix.replication.api.PartitionReplica;
+import org.apache.asterix.replication.messaging.ComponentMaskTask;
+import org.apache.asterix.replication.messaging.DropIndexTask;
+import org.apache.asterix.replication.messaging.MarkComponentValidTask;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.replication.IReplicationJob;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
+import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class IndexSynchronizer {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final IReplicationJob job;
+    private final INcApplicationContext appCtx;
+
+    public IndexSynchronizer(IReplicationJob job, INcApplicationContext appCtx) {
+        this.job = job;
+        this.appCtx = appCtx;
+    }
+
+    public void sync(PartitionReplica replica) throws IOException {
+        switch (job.getJobType()) {
+            case LSM_COMPONENT:
+                syncComponent(replica);
+                break;
+            case METADATA:
+                syncMetadata(replica);
+                break;
+            default:
+                throw new IllegalStateException("unrecognized job type: " + job.getJobType().name());
+        }
+    }
+
+    private void syncComponent(PartitionReplica replica) throws IOException {
+        if (job.getOperation() == REPLICATE) {
+            replicateComponent(replica);
+        } else if (job.getOperation() == DELETE) {
+            deleteComponent(replica);
+        }
+    }
+
+    private void syncMetadata(PartitionReplica replica) throws IOException {
+        if (job.getOperation() == REPLICATE) {
+            replicateIndexMetadata(replica);
+        } else if (job.getOperation() == DELETE) {
+            deleteIndexMetadata(replica);
+        }
+    }
+
+    private void replicateComponent(PartitionReplica replica) throws IOException {
+        // send component header
+        final String anyFile = job.getAnyFile();
+        final String lsmComponentID = getComponentId(anyFile);
+        final String indexFile = StoragePathUtil.getFileRelativePath(anyFile);
+        final ComponentMaskTask maskTask = new ComponentMaskTask(indexFile, lsmComponentID);
+        ReplicationProtocol.sendTo(replica, maskTask);
+        ReplicationProtocol.waitForAck(replica);
+        // send component files
+        final FileSynchronizer fileSynchronizer = new FileSynchronizer(appCtx, replica);
+        job.getJobFiles().stream().map(StoragePathUtil::getFileRelativePath).forEach(fileSynchronizer::replicate);
+        // send mark component valid
+        MarkComponentValidTask markValidTask = new MarkComponentValidTask(indexFile, getReplicatedComponentLsn());
+        ReplicationProtocol.sendTo(replica, markValidTask);
+        ReplicationProtocol.waitForAck(replica);
+        LOGGER.debug("Replicated component ({}) to replica {}", indexFile, replica);
+    }
+
+    private void deleteComponent(PartitionReplica replica) {
+        FileSynchronizer fileSynchronizer = new FileSynchronizer(appCtx, replica);
+        job.getJobFiles().stream().map(StoragePathUtil::getFileRelativePath).forEach(fileSynchronizer::delete);
+    }
+
+    private void replicateIndexMetadata(PartitionReplica replica) {
+        // send the index metadata file
+        final FileSynchronizer fileSynchronizer = new FileSynchronizer(appCtx, replica);
+        job.getJobFiles().stream().map(StoragePathUtil::getFileRelativePath)
+                .forEach(file -> fileSynchronizer.replicate(file, true));
+    }
+
+    private void deleteIndexMetadata(PartitionReplica replica) throws IOException {
+        final String file = StoragePathUtil.getFileRelativePath(job.getAnyFile());
+        final DropIndexTask task = new DropIndexTask(file);
+        ReplicationProtocol.sendTo(replica, task);
+        ReplicationProtocol.waitForAck(replica);
+    }
+
+    private long getReplicatedComponentLsn() throws HyracksDataException {
+        final ILSMIndexReplicationJob indexReplJob = (ILSMIndexReplicationJob) job;
+        if (indexReplJob.getLSMOpType() != LSMOperationType.FLUSH) {
+            return AbstractLSMIOOperationCallback.INVALID;
+        }
+        final ILSMIndex lsmIndex = indexReplJob.getLSMIndex();
+        final ILSMIndexOperationContext ctx = indexReplJob.getLSMIndexOperationContext();
+        return ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
+                .getComponentLSN(ctx.getComponentsToBeReplicated());
+    }
+
+    private static String getComponentId(String filePath) {
+        final ResourceReference ref = ResourceReference.of(filePath);
+        final String fileUniqueTimestamp =
+                ref.getName().substring(0, ref.getName().lastIndexOf(AbstractLSMIndexFileManager.DELIMITER));
+        return Paths.get(ref.getRelativePath().toString(), fileUniqueTimestamp).toString();
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/ReplicaFilesSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaFilesSynchronizer.java
similarity index 82%
rename from asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/ReplicaFilesSynchronizer.java
rename to asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaFilesSynchronizer.java
index 2021cee..5658779 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/ReplicaFilesSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaFilesSynchronizer.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.replication.recovery;
+package org.apache.asterix.replication.sync;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -28,10 +28,11 @@
 
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
+import org.apache.asterix.replication.api.PartitionReplica;
 import org.apache.asterix.replication.messaging.PartitionResourcesListResponse;
 import org.apache.asterix.replication.messaging.PartitionResourcesListTask;
-import org.apache.asterix.replication.storage.PartitionReplica;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 
 /**
  * Ensures that the files between master and a replica are synchronized
@@ -49,9 +50,10 @@
     public void sync() throws IOException {
         final int partition = replica.getIdentifier().getPartition();
         final Set<String> replicaFiles = getReplicaFiles(partition);
-        final Set<String> masterFiles =
-                appCtx.getReplicaResourcesManager().getPartitionIndexesFiles(partition, false).stream()
-                        .map(StoragePathUtil::getFileRelativePath).collect(Collectors.toSet());
+        final PersistentLocalResourceRepository localResourceRepository =
+                (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+        final Set<String> masterFiles = localResourceRepository.getPartitionIndexesFiles(partition).stream()
+                .map(StoragePathUtil::getFileRelativePath).collect(Collectors.toSet());
         // find files on master and not on replica
         final List<String> replicaMissingFiles =
                 masterFiles.stream().filter(file -> !replicaFiles.contains(file)).collect(Collectors.toList());
@@ -65,7 +67,7 @@
     private Set<String> getReplicaFiles(int partition) throws IOException {
         final PartitionResourcesListTask replicaFilesRequest = new PartitionResourcesListTask(partition);
         final SocketChannel channel = replica.getChannel();
-        final ByteBuffer reusableBuffer = replica.gerReusableBuffer();
+        final ByteBuffer reusableBuffer = replica.getReusableBuffer();
         ReplicationProtocol.sendTo(replica, replicaFilesRequest);
         final PartitionResourcesListResponse response =
                 (PartitionResourcesListResponse) ReplicationProtocol.read(channel, reusableBuffer);
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/ReplicaSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
similarity index 90%
rename from asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/ReplicaSynchronizer.java
rename to asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
index 1fa3246..9f397d2 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/ReplicaSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
@@ -16,16 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.replication.recovery;
+package org.apache.asterix.replication.sync;
 
 import java.io.IOException;
 
 import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.config.ReplicationProperties;
 import org.apache.asterix.common.replication.IReplicationStrategy;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
 import org.apache.asterix.replication.messaging.CheckpointPartitionIndexesTask;
-import org.apache.asterix.replication.storage.PartitionReplica;
+import org.apache.asterix.replication.api.PartitionReplica;
 
 /**
  * Performs the steps required to ensure any newly added replica
diff --git a/asterixdb/asterix-replication/src/test/resources/data/fbu.adm b/asterixdb/asterix-replication/src/test/resources/data/fbu.adm
deleted file mode 100644
index 7e99ea4..0000000
--- a/asterixdb/asterix-replication/src/test/resources/data/fbu.adm
+++ /dev/null
@@ -1,10 +0,0 @@
-{"id":1,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]}
-{"id":2,"alias":"Isbel","name":"IsbelDull","user-since":datetime("2011-01-22T10:10:00"),"friend-ids":{{1,4}},"employment":[{"organization-name":"Hexviafind","start-date":date("2010-04-27")}]}
-{"id":3,"alias":"Emory","name":"EmoryUnk","user-since":datetime("2012-07-10T10:10:00"),"friend-ids":{{1,5,8,9}},"employment":[{"organization-name":"geomedia","start-date":date("2010-06-17"),"end-date":date("2010-01-26")}]}
-{"id":4,"alias":"Nicholas","name":"NicholasStroh","user-since":datetime("2010-12-27T10:10:00"),"friend-ids":{{2}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2010-06-08")}]}
-{"id":5,"alias":"Von","name":"VonKemble","user-since":datetime("2010-01-05T10:10:00"),"friend-ids":{{3,6,10}},"employment":[{"organization-name":"Kongreen","start-date":date("2010-11-27")}]}
-{"id":6,"alias":"Willis","name":"WillisWynne","user-since":datetime("2005-01-17T10:10:00"),"friend-ids":{{1,3,7}},"employment":[{"organization-name":"jaydax","start-date":date("2009-05-15")}]}
-{"id":7,"alias":"Suzanna","name":"SuzannaTillson","user-since":datetime("2012-08-07T10:10:00"),"friend-ids":{{6}},"employment":[{"organization-name":"Labzatron","start-date":date("2011-04-19")}]}
-{"id":8,"alias":"Nila","name":"NilaMilliron","user-since":datetime("2008-01-01T10:10:00"),"friend-ids":{{3}},"employment":[{"organization-name":"Plexlane","start-date":date("2010-02-28")}]}
-{"id":9,"alias":"Woodrow","name":"WoodrowNehling","user-since":datetime("2005-09-20T10:10:00"),"friend-ids":{{3,10}},"employment":[{"organization-name":"Zuncan","start-date":date("2003-04-22"),"end-date":date("2009-12-13")}]}
-{"id":10,"alias":"Bram","name":"BramHatch","user-since":datetime("2010-10-16T10:10:00"),"friend-ids":{{1,5,9}},"employment":[{"organization-name":"physcane","start-date":date("2007-06-05"),"end-date":date("2011-11-05")}]}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReplicaEventMessage.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReplicaEventMessage.java
deleted file mode 100644
index 8ae1bd7..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReplicaEventMessage.java
+++ /dev/null
@@ -1,65 +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.runtime.message;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.messaging.api.INcAddressedMessage;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.replication.ReplicaEvent;
-import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class ReplicaEventMessage implements INcAddressedMessage {
-
-    private static final long serialVersionUID = 1L;
-    private final String nodeId;
-    private final ClusterEventType event;
-    private final String nodeIPAddress;
-    private final int nodePort;
-
-    public ReplicaEventMessage(String nodeId, String nodeIPAddress, int nodePort, ClusterEventType event) {
-        this.nodeId = nodeId;
-        this.nodeIPAddress = nodeIPAddress;
-        this.nodePort = nodePort;
-        this.event = event;
-    }
-
-    public String getNodeId() {
-        return nodeId;
-    }
-
-    public ClusterEventType getEvent() {
-        return event;
-    }
-
-    public String getNodeIPAddress() {
-        return nodeIPAddress;
-    }
-
-    @Override
-    public void handle(INcApplicationContext appContext) throws HyracksDataException, InterruptedException {
-        Replica replica = new Replica(nodeId, nodeIPAddress, nodePort);
-        appContext.getReplicationManager().reportReplicaEvent(new ReplicaEvent(replica, event));
-    }
-
-    @Override
-    public String toString() {
-        return ReplicaEventMessage.class.getSimpleName();
-    }
-}
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/MetadataReplicationIT.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/MetadataReplicationIT.java
deleted file mode 100644
index 5ae6843..0000000
--- a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/MetadataReplicationIT.java
+++ /dev/null
@@ -1,168 +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.server.test;
-
-import java.io.File;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.asterix.test.base.RetainLogsRule;
-import org.apache.asterix.test.common.TestExecutor;
-import org.apache.asterix.testframework.context.TestCaseContext;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hyracks.server.process.HyracksCCProcess;
-import org.apache.hyracks.server.process.HyracksNCServiceProcess;
-import org.apache.hyracks.server.process.HyracksVirtualCluster;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestRule;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class MetadataReplicationIT {
-
-    // Important paths and files for this test.
-
-    // The "target" subdirectory of asterix-server. All outputs go here.
-    public static final String TARGET_DIR = StringUtils.join(new String[] { "../asterix-server/target" },
-            File.separator);
-
-    // Directory where the NCs create and store all data, as configured by
-    // src/test/resources/NCServiceExecutionIT/cc.conf.
-    public static final String INSTANCE_DIR = StringUtils.join(new String[] { TARGET_DIR, "tmp" }, File.separator);
-
-    // The log directory, where all CC, NCService, and NC logs are written. CC and
-    // NCService logs are configured on the HyracksVirtualCluster below. NC logs
-    // are configured in src/test/resources/NCServiceExecutionIT/ncservice*.conf.
-    public static final String LOG_DIR = StringUtils.join(new String[] { TARGET_DIR, "failsafe-reports" },
-            File.separator);
-
-    // Directory where *.conf files are located.
-    public static final String CONF_DIR = StringUtils
-            .join(new String[] { TARGET_DIR, "test-classes", "MetadataReplicationIT" }, File.separator);
-
-    // The app.home specified for HyracksVirtualCluster. The NCService expects
-    // to find the NC startup script in ${app.home}/bin.
-    public static final String APP_HOME = StringUtils.join(new String[] { TARGET_DIR, "appassembler" }, File.separator);
-
-    // Path to the asterix-app directory. This is used as the current working
-    // directory for the CC and NCService processes, which allows relative file
-    // paths in "load" statements in test queries to find the right data. It is
-    // also used for HDFSCluster.
-    public static final String ASTERIX_APP_DIR = StringUtils.join(new String[] { "..", "asterix-app" }, File.separator);
-
-    // Path to the actual AQL test files, which we borrow from asterix-app. This is
-    // passed to TestExecutor.
-    protected static final String TESTS_DIR = StringUtils
-            .join(new String[] { ASTERIX_APP_DIR, "src", "test", "resources", "runtimets" }, File.separator);
-
-    // Path that actual results are written to. We create and clean this directory
-    // here, and also pass it to TestExecutor which writes the test output there.
-    public static final String ACTUAL_RESULTS_DIR = StringUtils.join(new String[] { TARGET_DIR, "ittest" },
-            File.separator);
-    private static final String PATH_BASE = Paths
-            .get("src", "test", "resources", "integrationts", "metadata_only_replication").toString() + File.separator;
-    private static final String PATH_ACTUAL = "target" + File.separator + "ittest" + File.separator;
-
-    private static final Logger LOGGER = LogManager.getLogger();
-    private static String reportPath = new File(
-            StringUtils.join(new String[] { "target", "failsafe-reports" }, File.separator)).getAbsolutePath();
-
-    private final TestExecutor testExecutor = new TestExecutor();
-    private TestCaseContext tcCtx;
-    private static String scriptHomePath;
-    private static File asterixInstallerPath;
-    private static ProcessBuilder pb;
-    private static Map<String, String> env;
-
-    private static HyracksCCProcess cc;
-    private static HyracksNCServiceProcess nc1;
-    private static HyracksNCServiceProcess nc2;
-    private static HyracksVirtualCluster cluster;
-
-    public MetadataReplicationIT(TestCaseContext tcCtx) {
-        this.tcCtx = tcCtx;
-    }
-
-    @Rule
-    public TestRule retainLogs = new RetainLogsRule(NCServiceExecutionIT.ASTERIX_APP_DIR, reportPath, this);
-
-    @Before
-    public void before() throws Exception {
-        LOGGER.info("Creating new instance...");
-        File instanceDir = new File(INSTANCE_DIR);
-        if (instanceDir.isDirectory()) {
-            FileUtils.deleteDirectory(instanceDir);
-        }
-
-        // HDFSCluster requires the input directory to end with a file separator.
-
-        cluster = new HyracksVirtualCluster(new File(APP_HOME), new File(ASTERIX_APP_DIR));
-        nc1 = cluster.addNCService(new File(CONF_DIR, "ncservice1.conf"), new File(LOG_DIR, "ncservice1.log"));
-
-        nc2 = cluster.addNCService(new File(CONF_DIR, "ncservice2.conf"), new File(LOG_DIR, "ncservice2.log"));
-
-        // Start CC
-        cc = cluster.start(new File(CONF_DIR, "cc.conf"), new File(LOG_DIR, "cc.log"));
-
-        LOGGER.info("Instance created.");
-        testExecutor.waitForClusterActive(30, TimeUnit.SECONDS);
-        LOGGER.info("Instance is in ACTIVE state.");
-
-    }
-
-    @After
-    public void after() throws Exception {
-        LOGGER.info("Destroying instance...");
-        cluster.stop();
-        LOGGER.info("Instance destroyed.");
-    }
-
-    @Test
-    public void test() throws Exception {
-        testExecutor.executeTest(PATH_ACTUAL, tcCtx, null, false);
-    }
-
-    @Parameterized.Parameters(name = "MetadataReplicationIT {index}: {0}")
-    public static Collection<Object[]> tests() throws Exception {
-        Collection<Object[]> testArgs = buildTestsInXml(TestCaseContext.DEFAULT_TESTSUITE_XML_NAME);
-        if (testArgs.size() == 0) {
-            testArgs = buildTestsInXml(TestCaseContext.DEFAULT_TESTSUITE_XML_NAME);
-        }
-        return testArgs;
-    }
-
-    protected static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
-        Collection<Object[]> testArgs = new ArrayList<>();
-        TestCaseContext.Builder b = new TestCaseContext.Builder();
-        for (TestCaseContext ctx : b.build(new File(PATH_BASE), xmlfile)) {
-            testArgs.add(new Object[] { ctx });
-        }
-        return testArgs;
-    }
-}
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/ReplicationIT.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/ReplicationIT.java
index aa2d97d..289dbf2 100644
--- a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/ReplicationIT.java
+++ b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/ReplicationIT.java
@@ -18,9 +18,19 @@
  */
 package org.apache.asterix.server.test;
 
+import static org.apache.asterix.server.test.NCServiceExecutionIT.APP_HOME;
+import static org.apache.asterix.server.test.NCServiceExecutionIT.ASTERIX_APP_DIR;
+import static org.apache.asterix.server.test.NCServiceExecutionIT.INSTANCE_DIR;
+import static org.apache.asterix.server.test.NCServiceExecutionIT.LOG_DIR;
+import static org.apache.asterix.server.test.NCServiceExecutionIT.TARGET_DIR;
+
 import java.io.File;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.asterix.test.base.RetainLogsRule;
@@ -28,8 +38,6 @@
 import org.apache.asterix.testframework.context.TestCaseContext;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.hyracks.server.process.HyracksCCProcess;
-import org.apache.hyracks.server.process.HyracksNCServiceProcess;
 import org.apache.hyracks.server.process.HyracksVirtualCluster;
 import org.apache.hyracks.util.file.FileUtil;
 import org.apache.logging.log4j.LogManager;
@@ -42,38 +50,38 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import static org.apache.asterix.server.test.NCServiceExecutionIT.APP_HOME;
-import static org.apache.asterix.server.test.NCServiceExecutionIT.INSTANCE_DIR;
-import static org.apache.asterix.server.test.NCServiceExecutionIT.TARGET_DIR;
-import static org.apache.asterix.server.test.NCServiceExecutionIT.ASTERIX_APP_DIR;
-import static org.apache.asterix.server.test.NCServiceExecutionIT.LOG_DIR;
-
 @RunWith(Parameterized.class)
 public class ReplicationIT {
 
-    private static final String PATH_BASE = FileUtil.joinPath("src", "test", "resources", "integrationts",
-            "replication");
-    public static final String CONF_DIR = StringUtils.join(new String[] { TARGET_DIR, "test-classes", "ReplicationIT" },
-            File.separator);
+    private static final String PATH_BASE =
+            FileUtil.joinPath("src", "test", "resources", "integrationts", "replication");
+    private static final String CONF_DIR =
+            StringUtils.join(new String[] { TARGET_DIR, "test-classes", "ReplicationIT" }, File.separator);
     private static final String PATH_ACTUAL = FileUtil.joinPath("target", "ittest");
     private static final Logger LOGGER = LogManager.getLogger();
     private static String reportPath = new File(FileUtil.joinPath("target", "failsafe-reports")).getAbsolutePath();
-
-    private final TestExecutor testExecutor = new TestExecutor();
-    private TestCaseContext tcCtx;
-    private static ProcessBuilder pb;
-
-    private static HyracksCCProcess cc;
-    private static HyracksNCServiceProcess nc1;
-    private static HyracksNCServiceProcess nc2;
+    private static final TestExecutor testExecutor = new TestExecutor();
     private static HyracksVirtualCluster cluster;
 
+    static {
+        final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+        final Map<String, InetSocketAddress> replicationAddress = new HashMap<>();
+        final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+        ncEndPoints.put("asterix_nc1", InetSocketAddress.createUnresolved(ip, 19004));
+        ncEndPoints.put("asterix_nc2", InetSocketAddress.createUnresolved(ip, 19005));
+        replicationAddress.put("asterix_nc1", InetSocketAddress.createUnresolved(ip, 2001));
+        replicationAddress.put("asterix_nc2", InetSocketAddress.createUnresolved(ip, 2002));
+        testExecutor.setNcEndPoints(ncEndPoints);
+        testExecutor.setNcReplicationAddress(replicationAddress);
+    }
+
+    private TestCaseContext tcCtx;
+
     public ReplicationIT(TestCaseContext tcCtx) {
         this.tcCtx = tcCtx;
     }
 
-    @Rule
-    public TestRule retainLogs = new RetainLogsRule(NCServiceExecutionIT.ASTERIX_APP_DIR, reportPath, this);
+    @Rule public TestRule retainLogs = new RetainLogsRule(NCServiceExecutionIT.ASTERIX_APP_DIR, reportPath, this);
 
     @Before
     public void before() throws Exception {
@@ -83,24 +91,19 @@
             FileUtils.deleteDirectory(instanceDir);
         }
 
-        // HDFSCluster requires the input directory to end with a file separator.
-
         cluster = new HyracksVirtualCluster(new File(APP_HOME), new File(ASTERIX_APP_DIR));
-        nc1 = cluster.addNCService(new File(CONF_DIR, "ncservice1.conf"), new File(LOG_DIR, "ncservice1.log"));
-
-        nc2 = cluster.addNCService(new File(CONF_DIR, "ncservice2.conf"), new File(LOG_DIR, "ncservice2.log"));
+        cluster.addNCService(new File(CONF_DIR, "ncservice1.conf"), new File(LOG_DIR, "ncservice1.log"));
+        cluster.addNCService(new File(CONF_DIR, "ncservice2.conf"), new File(LOG_DIR, "ncservice2.log"));
 
         // Start CC
-        cc = cluster.start(new File(CONF_DIR, "cc.conf"), new File(LOG_DIR, "cc.log"));
-
+        cluster.start(new File(CONF_DIR, "cc.conf"), new File(LOG_DIR, "cc.log"));
         LOGGER.info("Instance created.");
         testExecutor.waitForClusterActive(30, TimeUnit.SECONDS);
         LOGGER.info("Instance is in ACTIVE state.");
-
     }
 
     @After
-    public void after() throws Exception {
+    public void after() {
         LOGGER.info("Destroying instance...");
         cluster.stop();
         LOGGER.info("Instance destroyed.");
@@ -120,7 +123,7 @@
         return testArgs;
     }
 
-    protected static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
+    private static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
         Collection<Object[]> testArgs = new ArrayList<>();
         TestCaseContext.Builder b = new TestCaseContext.Builder();
         for (TestCaseContext ctx : b.build(new File(PATH_BASE), xmlfile)) {
diff --git a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/cc.conf b/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/cc.conf
deleted file mode 100644
index 32e38ae..0000000
--- a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/cc.conf
+++ /dev/null
@@ -1,52 +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.
-
-[nc/asterix_nc1]
-txn.log.dir=../asterix-server/target/tmp/asterix_nc1/txnlog
-core.dump.dir=../asterix-server/target/tmp/asterix_nc1/coredump
-iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice1,../asterix-server/target/tmp/asterix_nc1/iodevice2
-jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5006
-replication.listen.port=2001
-nc.api.port=19004
-
-[nc/asterix_nc2]
-ncservice.port=9091
-txn.log.dir=../asterix-server/target/tmp/asterix_nc2/txnlog
-core.dump.dir=../asterix-server/target/tmp/asterix_nc2/coredump
-iodevices=../asterix-server/target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
-jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5007
-replication.listen.port=2002
-nc.api.port=19005
-
-[nc]
-address=127.0.0.1
-command=asterixnc
-app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
-jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
-storage.subdir=test_storage
-storage.memorycomponent.globalbudget = 1073741824
-
-[cc]
-address = 127.0.0.1
-app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
-heartbeat.period=2000
-
-[common]
-log.level = INFO
-replication.enabled=true
-replication.factor=2
-replication.strategy=metadata_only
diff --git a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice1.conf b/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice1.conf
deleted file mode 100644
index ba10142..0000000
--- a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice1.conf
+++ /dev/null
@@ -1,20 +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.
-
-[ncservice]
-logdir=../asterix-server/target/failsafe-reports
-
diff --git a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice2.conf b/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice2.conf
deleted file mode 100644
index 2036584..0000000
--- a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice2.conf
+++ /dev/null
@@ -1,21 +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.
-
-[ncservice]
-logdir=../asterix-server/target/failsafe-reports
-port=9091
-
diff --git a/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf b/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf
index 776a89a..2b68dd2 100644
--- a/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf
+++ b/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf
@@ -48,4 +48,5 @@
 [common]
 log.level = INFO
 replication.enabled=true
+replication.strategy=all
 replication.factor=2
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.1.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.1.ddl.aql
deleted file mode 100644
index 725ed61..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.1.ddl.aql
+++ /dev/null
@@ -1,50 +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.
- */
-/*
- * Test case Name  : metadata_node_recovery.aql
- * Description     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-drop dataverse Social if exists;
-create dataverse Social;
-use dataverse Social;
-
-create type EmploymentType as open {
-    organization-name: string,
-    start-date: date,
-    end-date: date?
-}
-
-create type UserType as closed {
-    id: int,
-    alias: string,
-    name: string,
-    user-since: datetime,
-    friend-ids: {{ int32 }},
-    employment: [EmploymentType]
-}
-
-create dataset Users(UserType)
-primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.10.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.10.node.aql
deleted file mode 100644
index f4685df..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.10.node.aql
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * Test case Name  : metadata_node_recovery.aql
- * Description     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-kill asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.11.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.11.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.11.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.13.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.13.sleep.aql
deleted file mode 100644
index dd83b4b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.13.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-20000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.14.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.14.ddl.aql
deleted file mode 100644
index eee7423..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.14.ddl.aql
+++ /dev/null
@@ -1,33 +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.
- */
-/*
- * Test case Name  : metadata_node_recovery.aql
- * Description     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-use dataverse Social;
-
-create dataset UsersAfterReplicaRecovery(UserType)
-primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.15.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.15.node.aql
deleted file mode 100644
index 720f33e..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.15.node.aql
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * Test case Name  : metadata_node_recovery.aql
- * Description     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-kill asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.16.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.16.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.16.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.18.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.18.sleep.aql
deleted file mode 100644
index dd83b4b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.18.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-20000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.19.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.19.query.aql
deleted file mode 100644
index 5da6c59..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.19.query.aql
+++ /dev/null
@@ -1,35 +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.
- */
-/*
- * Test case Name  : metadata_node_recovery.aql
- * Description     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-count(
-for $x in dataset Metadata.Dataset
-where $x.DatasetName ='UsersAfterRecovery'
-or $x.DatasetName ='UsersAfterReplicaRecovery'
-return $x.DatasetName
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.2.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.2.node.aql
deleted file mode 100644
index 720f33e..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.2.node.aql
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * Test case Name  : metadata_node_recovery.aql
- * Description     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-kill asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.3.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.3.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.3.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.4.get.http b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.4.get.http
deleted file mode 100644
index fdacfd0..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.4.get.http
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * Test case Name  : metadata_node_recovery.aql
- * Description     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.5.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.5.node.aql
deleted file mode 100644
index b750c64..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.5.node.aql
+++ /dev/null
@@ -1,19 +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.
- */
-start asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.6.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.6.sleep.aql
deleted file mode 100644
index dd83b4b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.6.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-20000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.7.get.http b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.7.get.http
deleted file mode 100644
index fdacfd0..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.7.get.http
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * Test case Name  : metadata_node_recovery.aql
- * Description     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.8.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.8.query.aql
deleted file mode 100644
index 778e9f3..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.8.query.aql
+++ /dev/null
@@ -1,34 +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.
- */
-/*
- * Test case Name  : metadata_node_recovery.aql
- * Description     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-count(
-for $x in dataset Metadata.Dataset
-where $x.DatasetName ='Users'
-return $x.DatasetName
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.9.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.9.ddl.aql
deleted file mode 100644
index 376df70..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.9.ddl.aql
+++ /dev/null
@@ -1,33 +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.
- */
-/*
- * Test case Name  : metadata_node_recovery.aql
- * Description     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-use dataverse Social;
-
-create dataset UsersAfterRecovery(UserType)
-primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.19.adm b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.19.adm
deleted file mode 100644
index d8263ee..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.19.adm
+++ /dev/null
@@ -1 +0,0 @@
-2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.8.adm b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.8.adm
deleted file mode 100644
index 56a6051..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.8.adm
+++ /dev/null
@@ -1 +0,0 @@
-1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/testsuite.xml b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/testsuite.xml
deleted file mode 100644
index d0ac325..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/testsuite.xml
+++ /dev/null
@@ -1,27 +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.
- !-->
-<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries" QueryFileExtension=".aql">
-  <test-group name="metadata_recovery">
-    <test-case FilePath="metadata_recovery">
-      <compilation-unit name="metadata_node_recovery">
-        <output-dir compare="Text">metadata_node_recovery</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
-</test-suite>
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql
deleted file mode 100644
index 2c49a01..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql
+++ /dev/null
@@ -1,59 +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.
- */
-/*
- * Test case Name  : node_failback.aql
- * Description     : Make sure node failback completes as expected.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
-                     kill one node and wait until the failover complete, query cluster state,
-                     query data, insert new data, start the killed node and wait for failback,
-                     query cluster state, query data.
- * Expected Result : Success
- * Date            : February 3 2016
- */
-
-drop dataverse TinySocial if exists;
-create dataverse TinySocial;
-use dataverse TinySocial;
-
-create type EmploymentType as open {
-    organization-name: string,
-    start-date: date,
-    end-date: date?
-}
-
-create type FacebookUserType as closed {
-    id: int,
-    alias: string,
-    name: string,
-    user-since: datetime,
-    friend-ids: {{ int32 }},
-    employment: [EmploymentType]
-}
-
-/********* 2. Create Datasets  ***********/
-use dataverse TinySocial;
-
-drop dataset FacebookUsers if exists;
-
-create dataset FacebookUsers(FacebookUserType)
-primary key id;
-
-create dataset FacebookUsersInMemory(FacebookUserType)
-primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http
deleted file mode 100644
index 3faa945..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * Test case Name  : node_failback.aql
- * Description     : Make sure node failback completes as expected.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
-                     kill one node and wait until the failover complete, query cluster state,
-                     query data, insert new data, start the killed node and wait for failback,
-                     query cluster state, query data.
- * Expected Result : Success
- * Date            : February 3 2016
- */
-/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql
deleted file mode 100644
index b09c3d3..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql
+++ /dev/null
@@ -1,33 +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.
- */
-/*
- * Test case Name  : node_failback.aql
- * Description     : Make sure node failback completes as expected.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
-                     kill one node and wait until the failover complete, query cluster state,
-                     query data, insert new data, start the killed node and wait for failback,
-                     query cluster state, query data.
- * Expected Result : Success
- * Date            : February 3 2016
- */
-
-use dataverse TinySocial;
-
-count (for $x in dataset FacebookUsersInMemory return $x);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
deleted file mode 100644
index 377e097..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
+++ /dev/null
@@ -1,36 +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.
- */
-/*
- * Test case Name  : node_failback.aql
- * Description     : Make sure node failback completes as expected.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
-                     kill one node and wait until the failover complete, query cluster state,
-                     query data, insert new data, start the killed node and wait for failback,
-                     query cluster state, query data.
- * Expected Result : Success
- * Date            : February 3 2016
- */
-use dataverse TinySocial;
-
-load dataset FacebookUsers using localfs
-(("path"="asterix_nc1://../asterix-server/src/test/resources/integrationts/replication/data/fbu.adm"),
-("format"="adm"));
-
-insert into dataset TinySocial.FacebookUsersInMemory(for $x in dataset TinySocial.FacebookUsers return $x);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql
deleted file mode 100644
index 9d5dc9b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql
+++ /dev/null
@@ -1,28 +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.
- */
-/*
- * Test case Name  : bulkload.aql
- * Description     : Check that Bulkload LSM component are replicated correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, query data, kill one node
-                     and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-kill asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.4.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.4.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.4.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.get.http b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.get.http
deleted file mode 100644
index 3faa945..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.get.http
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * Test case Name  : node_failback.aql
- * Description     : Make sure node failback completes as expected.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
-                     kill one node and wait until the failover complete, query cluster state,
-                     query data, insert new data, start the killed node and wait for failback,
-                     query cluster state, query data.
- * Expected Result : Success
- * Date            : February 3 2016
- */
-/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.6.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.6.query.aql
deleted file mode 100644
index b09c3d3..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.6.query.aql
+++ /dev/null
@@ -1,33 +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.
- */
-/*
- * Test case Name  : node_failback.aql
- * Description     : Make sure node failback completes as expected.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
-                     kill one node and wait until the failover complete, query cluster state,
-                     query data, insert new data, start the killed node and wait for failback,
-                     query cluster state, query data.
- * Expected Result : Success
- * Date            : February 3 2016
- */
-
-use dataverse TinySocial;
-
-count (for $x in dataset FacebookUsersInMemory return $x);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.7.update.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.7.update.aql
deleted file mode 100644
index 56a88a2..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.7.update.aql
+++ /dev/null
@@ -1,51 +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.
- */
-/*
- * Test case Name  : node_failback.aql
- * Description     : Make sure node failback completes as expected.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
-                     kill one node and wait until the failover complete, query cluster state,
-                     query data, insert new data, start the killed node and wait for failback,
-                     query cluster state, query data.
- * Expected Result : Success
- * Date            : February 3 2016
- */
-use dataverse TinySocial;
-
-/* insert ids 11-20 */
-insert into dataset TinySocial.FacebookUsersInMemory {"id":11,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
-
-insert into dataset TinySocial.FacebookUsersInMemory {"id":12,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
-
-insert into dataset TinySocial.FacebookUsersInMemory {"id":13,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
-
-insert into dataset TinySocial.FacebookUsersInMemory {"id":14,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
-
-insert into dataset TinySocial.FacebookUsersInMemory {"id":15,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
-
-insert into dataset TinySocial.FacebookUsersInMemory {"id":16,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
-
-insert into dataset TinySocial.FacebookUsersInMemory {"id":17,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
-
-insert into dataset TinySocial.FacebookUsersInMemory {"id":18,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
-
-insert into dataset TinySocial.FacebookUsersInMemory {"id":19,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
-
-insert into dataset TinySocial.FacebookUsersInMemory {"id":20,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.node.aql
deleted file mode 100644
index 98e2761..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.node.aql
+++ /dev/null
@@ -1,28 +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.
- */
-/*
- * Test case Name  : bulkload.aql
- * Description     : Check that Bulkload LSM component are replicated correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, query data, kill one node
-                     and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-start asterix_nc1
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql
deleted file mode 100644
index dd83b4b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-20000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.1.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.1.ddl.aql
deleted file mode 100644
index 6a2441e..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.1.ddl.aql
+++ /dev/null
@@ -1,54 +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.
- */
-/*
- * Test case Name  : bulkload.aql
- * Description     : Check that Bulkload LSM component are replicated correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, query data, kill one node
-                     and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-
-drop dataverse TinySocial if exists;
-create dataverse TinySocial;
-use dataverse TinySocial;
-
-create type EmploymentType as open {
-    organization-name: string,
-    start-date: date,
-    end-date: date?
-}
-
-create type FacebookUserType as closed {
-    id: int,
-    alias: string,
-    name: string,
-    user-since: datetime,
-    friend-ids: {{ int32 }},
-    employment: [EmploymentType]
-}
-
-/********* 2. Create Datasets  ***********/
-use dataverse TinySocial;
-
-drop dataset FacebookUsers if exists;
-
-create dataset FacebookUsers(FacebookUserType)
-primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
deleted file mode 100644
index 3cb5e80..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
+++ /dev/null
@@ -1,33 +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.
- */
-/*
- * Test case Name  : bulkload.aql
- * Description     : Check that Bulkload LSM component are replicated correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, query data, kill one node
-                     and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-
-use dataverse TinySocial;
-
-load dataset FacebookUsers using localfs
-(("path"="asterix_nc1://../asterix-server/src/test/resources/integrationts/replication/data/fbu.adm"),
-("format"="adm"));
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.node.aql
deleted file mode 100644
index 118b7e8..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.node.aql
+++ /dev/null
@@ -1,28 +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.
- */
-/*
- * Test case Name  : bulkload.aql
- * Description     : Check that Bulkload LSM component are replicated correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, query data, kill one node
-                     and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-kill asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.query.aql
deleted file mode 100644
index 9c8cb96..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.query.aql
+++ /dev/null
@@ -1,31 +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.
- */
-/*
- * Test case Name  : bulkload.aql
- * Description     : Check that Bulkload LSM component are replicated correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, query data, kill one node
-                     and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-
-use dataverse TinySocial;
-
-count (for $x in dataset FacebookUsers return $x);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.1.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.1.ddl.aql
deleted file mode 100644
index 8de2067..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.1.ddl.aql
+++ /dev/null
@@ -1,58 +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.
- */
-/*
- * Test case Name  : mem_component_recovery.aql
- * Description     : Check that Memory LSM component are replicated and recovered correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset, query
-                     data from memory, kill one node and wait until the failover complete,
-                     query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-
-drop dataverse TinySocial if exists;
-create dataverse TinySocial;
-use dataverse TinySocial;
-
-create type EmploymentType as open {
-    organization-name: string,
-    start-date: date,
-    end-date: date?
-}
-
-create type FacebookUserType as closed {
-    id: int,
-    alias: string,
-    name: string,
-    user-since: datetime,
-    friend-ids: {{ int32 }},
-    employment: [EmploymentType]
-}
-
-/********* 2. Create Datasets  ***********/
-use dataverse TinySocial;
-
-drop dataset FacebookUsers if exists;
-
-create dataset FacebookUsers(FacebookUserType)
-primary key id;
-
-create dataset FacebookUsersInMemory(FacebookUserType)
-primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
deleted file mode 100644
index 6dd2cea..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
+++ /dev/null
@@ -1,34 +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.
- */
-/*
- * Test case Name  : mem_component_recovery.aql
- * Description     : Check that Memory LSM component are replicated and recovered correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset, query
-                     data from memory, kill one node and wait until the failover complete,
-                     query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-use dataverse TinySocial;
-
-load dataset FacebookUsers using localfs
-(("path"="asterix_nc1://../asterix-server/src/test/resources/integrationts/replication/data/fbu.adm"),("format"="adm"));
-
-insert into dataset TinySocial.FacebookUsersInMemory(for $x in dataset TinySocial.FacebookUsers return $x);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.node.aql
deleted file mode 100644
index 118b7e8..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.node.aql
+++ /dev/null
@@ -1,28 +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.
- */
-/*
- * Test case Name  : bulkload.aql
- * Description     : Check that Bulkload LSM component are replicated correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, query data, kill one node
-                     and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-kill asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.query.aql
deleted file mode 100644
index e25e409..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.query.aql
+++ /dev/null
@@ -1,32 +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.
- */
-/*
- * Test case Name  : mem_component_recovery.aql
- * Description     : Check that Memory LSM component are replicated and recovered correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset, query
-                     data from memory, kill one node and wait until the failover complete,
-                     query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-
-use dataverse TinySocial;
-
-count (for $x in dataset FacebookUsersInMemory return $x);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.1.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.1.ddl.aql
deleted file mode 100644
index 113d144..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.1.ddl.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Test case Name  : metadata_node_recovery.aql
- * Description     : Check that metadata node failover is done correctly.
-                     The test goes as follows:
-                     start 2 nodes, create a dataset, kill metadata node
-                     and wait until the failover complete, verify the
-                     dataset still exists.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-
-drop dataverse TinySocial if exists;
-create dataverse TinySocial;
-use dataverse TinySocial;
-
-create type EmploymentType as open {
-    organization-name: string,
-    start-date: date,
-    end-date: date?
-}
-
-create type FacebookUserType as closed {
-    id: int,
-    alias: string,
-    name: string,
-    user-since: datetime,
-    friend-ids: {{ int32 }},
-    employment: [EmploymentType]
-}
-
-/********* 2. Create Datasets  ***********/
-use dataverse TinySocial;
-
-drop dataset FacebookUsers if exists;
-
-create dataset FacebookUsers(FacebookUserType)
-primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.node.aql
deleted file mode 100644
index 9d5dc9b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.node.aql
+++ /dev/null
@@ -1,28 +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.
- */
-/*
- * Test case Name  : bulkload.aql
- * Description     : Check that Bulkload LSM component are replicated correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, query data, kill one node
-                     and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-kill asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.4.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.4.query.aql
deleted file mode 100644
index 3d525f6..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.4.query.aql
+++ /dev/null
@@ -1,36 +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.
- */
-/*
- * Test case Name  : metadata_node_recovery.aql
- * Description     : Check that metadata node failover is done correctly.
-                     The test goes as follows:
-                     start 2 nodes, create a dataset, kill metadata node
-                     and wait until the failover complete, verify the
-                     dataset still exists.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-
-use dataverse TinySocial;
-
-count(
-for $x in dataset Metadata.Dataset
-where $x.DatasetName ='FacebookUsers'
-return $x.DatasetName
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.1.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.1.sto.cmd
index b750c64..7ddaa20 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.1.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /addReplica 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.10.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.10.pollget.http
index b750c64..6867a5d 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.10.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc1 /admin/storage/partition/0
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.11.post.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.11.post.http
index b750c64..2e8fc63 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.11.post.http
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/partition/master?partition=0&node=asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.12.post.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.12.post.http
index b750c64..e8dca0b 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.12.post.http
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/metadataNode?node=asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.13.pollget.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.13.pollget.http
index b750c64..32e2f78 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.13.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.14.query.sqlpp
similarity index 90%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.14.query.sqlpp
index b750c64..a612cbb 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.14.query.sqlpp
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+SELECT value count(*)
+FROM Metadata.`Dataset`
+WHERE DatasetName = 'ds_1';
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.15.ddl.sqlpp
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.15.ddl.sqlpp
index b750c64..f96d5a8 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.15.ddl.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+CREATE DATASET ds_2(MyType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.16.query.sqlpp
similarity index 90%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.16.query.sqlpp
index b750c64..555954d 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.16.query.sqlpp
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+SELECT value count(*)
+FROM Metadata.`Dataset`
+WHERE DatasetName = 'ds_2';
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.17.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.17.sto.cmd
index b750c64..71621ac 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.17.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /removeReplica 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.2.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.2.pollget.http
index b750c64..6867a5d 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.2.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc1 /admin/storage/partition/0
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.3.ddl.sqlpp
similarity index 89%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.3.ddl.sqlpp
index b750c64..15bc3c5 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.3.ddl.sqlpp
@@ -16,4 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+CREATE TYPE MyType AS {
+  id : int
+};
+
+CREATE DATASET ds_1(MyType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.4.node.cmd
similarity index 97%
rename from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
rename to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.4.node.cmd
index b750c64..a04c093 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.4.node.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+kill asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.5.pollget.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.5.pollget.http
index b750c64..777e3dd 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.5.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=60
+
+/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.6.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.6.pollget.http
index b750c64..6867a5d 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.6.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc1 /admin/storage/partition/0
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.12.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.7.node.cmd
similarity index 100%
rename from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.12.node.aql
rename to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.7.node.cmd
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.8.pollget.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.8.pollget.http
index b750c64..777e3dd 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.8.pollget.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=60
+
+/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.9.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.9.sto.cmd
index b750c64..7ddaa20 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.9.sto.cmd
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /addReplica 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.11.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.11.adm
deleted file mode 100644
index 2edeafb..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.11.adm
+++ /dev/null
@@ -1 +0,0 @@
-20
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.6.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.6.adm
deleted file mode 100644
index 9a03714..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.6.adm
+++ /dev/null
@@ -1 +0,0 @@
-10
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/bulkload/bulkload.5.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/bulkload/bulkload.5.adm
deleted file mode 100644
index 9a03714..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/bulkload/bulkload.5.adm
+++ /dev/null
@@ -1 +0,0 @@
-10
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/mem_component_recovery/mem_component_recovery.5.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/mem_component_recovery/mem_component_recovery.5.adm
deleted file mode 100644
index 9a03714..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/mem_component_recovery/mem_component_recovery.5.adm
+++ /dev/null
@@ -1 +0,0 @@
-10
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.10.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.10.adm
new file mode 100644
index 0000000..7e92c87
--- /dev/null
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.10.adm
@@ -0,0 +1,7 @@
+[ {
+  "partition" : 0,
+  "replicas" : [ {
+    "location" : "127.0.0.1:2002",
+    "status" : "IN_SYNC"
+  } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.11.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.11.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.11.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.12.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.12.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.12.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.13.adm
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.13.adm
index 15c200c..fa5cfb4 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.13.adm
@@ -11,7 +11,7 @@
     },
     "1" : {
       "active" : true,
-      "activeNodeId" : "asterix_nc2",
+      "activeNodeId" : "asterix_nc1",
       "iodeviceNum" : 1,
       "nodeId" : "asterix_nc1",
       "partitionId" : 1,
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/metadata_node/metadata_node.4.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.14.adm
similarity index 100%
rename from asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/metadata_node/metadata_node.4.adm
rename to asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.14.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.16.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.16.adm
new file mode 100644
index 0000000..d0138cb
--- /dev/null
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.16.adm
@@ -0,0 +1,3 @@
+1
+
+
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.2.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.2.adm
new file mode 100644
index 0000000..7e92c87
--- /dev/null
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.2.adm
@@ -0,0 +1,7 @@
+[ {
+  "partition" : 0,
+  "replicas" : [ {
+    "location" : "127.0.0.1:2002",
+    "status" : "IN_SYNC"
+  } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.4.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.5.adm
similarity index 99%
rename from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.4.adm
rename to asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.5.adm
index c327ecf..8d5d123 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.4.adm
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.5.adm
@@ -2,7 +2,7 @@
   "metadata_node" : "asterix_nc1",
   "partitions" : {
     "0" : {
-      "active" : false,
+      "active" : true,
       "activeNodeId" : "asterix_nc1",
       "iodeviceNum" : 0,
       "nodeId" : "asterix_nc1",
@@ -10,7 +10,7 @@
       "pendingActivation" : false
     },
     "1" : {
-      "active" : false,
+      "active" : true,
       "activeNodeId" : "asterix_nc1",
       "iodeviceNum" : 1,
       "nodeId" : "asterix_nc1",
@@ -18,7 +18,7 @@
       "pendingActivation" : false
     },
     "2" : {
-      "active" : true,
+      "active" : false,
       "activeNodeId" : "asterix_nc2",
       "iodeviceNum" : 0,
       "nodeId" : "asterix_nc2",
@@ -26,7 +26,7 @@
       "pendingActivation" : false
     },
     "3" : {
-      "active" : true,
+      "active" : false,
       "activeNodeId" : "asterix_nc2",
       "iodeviceNum" : 1,
       "nodeId" : "asterix_nc2",
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.6.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.6.adm
new file mode 100644
index 0000000..4da3d42
--- /dev/null
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.6.adm
@@ -0,0 +1,7 @@
+[ {
+  "partition" : 0,
+  "replicas" : [ {
+    "location" : "127.0.0.1:2002",
+    "status" : "DISCONNECTED"
+  } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.10.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.8.adm
similarity index 100%
rename from asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.10.adm
rename to asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.8.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/testsuite.xml b/asterixdb/asterix-server/src/test/resources/integrationts/replication/testsuite.xml
index 0f6b528..df5dbac 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/testsuite.xml
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/testsuite.xml
@@ -19,25 +19,8 @@
 <test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries" QueryFileExtension=".aql">
   <test-group name="failover">
     <test-case FilePath="failover">
-      <compilation-unit name="bulkload">
-        <output-dir compare="Text">bulkload</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="failover">
-      <compilation-unit name="mem_component_recovery">
-        <output-dir compare="Text">mem_component_recovery</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="failover">
-      <compilation-unit name="metadata_node">
-        <output-dir compare="Text">metadata_node</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
-  <test-group name="failback">
-    <test-case FilePath="failback">
-      <compilation-unit name="node_failback">
-        <output-dir compare="Text">node_failback</output-dir>
+      <compilation-unit name="resync_failed_replica">
+        <output-dir compare="Text">resync_failed_replica</output-dir>
       </compilation-unit>
     </test-case>
   </test-group>
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 601dec3..54d6268 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -18,37 +18,37 @@
  */
 package org.apache.asterix.transaction.management.resource;
 
+import static org.apache.asterix.common.utils.StorageConstants.INDEX_CHECKPOINT_FILE_PREFIX;
 import static org.apache.hyracks.api.exceptions.ErrorCode.CANNOT_CREATE_FILE;
 
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
+import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
-import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.dataflow.DatasetLocalResource;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.replication.IReplicationManager;
 import org.apache.asterix.common.replication.ReplicationJob;
 import org.apache.asterix.common.storage.DatasetResourceReference;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.asterix.common.storage.ResourceReference;
 import org.apache.asterix.common.utils.StorageConstants;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.commons.io.FileUtils;
@@ -71,7 +71,8 @@
 public class PersistentLocalResourceRepository implements ILocalResourceRepository {
 
     public static final Predicate<Path> INDEX_COMPONENTS = path -> !path.endsWith(StorageConstants.METADATA_FILE_NAME);
-    // Private constants
+    private static final FilenameFilter LSM_INDEX_FILES_FILTER =
+            (dir, name) -> !name.startsWith(INDEX_CHECKPOINT_FILE_PREFIX);
     private static final int MAX_CACHED_RESOURCES = 1000;
     private static final IOFileFilter METADATA_FILES_FILTER = new IOFileFilter() {
         @Override
@@ -100,17 +101,14 @@
     // Finals
     private final IIOManager ioManager;
     private final Cache<String, LocalResource> resourceCache;
-    private final Set<Integer> nodeOriginalPartitions;
-    private final Set<Integer> nodeActivePartitions;
     // Mutables
     private boolean isReplicationEnabled = false;
     private Set<String> filesToBeReplicated;
     private IReplicationManager replicationManager;
-    private Set<Integer> nodeInactivePartitions;
     private final Path[] storageRoots;
     private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
 
-    public PersistentLocalResourceRepository(IIOManager ioManager, String nodeId, MetadataProperties metadataProperties,
+    public PersistentLocalResourceRepository(IIOManager ioManager,
             IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
         this.ioManager = ioManager;
         this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
@@ -122,15 +120,6 @@
         }
         createStorageRoots();
         resourceCache = CacheBuilder.newBuilder().maximumSize(MAX_CACHED_RESOURCES).build();
-        ClusterPartition[] nodePartitions = metadataProperties.getNodePartitions().get(nodeId);
-        //initially the node active partitions are the same as the original partitions
-        nodeOriginalPartitions = new HashSet<>(nodePartitions.length);
-        nodeActivePartitions = new HashSet<>(nodePartitions.length);
-        nodeInactivePartitions = new HashSet<>(nodePartitions.length);
-        for (ClusterPartition partition : nodePartitions) {
-            nodeOriginalPartitions.add(partition.getPartitionId());
-            nodeActivePartitions.add(partition.getPartitionId());
-        }
     }
 
     @Override
@@ -267,7 +256,6 @@
 
         if (isReplicationEnabled) {
             filesToBeReplicated = new HashSet<>();
-            nodeInactivePartitions = ConcurrentHashMap.newKeySet();
         }
     }
 
@@ -299,26 +287,13 @@
         createStorageRoots();
     }
 
-    public Set<Integer> getActivePartitions() {
-        return Collections.unmodifiableSet(nodeActivePartitions);
-    }
-
-    public Set<Integer> getInactivePartitions() {
-        return Collections.unmodifiableSet(nodeInactivePartitions);
-    }
-
-    public synchronized void addActivePartition(int partitonId) {
-        nodeActivePartitions.add(partitonId);
-        nodeInactivePartitions.remove(partitonId);
-    }
-
-    public synchronized void addInactivePartition(int partitonId) {
-        nodeInactivePartitions.add(partitonId);
-        nodeActivePartitions.remove(partitonId);
+    public Set<Integer> getAllPartitions() throws HyracksDataException {
+        return loadAndGetAllResources().values().stream().map(LocalResource::getResource)
+                .map(DatasetLocalResource.class::cast).map(DatasetLocalResource::getPartition)
+                .collect(Collectors.toSet());
     }
 
     public DatasetResourceReference getLocalResourceReference(String absoluteFilePath) throws HyracksDataException {
-        //TODO pass relative path
         final String localResourcePath = StoragePathUtil.getIndexFileRelativePath(absoluteFilePath);
         final LocalResource lr = get(localResourcePath);
         return DatasetResourceReference.of(lr);
@@ -351,17 +326,18 @@
         });
     }
 
-    /**
-     * Given any index file, an absolute {@link FileReference} is returned which points to where the index of
-     * {@code indexFile} is located.
-     *
-     * @param indexFile
-     * @return
-     * @throws HyracksDataException
-     */
-    public FileReference getIndexPath(Path indexFile) throws HyracksDataException {
-        final ResourceReference ref = ResourceReference.of(indexFile.toString());
-        return ioManager.resolve(ref.getRelativePath().toString());
+    public List<String> getPartitionIndexesFiles(int partition) throws HyracksDataException {
+        List<String> partitionFiles = new ArrayList<>();
+        Set<File> partitionIndexes = getPartitionIndexes(partition);
+        for (File indexDir : partitionIndexes) {
+            if (indexDir.isDirectory()) {
+                File[] indexFiles = indexDir.listFiles(LSM_INDEX_FILES_FILTER);
+                if (indexFiles != null) {
+                    Stream.of(indexFiles).map(File::getAbsolutePath).forEach(partitionFiles::add);
+                }
+            }
+        }
+        return partitionFiles;
     }
 
     private void createStorageRoots() {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
index 33c6260..d15e6ff 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
@@ -18,30 +18,23 @@
  */
 package org.apache.asterix.transaction.management.resource;
 
-import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.storage.common.ILocalResourceRepository;
 import org.apache.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
 
 public class PersistentLocalResourceRepositoryFactory implements ILocalResourceRepositoryFactory {
     private final IIOManager ioManager;
-    private final String nodeId;
-    private final MetadataProperties metadataProperties;
     private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
 
-    public PersistentLocalResourceRepositoryFactory(IIOManager ioManager, String nodeId,
-            MetadataProperties metadataProperties, IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
+    public PersistentLocalResourceRepositoryFactory(IIOManager ioManager,
+            IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
         this.ioManager = ioManager;
-        this.nodeId = nodeId;
-        this.metadataProperties = metadataProperties;
         this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
     }
 
     @Override
-    public ILocalResourceRepository createRepository() throws HyracksDataException {
-        return new PersistentLocalResourceRepository(ioManager, nodeId, metadataProperties,
-                indexCheckpointManagerProvider);
+    public ILocalResourceRepository createRepository() {
+        return new PersistentLocalResourceRepository(ioManager, indexCheckpointManagerProvider);
     }
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
index b67da80..011d2a1 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
@@ -26,10 +26,10 @@
 
 import org.apache.asterix.common.context.PrimaryIndexOperationTracker;
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.replication.IReplicationThread;
 import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.ILogBuffer;
 import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.ILogRequester;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.LogRecord;
@@ -318,10 +318,9 @@
             }
         }
         logRecord.isFlushed(true);
-        IReplicationThread replicationThread = logRecord.getReplicationThread();
-
-        if (replicationThread != null) {
-            replicationThread.notifyLogReplicationRequester(logRecord);
+        final ILogRequester logRequester = logRecord.getRequester();
+        if (logRequester != null) {
+            logRequester.notifyFlushed(logRecord);
         }
     }
 
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
index 9b6a4f9..53cd038 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
@@ -61,7 +61,7 @@
                     shouldReplicate = false;
             }
         }
-        logRecord.setReplicated(shouldReplicate);
+        logRecord.setReplicate(shouldReplicate);
 
         //Remote flush logs do not need to be flushed separately since they may not trigger local flush
         if (logRecord.getLogType() == LogType.FLUSH && logRecord.getLogSource() == LogSource.LOCAL) {
@@ -76,9 +76,9 @@
     protected void appendToLogTail(ILogRecord logRecord) {
         syncAppendToLogTail(logRecord);
 
-        if (logRecord.isReplicated()) {
+        if (logRecord.isReplicate()) {
             try {
-                replicationManager.replicateLog(logRecord);
+                replicationManager.replicate(logRecord);
             } catch (InterruptedException e) {
                 Thread.currentThread().interrupt();
                 throw new ACIDException(e);
@@ -92,9 +92,9 @@
                         logRecord.wait();
                     }
                     //wait for job Commit/Abort ACK from replicas
-                    if (logRecord.isReplicated() && (logRecord.getLogType() == LogType.JOB_COMMIT
+                    if (logRecord.isReplicate() && (logRecord.getLogType() == LogType.JOB_COMMIT
                             || logRecord.getLogType() == LogType.ABORT)) {
-                        while (!replicationManager.hasBeenReplicated(logRecord)) {
+                        while (!logRecord.isReplicated()) {
                             logRecord.wait();
                         }
                     }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManagerFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManagerFactory.java
index 68c5ce1..ca7d37a 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManagerFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManagerFactory.java
@@ -29,11 +29,7 @@
     }
 
     public static ICheckpointManager create(ITransactionSubsystem txnSubsystem,
-            CheckpointProperties checkpointProperties, boolean replicationEnabled) {
-        if (!replicationEnabled) {
-            return new CheckpointManager(txnSubsystem, checkpointProperties);
-        } else {
-            return new ReplicationCheckpointManager(txnSubsystem, checkpointProperties);
-        }
+            CheckpointProperties checkpointProperties) {
+        return new CheckpointManager(txnSubsystem, checkpointProperties);
     }
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/ReplicationCheckpointManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/ReplicationCheckpointManager.java
deleted file mode 100644
index 4bbcabe..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/ReplicationCheckpointManager.java
+++ /dev/null
@@ -1,144 +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.transaction.management.service.recovery;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.asterix.common.api.IApplicationContext;
-import org.apache.asterix.common.api.IDatasetLifecycleManager;
-import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.config.MetadataProperties;
-import org.apache.asterix.common.replication.IReplicaResourcesManager;
-import org.apache.asterix.common.replication.IReplicationManager;
-import org.apache.asterix.common.transactions.CheckpointProperties;
-import org.apache.asterix.common.transactions.ICheckpointManager;
-import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-/**
- * An implementation of {@link ICheckpointManager} that defines the logic
- * of checkpoints when replication is enabled..
- */
-public class ReplicationCheckpointManager extends AbstractCheckpointManager {
-
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    public ReplicationCheckpointManager(ITransactionSubsystem txnSubsystem, CheckpointProperties checkpointProperties) {
-        super(txnSubsystem, checkpointProperties);
-    }
-
-    /**
-     * Performs a sharp checkpoint. All datasets are flushed and all transaction
-     * log files are deleted except the files that are needed for dead replicas.
-     */
-    @Override
-    public synchronized void doSharpCheckpoint() throws HyracksDataException {
-        LOGGER.info("Starting sharp checkpoint...");
-        final IDatasetLifecycleManager datasetLifecycleManager =
-                txnSubsystem.getAsterixAppRuntimeContextProvider().getDatasetLifecycleManager();
-        datasetLifecycleManager.flushAllDatasets();
-        long minFirstLSN;
-        // If shutting down, need to check if we need to keep any remote logs for dead replicas
-        if (txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext().isShuttingdown()) {
-            final Set<String> deadReplicaIds = txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext()
-                    .getReplicationManager().getDeadReplicasIds();
-            if (deadReplicaIds.isEmpty()) {
-                // No dead replicas => no need to keep any log
-                minFirstLSN = SHARP_CHECKPOINT_LSN;
-            } else {
-                // Get min LSN of dead replicas remote resources
-                minFirstLSN = getDeadReplicasMinFirstLSN(deadReplicaIds);
-            }
-        } else {
-            // Start up complete checkpoint. Avoid deleting remote recovery logs.
-            minFirstLSN = txnSubsystem.getRecoveryManager().getMinFirstLSN();
-        }
-        capture(minFirstLSN, true);
-        if (minFirstLSN == SHARP_CHECKPOINT_LSN) {
-            // No need to keep any logs
-            txnSubsystem.getLogManager().renewLogFiles();
-        } else {
-            // Delete only log files with LSNs < any dead replica partition minimum LSN
-            txnSubsystem.getLogManager().deleteOldLogFiles(minFirstLSN);
-        }
-        LOGGER.info("Completed sharp checkpoint.");
-    }
-
-    /***
-     * Attempts to perform a soft checkpoint at the specified {@code checkpointTargetLSN}.
-     * If a checkpoint cannot be captured due to datasets having LSN < {@code checkpointTargetLSN},
-     * an asynchronous flush is triggered on them. If the checkpoint fails due to a replica index,
-     * a request is sent to the primary replica of the index to flush it.
-     * When a checkpoint is successful, all transaction log files that end with
-     * LSN < {@code checkpointTargetLSN} are deleted.
-     */
-    @Override
-    public synchronized long tryCheckpoint(long checkpointTargetLSN) throws HyracksDataException {
-        LOGGER.info("Attemping soft checkpoint...");
-        final long minFirstLSN = txnSubsystem.getRecoveryManager().getMinFirstLSN();
-        boolean checkpointSucceeded = minFirstLSN >= checkpointTargetLSN;
-        if (!checkpointSucceeded) {
-            // Flush datasets with indexes behind target checkpoint LSN
-            final IDatasetLifecycleManager datasetLifecycleManager =
-                    txnSubsystem.getAsterixAppRuntimeContextProvider().getDatasetLifecycleManager();
-            datasetLifecycleManager.scheduleAsyncFlushForLaggingDatasets(checkpointTargetLSN);
-            // Request remote replicas to flush lagging indexes
-            final IReplicationManager replicationManager =
-                    txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext().getReplicationManager();
-            try {
-                replicationManager.requestFlushLaggingReplicaIndexes(checkpointTargetLSN);
-            } catch (IOException e) {
-                throw new HyracksDataException(e);
-            }
-        }
-        capture(minFirstLSN, false);
-        if (checkpointSucceeded) {
-            txnSubsystem.getLogManager().deleteOldLogFiles(minFirstLSN);
-            LOGGER.info(String.format("soft checkpoint succeeded with at LSN(%s)", minFirstLSN));
-        }
-        return minFirstLSN;
-    }
-
-    private long getDeadReplicasMinFirstLSN(Set<String> deadReplicaIds) throws HyracksDataException {
-        final IReplicaResourcesManager remoteResourcesManager =
-                txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext().getReplicaResourcesManager();
-        final IApplicationContext propertiesProvider =
-                txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext();
-        final MetadataProperties metadataProperties = propertiesProvider.getMetadataProperties();
-        final PersistentLocalResourceRepository localResourceRepository =
-                (PersistentLocalResourceRepository) txnSubsystem.getAsterixAppRuntimeContextProvider()
-                        .getLocalResourceRepository();
-        // Get partitions of the dead replicas that are not active on this node
-        final Set<Integer> deadReplicasPartitions = new HashSet<>();
-        for (String deadReplicaId : deadReplicaIds) {
-            final ClusterPartition[] nodePartitons = metadataProperties.getNodePartitions().get(deadReplicaId);
-            for (ClusterPartition partition : nodePartitons) {
-                if (!localResourceRepository.getActivePartitions().contains(partition.getPartitionId())) {
-                    deadReplicasPartitions.add(partition.getPartitionId());
-                }
-            }
-        }
-        return remoteResourcesManager.getPartitionsMinLSN(deadReplicasPartitions);
-    }
-}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/replication/IReplicationJob.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/replication/IReplicationJob.java
index cbe6d1a..33c9c96 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/replication/IReplicationJob.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/replication/IReplicationJob.java
@@ -45,4 +45,8 @@
 
     public Set<String> getJobFiles();
 
+    default String getAnyFile() {
+        return getJobFiles().stream().findAny()
+                .orElseThrow(() -> new IllegalStateException("Replication job without any files"));
+    }
 }