You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mh...@apache.org on 2016/02/18 10:54:27 UTC

[1/6] incubator-asterixdb git commit: Asterix NCs Failback Support

Repository: incubator-asterixdb
Updated Branches:
  refs/heads/master c318249ef -> 98d38e6a0


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
index 11dc282..4a17541 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -49,19 +49,18 @@ import java.util.logging.Logger;
 
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.ILocalResourceMetadata;
+import org.apache.asterix.common.cluster.ClusterPartition;
+import org.apache.asterix.common.config.AsterixMetadataProperties;
 import org.apache.asterix.common.config.IAsterixPropertiesProvider;
 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.IReplicationManager;
-import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import org.apache.asterix.common.transactions.ILogReader;
 import org.apache.asterix.common.transactions.ILogRecord;
 import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.common.transactions.LogRecord;
 import org.apache.asterix.common.transactions.LogType;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import org.apache.asterix.transaction.management.service.logging.LogManager;
@@ -71,25 +70,21 @@ import org.apache.asterix.transaction.management.service.transaction.Transaction
 import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.api.IIndex;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
-import org.apache.hyracks.storage.common.file.ILocalResourceRepository;
 import org.apache.hyracks.storage.common.file.LocalResource;
 
 /**
  * This is the Recovery Manager and is responsible for rolling back a
- * transaction as well as doing a system recovery. TODO: Crash Recovery logic is
- * not in place completely. Once we have physical logging implemented, we would
- * add support for crash recovery.
+ * transaction as well as doing a system recovery.
  */
 public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
 
-    public static final boolean IS_DEBUG_MODE = false;//true
+    public static final boolean IS_DEBUG_MODE = false;
     private static final Logger LOGGER = Logger.getLogger(RecoveryManager.class.getName());
     private final TransactionSubsystem txnSubsystem;
     private final LogManager logMgr;
@@ -101,6 +96,7 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
     private static final long MEGABYTE = 1024L * 1024L;
     private Map<Integer, JobEntityCommits> jobId2WinnerEntitiesMap = null;
     private static final long MAX_CACHED_ENTITY_COMMITS_PER_JOB_SIZE = 4 * MEGABYTE; //4MB;
+    private final PersistentLocalResourceRepository localResourceRepository;
 
     /**
      * A file at a known location that contains the LSN of the last log record
@@ -111,11 +107,13 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
 
     public RecoveryManager(TransactionSubsystem txnSubsystem) {
         this.txnSubsystem = txnSubsystem;
-        this.logMgr = (LogManager) txnSubsystem.getLogManager();
-        this.checkpointHistory = this.txnSubsystem.getTransactionProperties().getCheckpointHistory();
+        logMgr = (LogManager) txnSubsystem.getLogManager();
+        checkpointHistory = txnSubsystem.getTransactionProperties().getCheckpointHistory();
         IAsterixPropertiesProvider propertiesProvider = (IAsterixPropertiesProvider) txnSubsystem
                 .getAsterixAppRuntimeContextProvider().getAppContext();
-        this.replicationEnabled = propertiesProvider.getReplicationProperties().isReplicationEnabled();
+        replicationEnabled = propertiesProvider.getReplicationProperties().isReplicationEnabled();
+        localResourceRepository = (PersistentLocalResourceRepository) txnSubsystem.getAsterixAppRuntimeContextProvider()
+                .getLocalResourceRepository();
     }
 
     /**
@@ -160,12 +158,8 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
             long readableSmallestLSN = logMgr.getReadableSmallestLSN();
             if (logMgr.getAppendLSN() == readableSmallestLSN) {
                 if (checkpointObject.getMinMCTFirstLsn() != SHARP_CHECKPOINT_LSN) {
-                    if (LOGGER.isLoggable(Level.INFO)) {
-                        LOGGER.info("[Warning] ---------------------------------------------------");
-                        LOGGER.info("[Warning] Some(or all) of transaction log files are lost.");
-                        LOGGER.info("[Warning] ---------------------------------------------------");
-                        //No choice but continuing when the log files are lost.
-                    }
+                    LOGGER.warning("Some(or all) of transaction log files are lost.");
+                    //No choice but continuing when the log files are lost.
                 }
                 state = SystemState.HEALTHY;
                 return state;
@@ -180,412 +174,227 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
         }
     }
 
-    //This method is used only when replication is disabled. Therefore, there is no need to check logs node ids
+    //This method is used only when replication is disabled.
     @Override
     public void startRecovery(boolean synchronous) throws IOException, ACIDException {
-        //delete any recovery files from previous failed recovery attempts
-        deleteRecoveryTemporaryFiles();
-
-        int updateLogCount = 0;
-        int entityCommitLogCount = 0;
-        int jobCommitLogCount = 0;
-        int redoCount = 0;
-        int abortLogCount = 0;
-        int jobId = -1;
-
         state = SystemState.RECOVERING;
-        LOGGER.log(Level.INFO, "[RecoveryMgr] starting recovery ...");
-
-        Set<Integer> winnerJobSet = new HashSet<Integer>();
-        jobId2WinnerEntitiesMap = new HashMap<>();
+        LOGGER.log(Level.INFO, "starting recovery ...");
 
-        TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
-        JobEntityCommits jobEntityWinners = null;
-        //#. read checkpoint file and set lowWaterMark where anaylsis and redo start
         long readableSmallestLSN = logMgr.getReadableSmallestLSN();
         CheckpointObject checkpointObject = readCheckpoint();
         long lowWaterMarkLSN = checkpointObject.getMinMCTFirstLsn();
         if (lowWaterMarkLSN < readableSmallestLSN) {
             lowWaterMarkLSN = readableSmallestLSN;
         }
-        int maxJobId = checkpointObject.getMaxJobId();
 
-        //-------------------------------------------------------------------------
-        //  [ analysis phase ]
-        //  - collect all committed Lsn
-        //-------------------------------------------------------------------------
-        LOGGER.log(Level.INFO, "[RecoveryMgr] in analysis phase");
-
-        //#. set log reader to the lowWaterMarkLsn
-        ILogReader logReader = logMgr.getLogReader(true);
-        ILogRecord logRecord = null;
-        try {
-            logReader.initializeScan(lowWaterMarkLSN);
-            logRecord = logReader.next();
-            while (logRecord != null) {
-                if (IS_DEBUG_MODE) {
-                    LOGGER.info(logRecord.getLogRecordForDisplay());
-                }
-                //update max jobId
-                if (logRecord.getJobId() > maxJobId) {
-                    maxJobId = logRecord.getJobId();
-                }
-                switch (logRecord.getLogType()) {
-                    case LogType.UPDATE:
-                        updateLogCount++;
-                        break;
-                    case LogType.JOB_COMMIT:
-                        jobId = logRecord.getJobId();
-                        winnerJobSet.add(jobId);
-                        if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
-                            jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
-                            //to delete any spilled files as well
-                            jobEntityWinners.clear();
-                            jobId2WinnerEntitiesMap.remove(jobId);
-                        }
-                        jobCommitLogCount++;
-                        break;
-                    case LogType.ENTITY_COMMIT:
-                    case LogType.UPSERT_ENTITY_COMMIT:
-                        jobId = logRecord.getJobId();
-                        if (!jobId2WinnerEntitiesMap.containsKey(jobId)) {
-                            jobEntityWinners = new JobEntityCommits(jobId);
-                            if (needToFreeMemory()) {
-                                //if we don't have enough memory for one more job, we will force all jobs to spill their cached entities to disk.
-                                //This could happen only when we have many jobs with small number of records and none of them have job commit.
-                                freeJobsCachedEntities(jobId);
-                            }
-                            jobId2WinnerEntitiesMap.put(jobId, jobEntityWinners);
-                        } else {
-                            jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
-                        }
-                        jobEntityWinners.add(logRecord);
-                        entityCommitLogCount++;
-                        break;
-                    case LogType.ABORT:
-                        abortLogCount++;
-                        break;
-                    case LogType.FLUSH:
-                        break;
-                    default:
-                        throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
-                }
-                logRecord = logReader.next();
-            }
-
-            //prepare winners for search after analysis is done to flush anything remaining in memory to disk.
-            for (JobEntityCommits winners : jobId2WinnerEntitiesMap.values()) {
-                winners.prepareForSearch();
-            }
-            //-------------------------------------------------------------------------
-            //  [ redo phase ]
-            //  - redo if
-            //    1) The TxnId is committed && --> guarantee durability
-            //    2) lsn > maxDiskLastLsn of the index --> guarantee idempotence
-            //-------------------------------------------------------------------------
-            LOGGER.info("[RecoveryMgr] in redo phase");
-
-            long resourceId;
-            long maxDiskLastLsn;
-            long LSN = -1;
-            ILSMIndex index = null;
-            LocalResource localResource = null;
-            ILocalResourceMetadata localResourceMetadata = null;
-            Map<Long, Long> resourceId2MaxLSNMap = new HashMap<Long, Long>();
-            boolean foundWinner = false;
-
-            IAsterixAppRuntimeContextProvider appRuntimeContext = txnSubsystem.getAsterixAppRuntimeContextProvider();
-            //get datasetLifeCycleManager
-            IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
-                    .getDatasetLifecycleManager();
-            ILocalResourceRepository localResourceRepository = appRuntimeContext.getLocalResourceRepository();
-            Map<Long, LocalResource> resourcesMap = ((PersistentLocalResourceRepository) localResourceRepository)
-                    .loadAndGetAllResources();
-            //set log reader to the lowWaterMarkLsn again.
-            logReader.initializeScan(lowWaterMarkLSN);
-            logRecord = logReader.next();
-            while (logRecord != null) {
-                if (IS_DEBUG_MODE) {
-                    LOGGER.info(logRecord.getLogRecordForDisplay());
-                }
-                LSN = logRecord.getLSN();
-                jobId = logRecord.getJobId();
-                foundWinner = false;
-                switch (logRecord.getLogType()) {
-                    case LogType.UPDATE:
-                        if (winnerJobSet.contains(jobId)) {
-                            foundWinner = true;
-                        } else if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
-                            jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
-                            tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
-                                    logRecord.getPKValue(), logRecord.getPKValueSize());
-                            if (jobEntityWinners.containsEntityCommitForTxnId(LSN, tempKeyTxnId)) {
-                                foundWinner = true;
-                            }
-                        }
-                        if (foundWinner) {
-                            resourceId = logRecord.getResourceId();
-                            localResource = resourcesMap.get(resourceId);
-                            /*******************************************************************
-                             * [Notice]
-                             * -> Issue
-                             * Delete index may cause a problem during redo.
-                             * The index operation to be redone couldn't be redone because the corresponding index
-                             * may not exist in NC due to the possible index drop DDL operation.
-                             * -> Approach
-                             * Avoid the problem during redo.
-                             * More specifically, the problem will be detected when the localResource of
-                             * the corresponding index is retrieved, which will end up with 'null'.
-                             * If null is returned, then just go and process the next
-                             * log record.
-                             *******************************************************************/
-                            if (localResource == null) {
-                                logRecord = logReader.next();
-                                continue;
-                            }
-                            /*******************************************************************/
-
-                            //get index instance from IndexLifeCycleManager
-                            //if index is not registered into IndexLifeCycleManager,
-                            //create the index using LocalMetadata stored in LocalResourceRepository
-                            String resourceAbsolutePath = localResource.getResourcePath();
-                            index = (ILSMIndex) datasetLifecycleManager.getIndex(resourceAbsolutePath);
-                            if (index == null) {
-                                //#. create index instance and register to indexLifeCycleManager
-                                localResourceMetadata = (ILocalResourceMetadata) localResource.getResourceObject();
-                                index = localResourceMetadata.createIndexInstance(appRuntimeContext,
-                                        resourceAbsolutePath, localResource.getPartition());
-                                datasetLifecycleManager.register(resourceAbsolutePath, index);
-                                datasetLifecycleManager.open(resourceAbsolutePath);
-
-                                //#. get maxDiskLastLSN
-                                ILSMIndex lsmIndex = index;
-                                maxDiskLastLsn = ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
-                                        .getComponentLSN(lsmIndex.getImmutableComponents());
-
-                                //#. set resourceId and maxDiskLastLSN to the map
-                                resourceId2MaxLSNMap.put(resourceId, maxDiskLastLsn);
-                            } else {
-                                maxDiskLastLsn = resourceId2MaxLSNMap.get(resourceId);
-                            }
-
-                            if (LSN > maxDiskLastLsn) {
-                                redo(logRecord, datasetLifecycleManager);
-                                redoCount++;
-                            }
-                        }
-                        break;
-                    case LogType.JOB_COMMIT:
-                    case LogType.ENTITY_COMMIT:
-                    case LogType.ABORT:
-                    case LogType.FLUSH:
-                    case LogType.UPSERT_ENTITY_COMMIT:
-                        //do nothing
-                        break;
-                    default:
-                        throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
-                }
-                logRecord = logReader.next();
-            }
+        //delete any recovery files from previous failed recovery attempts
+        deleteRecoveryTemporaryFiles();
 
-            //close all indexes
-            Set<Long> resourceIdList = resourceId2MaxLSNMap.keySet();
-            for (long r : resourceIdList) {
-                datasetLifecycleManager.close(resourcesMap.get(r).getResourcePath());
-            }
+        //get active partitions on this node
+        Set<Integer> activePartitions = localResourceRepository.getNodeOrignalPartitions();
+        replayPartitionsLogs(activePartitions, logMgr.getLogReader(true), lowWaterMarkLSN);
+    }
 
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("[RecoveryMgr] recovery is completed.");
-                LOGGER.info("[RecoveryMgr's recovery log count] update/entityCommit/jobCommit/abort/redo = "
-                        + updateLogCount + "/" + entityCommitLogCount + "/" + jobCommitLogCount + "/" + abortLogCount
-                        + "/" + redoCount);
-            }
+    @Override
+    public synchronized void replayPartitionsLogs(Set<Integer> partitions, ILogReader logReader, long lowWaterMarkLSN)
+            throws IOException, ACIDException {
+        try {
+            Set<Integer> winnerJobSet = startRecoverysAnalysisPhase(partitions, logReader, lowWaterMarkLSN);
+            startRecoveryRedoPhase(partitions, logReader, lowWaterMarkLSN, winnerJobSet);
         } finally {
             logReader.close();
-            //delete any recovery files after completing recovery
             deleteRecoveryTemporaryFiles();
         }
     }
 
-    private static boolean needToFreeMemory() {
-        return Runtime.getRuntime().freeMemory() < MAX_CACHED_ENTITY_COMMITS_PER_JOB_SIZE;
-    }
-
-    @Override
-    public void replayRemoteLogs(ArrayList<ILogRecord> remoteLogs) throws HyracksDataException, ACIDException {
+    private synchronized Set<Integer> startRecoverysAnalysisPhase(Set<Integer> partitions, ILogReader logReader,
+            long lowWaterMarkLSN) throws IOException, ACIDException {
         int updateLogCount = 0;
         int entityCommitLogCount = 0;
         int jobCommitLogCount = 0;
-        int redoCount = 0;
         int abortLogCount = 0;
         int jobId = -1;
 
-        state = SystemState.RECOVERING;
+        Set<Integer> winnerJobSet = new HashSet<Integer>();
+        jobId2WinnerEntitiesMap = new HashMap<>();
 
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("[RecoveryMgr] starting recovery ...");
-        }
+        //set log reader to the lowWaterMarkLsn
+        ILogRecord logRecord = null;
+        logReader.initializeScan(lowWaterMarkLSN);
 
-        Set<Integer> winnerJobSet = new HashSet<Integer>();
-        Map<Integer, Set<TxnId>> jobId2WinnerEntitiesMap = new HashMap<Integer, Set<TxnId>>();
-        //winnerEntity is used to add pairs, <committed TxnId, the most recent commit Lsn of the TxnId>
-        Set<TxnId> winnerEntitySet = null;
-        TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
-        TxnId winnerEntity = null;
-
-        //-------------------------------------------------------------------------
-        //  [ analysis phase ]
-        //  - collect all committed Lsn
-        //-------------------------------------------------------------------------
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("[RecoveryMgr] in analysis phase");
-        }
+        //collect all committed Lsn
+        JobEntityCommits jobEntityWinners = null;
 
-        String nodeId = logMgr.getNodeId();
-        ILogRecord logRecord;
-        for (int i = 0; i < remoteLogs.size(); i++) {
-            logRecord = remoteLogs.get(i);
+        logRecord = logReader.next();
+        while (logRecord != null) {
             if (IS_DEBUG_MODE) {
                 LOGGER.info(logRecord.getLogRecordForDisplay());
             }
-
-            if (logRecord.getNodeId().equals(nodeId)) {
-                //update max jobId
-                switch (logRecord.getLogType()) {
-                    case LogType.UPDATE:
+            switch (logRecord.getLogType()) {
+                case LogType.UPDATE:
+                    if (partitions.contains(logRecord.getResourcePartition())) {
                         updateLogCount++;
-                        break;
-                    case LogType.JOB_COMMIT:
-                    case LogType.UPSERT_ENTITY_COMMIT:
-                        winnerJobSet.add(Integer.valueOf(logRecord.getJobId()));
-                        jobId2WinnerEntitiesMap.remove(Integer.valueOf(logRecord.getJobId()));
-                        jobCommitLogCount++;
-                        break;
-                    case LogType.ENTITY_COMMIT:
+                    }
+                    break;
+                case LogType.JOB_COMMIT:
+                    jobId = logRecord.getJobId();
+                    winnerJobSet.add(jobId);
+                    if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
+                        jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
+                        //to delete any spilled files as well
+                        jobEntityWinners.clear();
+                        jobId2WinnerEntitiesMap.remove(jobId);
+                    }
+                    jobCommitLogCount++;
+                    break;
+                case LogType.ENTITY_COMMIT:
+                case LogType.UPSERT_ENTITY_COMMIT:
+                    if (partitions.contains(logRecord.getResourcePartition())) {
                         jobId = logRecord.getJobId();
-                        winnerEntity = new TxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
-                                logRecord.getPKValue(), logRecord.getPKValueSize(), true);
-                        if (!jobId2WinnerEntitiesMap.containsKey(Integer.valueOf(jobId))) {
-                            winnerEntitySet = new HashSet<TxnId>();
-                            jobId2WinnerEntitiesMap.put(Integer.valueOf(jobId), winnerEntitySet);
+                        if (!jobId2WinnerEntitiesMap.containsKey(jobId)) {
+                            jobEntityWinners = new JobEntityCommits(jobId);
+                            if (needToFreeMemory()) {
+                                //if we don't have enough memory for one more job, we will force all jobs to spill their cached entities to disk.
+                                //This could happen only when we have many jobs with small number of records and none of them have job commit.
+                                freeJobsCachedEntities(jobId);
+                            }
+                            jobId2WinnerEntitiesMap.put(jobId, jobEntityWinners);
                         } else {
-                            winnerEntitySet = jobId2WinnerEntitiesMap.get(Integer.valueOf(jobId));
+                            jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
                         }
-                        winnerEntitySet.add(winnerEntity);
+                        jobEntityWinners.add(logRecord);
                         entityCommitLogCount++;
-                        break;
-                    case LogType.ABORT:
-                        abortLogCount++;
-                        break;
-                    case LogType.FLUSH:
-                        break;
-                    default:
-                        throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
-                }
+                    }
+                    break;
+                case LogType.ABORT:
+                    abortLogCount++;
+                    break;
+                case LogType.FLUSH:
+                    break;
+                default:
+                    throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
             }
+            logRecord = logReader.next();
         }
 
-        //-------------------------------------------------------------------------
-        //  [ redo phase ]
-        //  - redo if
-        //    1) The TxnId is committed && --> guarantee durability
-        //    2) lsn > maxDiskLastLsn of the index --> guarantee idempotence
-        //-------------------------------------------------------------------------
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("[RecoveryMgr] in redo phase");
+        //prepare winners for search after analysis is done to flush anything remaining in memory to disk.
+        for (JobEntityCommits winners : jobId2WinnerEntitiesMap.values()) {
+            winners.prepareForSearch();
         }
 
+        LOGGER.info("Logs analysis phase completed.");
+        LOGGER.info("Analysis log count update/entityCommit/jobCommit/abort = " + updateLogCount + "/"
+                + entityCommitLogCount + "/" + jobCommitLogCount + "/" + abortLogCount);
+
+        return winnerJobSet;
+    }
+
+    private synchronized void startRecoveryRedoPhase(Set<Integer> partitions, ILogReader logReader, long lowWaterMarkLSN,
+            Set<Integer> winnerJobSet) throws IOException, ACIDException {
+        int redoCount = 0;
+        int jobId = -1;
+
         long resourceId;
         long maxDiskLastLsn;
         long LSN = -1;
         ILSMIndex index = null;
         LocalResource localResource = null;
         ILocalResourceMetadata localResourceMetadata = null;
-        Map<Long, Long> resourceId2MaxLSNMap = new HashMap<Long, Long>();
         boolean foundWinner = false;
+        JobEntityCommits jobEntityWinners = null;
 
-        //#. get indexLifeCycleManager
         IAsterixAppRuntimeContextProvider appRuntimeContext = txnSubsystem.getAsterixAppRuntimeContextProvider();
         IDatasetLifecycleManager datasetLifecycleManager = appRuntimeContext.getDatasetLifecycleManager();
-        PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) appRuntimeContext
-                .getLocalResourceRepository();
+
         Map<Long, LocalResource> resourcesMap = localResourceRepository.loadAndGetAllResources();
-        //#. set log reader to the lowWaterMarkLsn again.
-        for (int i = 0; i < remoteLogs.size(); i++) {
-            logRecord = remoteLogs.get(i);
-            if (IS_DEBUG_MODE) {
-                LOGGER.info(logRecord.getLogRecordForDisplay());
-            }
-            if (logRecord.getNodeId().equals(nodeId)) {
+        Map<Long, Long> resourceId2MaxLSNMap = new HashMap<Long, Long>();
+        TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
+
+        ILogRecord logRecord = null;
+        try {
+            logReader.initializeScan(lowWaterMarkLSN);
+            logRecord = logReader.next();
+            while (logRecord != null) {
+                if (IS_DEBUG_MODE) {
+                    LOGGER.info(logRecord.getLogRecordForDisplay());
+                }
                 LSN = logRecord.getLSN();
                 jobId = logRecord.getJobId();
                 foundWinner = false;
                 switch (logRecord.getLogType()) {
                     case LogType.UPDATE:
-                        if (winnerJobSet.contains(Integer.valueOf(jobId))) {
-                            foundWinner = true;
-                        } else if (jobId2WinnerEntitiesMap.containsKey(Integer.valueOf(jobId))) {
-                            winnerEntitySet = jobId2WinnerEntitiesMap.get(Integer.valueOf(jobId));
-                            tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
-                                    logRecord.getPKValue(), logRecord.getPKValueSize());
-                            if (winnerEntitySet.contains(tempKeyTxnId)) {
+                        if (partitions.contains(logRecord.getResourcePartition())) {
+                            if (winnerJobSet.contains(jobId)) {
                                 foundWinner = true;
+                            } else if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
+                                jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
+                                tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+                                        logRecord.getPKValue(), logRecord.getPKValueSize());
+                                if (jobEntityWinners.containsEntityCommitForTxnId(LSN, tempKeyTxnId)) {
+                                    foundWinner = true;
+                                }
                             }
-                        }
-                        if (foundWinner) {
-                            resourceId = logRecord.getResourceId();
-                            localResource = resourcesMap.get(resourceId);
-
-                            /*******************************************************************
-                             * [Notice]
-                             * -> Issue
-                             * Delete index may cause a problem during redo.
-                             * The index operation to be redone couldn't be redone because the corresponding index
-                             * may not exist in NC due to the possible index drop DDL operation.
-                             * -> Approach
-                             * Avoid the problem during redo.
-                             * More specifically, the problem will be detected when the localResource of
-                             * the corresponding index is retrieved, which will end up with 'null'.
-                             * If null is returned, then just go and process the next
-                             * log record.
-                             *******************************************************************/
-                            if (localResource == null) {
-                                continue;
-                            }
-                            /*******************************************************************/
-
-                            //get index instance from IndexLifeCycleManager
-                            //if index is not registered into IndexLifeCycleManager,
-                            //create the index using LocalMetadata stored in LocalResourceRepository
-                            //get partition path in this node
-                            String partitionIODevicePath = localResourceRepository
-                                    .getPartitionPath(localResource.getPartition());
-                            String resourceAbsolutePath = partitionIODevicePath + File.separator
-                                    + localResource.getResourceName();
-                            localResource.setResourcePath(resourceAbsolutePath);
-                            index = (ILSMIndex) datasetLifecycleManager.getIndex(resourceAbsolutePath);
-                            if (index == null) {
-                                //#. create index instance and register to indexLifeCycleManager
-                                localResourceMetadata = (ILocalResourceMetadata) localResource.getResourceObject();
-                                index = localResourceMetadata.createIndexInstance(appRuntimeContext,
-                                        resourceAbsolutePath, localResource.getPartition());
-                                datasetLifecycleManager.register(resourceAbsolutePath, index);
-                                datasetLifecycleManager.open(resourceAbsolutePath);
-
-                                //#. get maxDiskLastLSN
-                                ILSMIndex lsmIndex = index;
-                                maxDiskLastLsn = ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
-                                        .getComponentLSN(lsmIndex.getImmutableComponents());
-
-                                //#. set resourceId and maxDiskLastLSN to the map
-                                resourceId2MaxLSNMap.put(Long.valueOf(resourceId), Long.valueOf(maxDiskLastLsn));
-                            } else {
-                                maxDiskLastLsn = resourceId2MaxLSNMap.get(Long.valueOf(resourceId));
-                            }
+                            if (foundWinner) {
+                                resourceId = logRecord.getResourceId();
+                                localResource = resourcesMap.get(resourceId);
+                                /*******************************************************************
+                                 * [Notice]
+                                 * -> Issue
+                                 * Delete index may cause a problem during redo.
+                                 * The index operation to be redone couldn't be redone because the corresponding index
+                                 * may not exist in NC due to the possible index drop DDL operation.
+                                 * -> Approach
+                                 * Avoid the problem during redo.
+                                 * More specifically, the problem will be detected when the localResource of
+                                 * the corresponding index is retrieved, which will end up with 'null'.
+                                 * If null is returned, then just go and process the next
+                                 * log record.
+                                 *******************************************************************/
+                                if (localResource == null) {
+                                    logRecord = logReader.next();
+                                    continue;
+                                }
+                                /*******************************************************************/
+
+                                //get index instance from IndexLifeCycleManager
+                                //if index is not registered into IndexLifeCycleManager,
+                                //create the index using LocalMetadata stored in LocalResourceRepository
+                                //get partition path in this node
+                                String partitionIODevicePath = localResourceRepository
+                                        .getPartitionPath(localResource.getPartition());
+                                String resourceAbsolutePath = partitionIODevicePath + File.separator
+                                        + localResource.getResourceName();
+                                localResource.setResourcePath(resourceAbsolutePath);
+                                index = (ILSMIndex) datasetLifecycleManager.getIndex(resourceAbsolutePath);
+                                if (index == null) {
+                                    //#. create index instance and register to indexLifeCycleManager
+                                    localResourceMetadata = (ILocalResourceMetadata) localResource.getResourceObject();
+                                    index = localResourceMetadata.createIndexInstance(appRuntimeContext,
+                                            resourceAbsolutePath, localResource.getPartition());
+                                    datasetLifecycleManager.register(resourceAbsolutePath, index);
+                                    datasetLifecycleManager.open(resourceAbsolutePath);
 
-                            if (LSN > maxDiskLastLsn) {
-                                redo(logRecord, datasetLifecycleManager);
-                                redoCount++;
+                                    //#. get maxDiskLastLSN
+                                    ILSMIndex lsmIndex = index;
+                                    try {
+                                        maxDiskLastLsn = ((AbstractLSMIOOperationCallback) lsmIndex
+                                                .getIOOperationCallback())
+                                                        .getComponentLSN(lsmIndex.getImmutableComponents());
+                                    } catch (HyracksDataException e) {
+                                        datasetLifecycleManager.close(resourceAbsolutePath);
+                                        throw e;
+                                    }
+
+                                    //#. set resourceId and maxDiskLastLSN to the map
+                                    resourceId2MaxLSNMap.put(resourceId, maxDiskLastLsn);
+                                } else {
+                                    maxDiskLastLsn = resourceId2MaxLSNMap.get(resourceId);
+                                }
+
+                                if (LSN > maxDiskLastLsn) {
+                                    redo(logRecord, datasetLifecycleManager);
+                                    redoCount++;
+                                }
                             }
                         }
                         break;
@@ -594,28 +403,25 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
                     case LogType.ABORT:
                     case LogType.FLUSH:
                     case LogType.UPSERT_ENTITY_COMMIT:
-
                         //do nothing
                         break;
-
                     default:
                         throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
                 }
+                logRecord = logReader.next();
+            }
+            LOGGER.info("Logs REDO phase completed. Redo logs count: " + redoCount);
+        } finally {
+            //close all indexes
+            Set<Long> resourceIdList = resourceId2MaxLSNMap.keySet();
+            for (long r : resourceIdList) {
+                datasetLifecycleManager.close(resourcesMap.get(r).getResourcePath());
             }
         }
+    }
 
-        //close all indexes
-        Set<Long> resourceIdList = resourceId2MaxLSNMap.keySet();
-        for (long r : resourceIdList) {
-            datasetLifecycleManager.close(resourcesMap.get(r).getResourcePath());
-        }
-
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("[RecoveryMgr] remote recovery is completed.");
-            LOGGER.info("[RecoveryMgr's recovery log count] update/entityCommit/jobCommit/abort/redo = "
-                    + updateLogCount + "/" + entityCommitLogCount + "/" + jobCommitLogCount + "/" + abortLogCount + "/"
-                    + redoCount);
-        }
+    private static boolean needToFreeMemory() {
+        return Runtime.getRuntime().freeMemory() < MAX_CACHED_ENTITY_COMMITS_PER_JOB_SIZE;
     }
 
     @Override
@@ -653,7 +459,22 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
                         //get min LSN of dead replicas remote resources
                         IReplicaResourcesManager remoteResourcesManager = txnSubsystem
                                 .getAsterixAppRuntimeContextProvider().getAppContext().getReplicaResourcesManager();
-                        minMCTFirstLSN = remoteResourcesManager.getMinRemoteLSN(deadReplicaIds);
+                        IAsterixPropertiesProvider propertiesProvider = (IAsterixPropertiesProvider) txnSubsystem
+                                .getAsterixAppRuntimeContextProvider().getAppContext();
+                        AsterixMetadataProperties metadataProperties = propertiesProvider.getMetadataProperties();
+                        Set<Integer> deadReplicasPartitions = new HashSet<>();
+                        //get partitions of the dead replicas that are not active on this node
+                        for (String deadReplicaId : deadReplicaIds) {
+                            ClusterPartition[] nodePartitons = metadataProperties.getNodePartitions()
+                                    .get(deadReplicaId);
+                            for (ClusterPartition partition : nodePartitons) {
+                                if (!localResourceRepository.getActivePartitions()
+                                        .contains(partition.getPartitionId())) {
+                                    deadReplicasPartitions.add(partition.getPartitionId());
+                                }
+                            }
+                        }
+                        minMCTFirstLSN = remoteResourcesManager.getPartitionsMinLSN(deadReplicasPartitions);
                     }
                 } else {
                     //start up complete checkpoint. Avoid deleting remote recovery logs.
@@ -683,31 +504,14 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
         CheckpointObject checkpointObject = new CheckpointObject(logMgr.getAppendLSN(), minMCTFirstLSN,
                 txnMgr.getMaxJobId(), System.currentTimeMillis(), isSharpCheckpoint);
 
-        FileOutputStream fos = null;
-        ObjectOutputStream oosToFos = null;
-        try {
-            String fileName = getCheckpointFileName(logDir, Long.toString(checkpointObject.getTimeStamp()));
-            fos = new FileOutputStream(fileName);
-            oosToFos = new ObjectOutputStream(fos);
+        String fileName = getCheckpointFileName(logDir, Long.toString(checkpointObject.getTimeStamp()));
+
+        try (FileOutputStream fos = new FileOutputStream(fileName);
+                ObjectOutputStream oosToFos = new ObjectOutputStream(fos)) {
             oosToFos.writeObject(checkpointObject);
             oosToFos.flush();
         } catch (IOException e) {
             throw new ACIDException("Failed to checkpoint", e);
-        } finally {
-            if (oosToFos != null) {
-                try {
-                    oosToFos.close();
-                } catch (IOException e) {
-                    throw new ACIDException("Failed to checkpoint", e);
-                }
-            }
-            if (oosToFos == null && fos != null) {
-                try {
-                    fos.close();
-                } catch (IOException e) {
-                    throw new ACIDException("Failed to checkpoint", e);
-                }
-            }
         }
 
         //#. delete the previous checkpoint files
@@ -735,7 +539,7 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
             logMgr.deleteOldLogFiles(minMCTFirstLSN);
         }
 
-        if (isSharpCheckpoint && LOGGER.isLoggable(Level.INFO)) {
+        if (isSharpCheckpoint) {
             LOGGER.info("Completed sharp checkpoint.");
         }
 
@@ -779,15 +583,10 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
     }
 
     private long getRemoteMinFirstLSN() {
-        IAsterixPropertiesProvider propertiesProvider = (IAsterixPropertiesProvider) txnSubsystem
-                .getAsterixAppRuntimeContextProvider().getAppContext();
-
-        Set<String> replicaIds = propertiesProvider.getReplicationProperties()
-                .getRemoteReplicasIds(txnSubsystem.getId());
         IReplicaResourcesManager remoteResourcesManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getAppContext().getReplicaResourcesManager();
-
-        return remoteResourcesManager.getMinRemoteLSN(replicaIds);
+        long minRemoteLSN = remoteResourcesManager.getPartitionsMinLSN(localResourceRepository.getInactivePartitions());
+        return minRemoteLSN;
     }
 
     private CheckpointObject readCheckpoint() throws ACIDException, FileNotFoundException {
@@ -801,38 +600,19 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
 
         List<CheckpointObject> checkpointObjectList = new ArrayList<CheckpointObject>();
         for (File file : prevCheckpointFiles) {
-            FileInputStream fis = null;
-            ObjectInputStream oisFromFis = null;
-
-            try {
-                fis = new FileInputStream(file);
-                oisFromFis = new ObjectInputStream(fis);
+            try (FileInputStream fis = new FileInputStream(file);
+                    ObjectInputStream oisFromFis = new ObjectInputStream(fis)) {
                 checkpointObject = (CheckpointObject) oisFromFis.readObject();
                 checkpointObjectList.add(checkpointObject);
             } catch (Exception e) {
                 throw new ACIDException("Failed to read a checkpoint file", e);
-            } finally {
-                if (oisFromFis != null) {
-                    try {
-                        oisFromFis.close();
-                    } catch (IOException e) {
-                        throw new ACIDException("Failed to read a checkpoint file", e);
-                    }
-                }
-                if (oisFromFis == null && fis != null) {
-                    try {
-                        fis.close();
-                    } catch (IOException e) {
-                        throw new ACIDException("Failed to read a checkpoint file", e);
-                    }
-                }
             }
         }
 
-        //#. sort checkpointObjects in descending order by timeStamp to find out the most recent one.
+        //sort checkpointObjects in descending order by timeStamp to find out the most recent one.
         Collections.sort(checkpointObjectList);
 
-        //#. return the most recent one (the first one in sorted list)
+        //return the most recent one (the first one in sorted list)
         return checkpointObjectList.get(0);
     }
 
@@ -843,11 +623,7 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
         FilenameFilter filter = new FilenameFilter() {
             @Override
             public boolean accept(File dir, String name) {
-                if (name.contains(CHECKPOINT_FILENAME_PREFIX)) {
-                    return true;
-                } else {
-                    return false;
-                }
+                return name.contains(CHECKPOINT_FILENAME_PREFIX);
             }
         };
 
@@ -861,7 +637,8 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
         return baseDir + CHECKPOINT_FILENAME_PREFIX + suffix;
     }
 
-    private File createJobRecoveryFile(int jobId, String fileName) throws IOException {
+    @Override
+    public File createJobRecoveryFile(int jobId, String fileName) throws IOException {
         String recoveryDirPath = getRecoveryDirPath();
         Path JobRecoveryFolder = Paths.get(recoveryDirPath + File.separator + jobId);
         if (!Files.exists(JobRecoveryFolder)) {
@@ -878,12 +655,11 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
         return jobRecoveryFile;
     }
 
-    private void deleteRecoveryTemporaryFiles() throws IOException {
+    @Override
+    public void deleteRecoveryTemporaryFiles() {
         String recoveryDirPath = getRecoveryDirPath();
         Path recoveryFolderPath = Paths.get(recoveryDirPath);
-        if (Files.exists(recoveryFolderPath)) {
-            FileUtils.deleteDirectory(recoveryFolderPath.toFile());
-        }
+        FileUtils.deleteQuietly(recoveryFolderPath.toFile());
     }
 
     private String getRecoveryDirPath() {
@@ -906,11 +682,6 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
         }
     }
 
-    /**
-     * Rollback a transaction
-     *
-     * @see org.apache.transaction.management.service.recovery.IRecoveryManager# rollbackTransaction (org.apache.TransactionContext.management.service.transaction .TransactionContext)
-     */
     @Override
     public void rollbackTransaction(ITransactionContext txnContext) throws ACIDException {
         int abortedJobId = txnContext.getJobId().getId();
@@ -937,7 +708,8 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
         long currentLSN = -1;
         TxnId loserEntity = null;
         List<Long> undoLSNSet = null;
-        String nodeId = logMgr.getNodeId();
+        //get active partitions on this node
+        Set<Integer> activePartitions = localResourceRepository.getActivePartitions();
         ILogReader logReader = logMgr.getLogReader(false);
         try {
             logReader.initializeScan(firstLSN);
@@ -953,15 +725,15 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
                         LOGGER.info(logRecord.getLogRecordForDisplay());
                     }
                 }
-                if (logRecord.getNodeId().equals(nodeId)) {
-                    logJobId = logRecord.getJobId();
-                    if (logJobId != abortedJobId) {
-                        continue;
-                    }
-                    tempKeyTxnId.setTxnId(logJobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
-                            logRecord.getPKValue(), logRecord.getPKValueSize());
-                    switch (logRecord.getLogType()) {
-                        case LogType.UPDATE:
+                logJobId = logRecord.getJobId();
+                if (logJobId != abortedJobId) {
+                    continue;
+                }
+                tempKeyTxnId.setTxnId(logJobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+                        logRecord.getPKValue(), logRecord.getPKValueSize());
+                switch (logRecord.getLogType()) {
+                    case LogType.UPDATE:
+                        if (activePartitions.contains(logRecord.getResourcePartition())) {
                             undoLSNSet = jobLoserEntity2LSNsMap.get(tempKeyTxnId);
                             if (undoLSNSet == null) {
                                 loserEntity = new TxnId(logJobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
@@ -975,27 +747,30 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
                                 LOGGER.info(Thread.currentThread().getId() + "======> update[" + currentLSN + "]:"
                                         + tempKeyTxnId);
                             }
-                            break;
-                        case LogType.ENTITY_COMMIT:
-                        case LogType.UPSERT_ENTITY_COMMIT:
+                        }
+                        break;
+                    case LogType.ENTITY_COMMIT:
+                    case LogType.UPSERT_ENTITY_COMMIT:
+                        if (activePartitions.contains(logRecord.getResourcePartition())) {
                             jobLoserEntity2LSNsMap.remove(tempKeyTxnId);
                             entityCommitLogCount++;
                             if (IS_DEBUG_MODE) {
                                 LOGGER.info(Thread.currentThread().getId() + "======> entity_commit[" + currentLSN + "]"
                                         + tempKeyTxnId);
                             }
-                            break;
-                        case LogType.JOB_COMMIT:
-                            throw new ACIDException("Unexpected LogType(" + logRecord.getLogType() + ") during abort.");
-                        case LogType.ABORT:
-                        case LogType.FLUSH:
-                            //ignore
-                            break;
-                        default:
-                            throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
-                    }
+                        }
+                        break;
+                    case LogType.JOB_COMMIT:
+                        throw new ACIDException("Unexpected LogType(" + logRecord.getLogType() + ") during abort.");
+                    case LogType.ABORT:
+                    case LogType.FLUSH:
+                        //ignore
+                        break;
+                    default:
+                        throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
                 }
             }
+
             if (currentLSN != lastLSN) {
                 throw new ACIDException("LastLSN mismatch: lastLSN(" + lastLSN + ") vs currentLSN(" + currentLSN
                         + ") during abort( " + txnContext.getJobId() + ")");
@@ -1034,6 +809,7 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
                 LOGGER.info("[RecoveryManager's rollback log count] update/entityCommit/undo:" + updateLogCount + "/"
                         + entityCommitLogCount + "/" + undoCount);
             }
+
         } finally {
             logReader.close();
         }
@@ -1096,242 +872,6 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
         }
     }
 
-    //TODO (mhubail) RecoveryManager has three methods that perform logs REDO based on different parameters.
-    //They need to be refactored to use partitions only once the log format includes partition id.
-    @Override
-    public synchronized void replayPartitionsLogs(Integer[] partitions, long lowWaterMarkLSN, String failedNode)
-            throws IOException, ACIDException {
-        //delete any recovery files from previous failed recovery attempts
-        deleteRecoveryTemporaryFiles();
-
-        int updateLogCount = 0;
-        int entityCommitLogCount = 0;
-        int jobCommitLogCount = 0;
-        int redoCount = 0;
-        int abortLogCount = 0;
-        int jobId = -1;
-
-        state = SystemState.RECOVERING;
-        LOGGER.log(Level.INFO, "[RecoveryMgr] starting recovery ...");
-
-        Set<Integer> winnerJobSet = new HashSet<Integer>();
-        jobId2WinnerEntitiesMap = new HashMap<>();
-
-        TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
-        JobEntityCommits jobEntityWinners = null;
-        //#. read checkpoint file and set lowWaterMark where anaylsis and redo start
-        long readableSmallestLSN = logMgr.getReadableSmallestLSN();
-        if (lowWaterMarkLSN < readableSmallestLSN) {
-            lowWaterMarkLSN = readableSmallestLSN;
-        }
-        //-------------------------------------------------------------------------
-        //  [ analysis phase ]
-        //  - collect all committed Lsn
-        //-------------------------------------------------------------------------
-        LOGGER.log(Level.INFO, "[RecoveryMgr] in analysis phase");
-        IAsterixAppRuntimeContextProvider appRuntimeContext = txnSubsystem.getAsterixAppRuntimeContextProvider();
-        //get datasetLifeCycleManager
-        IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
-                .getDatasetLifecycleManager();
-        PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) appRuntimeContext
-                .getLocalResourceRepository();
-        Map<Long, LocalResource> resourcesMap = localResourceRepository.loadAndGetAllResources();
-        Map<Long, Long> resourceId2MaxLSNMap = new HashMap<Long, Long>();
-
-        //#. set log reader to the lowWaterMarkLsn
-        ILogReader logReader = logMgr.getLogReader(true);
-        ILogRecord logRecord = null;
-        try {
-            logReader.initializeScan(lowWaterMarkLSN);
-            logRecord = logReader.next();
-            while (logRecord != null) {
-                if (IS_DEBUG_MODE) {
-                    LOGGER.info(logRecord.getLogRecordForDisplay());
-                }
-                //TODO update this partitions once the log format is updated to include partitons
-                if (logRecord.getNodeId().equals(failedNode)) {
-                    switch (logRecord.getLogType()) {
-                        case LogType.UPDATE:
-                            updateLogCount++;
-                            break;
-                        case LogType.JOB_COMMIT:
-                            jobId = logRecord.getJobId();
-                            winnerJobSet.add(jobId);
-                            if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
-                                jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
-                                //to delete any spilled files as well
-                                jobEntityWinners.clear();
-                                jobId2WinnerEntitiesMap.remove(jobId);
-                            }
-                            jobCommitLogCount++;
-                            break;
-                        case LogType.ENTITY_COMMIT:
-                            jobId = logRecord.getJobId();
-                            if (!jobId2WinnerEntitiesMap.containsKey(jobId)) {
-                                jobEntityWinners = new JobEntityCommits(jobId);
-                                if (needToFreeMemory()) {
-                                    //if we don't have enough memory for one more job, we will force all jobs to spill their cached entities to disk.
-                                    //This could happen only when we have many jobs with small number of records and none of them have job commit.
-                                    freeJobsCachedEntities(jobId);
-                                }
-                                jobId2WinnerEntitiesMap.put(jobId, jobEntityWinners);
-                            } else {
-                                jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
-                            }
-                            jobEntityWinners.add(logRecord);
-                            entityCommitLogCount++;
-                            break;
-                        case LogType.ABORT:
-                            abortLogCount++;
-                            break;
-                        case LogType.FLUSH:
-                            break;
-                        default:
-                            throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
-                    }
-                }
-                logRecord = logReader.next();
-            }
-
-            //prepare winners for search after analysis is done to flush anything remaining in memory to disk.
-            for (JobEntityCommits winners : jobId2WinnerEntitiesMap.values()) {
-                winners.prepareForSearch();
-            }
-            //-------------------------------------------------------------------------
-            //  [ redo phase ]
-            //  - redo if
-            //    1) The TxnId is committed && --> guarantee durability
-            //    2) lsn > maxDiskLastLsn of the index --> guarantee idempotence
-            //-------------------------------------------------------------------------
-            LOGGER.info("[RecoveryMgr] in redo phase");
-
-            long resourceId;
-            long maxDiskLastLsn;
-            long LSN = -1;
-            ILSMIndex index = null;
-            LocalResource localResource = null;
-            ILocalResourceMetadata localResourceMetadata = null;
-            boolean foundWinner = false;
-            //set log reader to the lowWaterMarkLsn again.
-            logReader.initializeScan(lowWaterMarkLSN);
-            logRecord = logReader.next();
-            while (logRecord != null) {
-                if (IS_DEBUG_MODE) {
-                    LOGGER.info(logRecord.getLogRecordForDisplay());
-                }
-                //TODO update this to check for partitions instead of node id once the log format is updated to include partitions
-                if (logRecord.getNodeId().equals(failedNode)) {
-                    LSN = logRecord.getLSN();
-                    jobId = logRecord.getJobId();
-                    foundWinner = false;
-                    switch (logRecord.getLogType()) {
-                        case LogType.UPDATE:
-                            if (winnerJobSet.contains(jobId)) {
-                                foundWinner = true;
-                            } else if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
-                                jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
-                                tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
-                                        logRecord.getPKValue(), logRecord.getPKValueSize());
-                                if (jobEntityWinners.containsEntityCommitForTxnId(LSN, tempKeyTxnId)) {
-                                    foundWinner = true;
-                                }
-                            }
-                            if (foundWinner) {
-                                resourceId = logRecord.getResourceId();
-                                localResource = resourcesMap.get(resourceId);
-                                /*******************************************************************
-                                 * [Notice]
-                                 * -> Issue
-                                 * Delete index may cause a problem during redo.
-                                 * The index operation to be redone couldn't be redone because the corresponding index
-                                 * may not exist in NC due to the possible index drop DDL operation.
-                                 * -> Approach
-                                 * Avoid the problem during redo.
-                                 * More specifically, the problem will be detected when the localResource of
-                                 * the corresponding index is retrieved, which will end up with 'null'.
-                                 * If null is returned, then just go and process the next
-                                 * log record.
-                                 *******************************************************************/
-                                if (localResource == null) {
-                                    logRecord = logReader.next();
-                                    continue;
-                                }
-                                /*******************************************************************/
-
-                                //get index instance from IndexLifeCycleManager
-                                //if index is not registered into IndexLifeCycleManager,
-                                //create the index using LocalMetadata stored in LocalResourceRepository
-                                //get partition path in this node
-                                String partitionIODevicePath = localResourceRepository
-                                        .getPartitionPath(localResource.getPartition());
-                                String resourceAbsolutePath = partitionIODevicePath + File.separator
-                                        + localResource.getResourceName();
-                                localResource.setResourcePath(resourceAbsolutePath);
-                                index = (ILSMIndex) datasetLifecycleManager.getIndex(resourceAbsolutePath);
-                                if (index == null) {
-                                    //#. create index instance and register to indexLifeCycleManager
-                                    localResourceMetadata = (ILocalResourceMetadata) localResource.getResourceObject();
-                                    index = localResourceMetadata.createIndexInstance(appRuntimeContext,
-                                            resourceAbsolutePath, localResource.getPartition());
-                                    datasetLifecycleManager.register(resourceAbsolutePath, index);
-                                    datasetLifecycleManager.open(resourceAbsolutePath);
-
-                                    //#. get maxDiskLastLSN
-                                    ILSMIndex lsmIndex = index;
-                                    try {
-                                        maxDiskLastLsn = ((AbstractLSMIOOperationCallback) lsmIndex
-                                                .getIOOperationCallback())
-                                                        .getComponentLSN(lsmIndex.getImmutableComponents());
-                                    } catch (HyracksDataException e) {
-                                        datasetLifecycleManager.close(resourceAbsolutePath);
-                                        throw e;
-                                    }
-
-                                    //#. set resourceId and maxDiskLastLSN to the map
-                                    resourceId2MaxLSNMap.put(resourceId, maxDiskLastLsn);
-                                } else {
-                                    maxDiskLastLsn = resourceId2MaxLSNMap.get(resourceId);
-                                }
-
-                                if (LSN > maxDiskLastLsn) {
-                                    redo(logRecord, datasetLifecycleManager);
-                                    redoCount++;
-                                }
-                            }
-                            break;
-                        case LogType.JOB_COMMIT:
-                        case LogType.ENTITY_COMMIT:
-                        case LogType.ABORT:
-                        case LogType.FLUSH:
-                            //do nothing
-                            break;
-                        default:
-                            throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
-                    }
-                }
-                logRecord = logReader.next();
-            }
-
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("[RecoveryMgr] recovery is completed.");
-                LOGGER.info("[RecoveryMgr's recovery log count] update/entityCommit/jobCommit/abort/redo = "
-                        + updateLogCount + "/" + entityCommitLogCount + "/" + jobCommitLogCount + "/" + abortLogCount
-                        + "/" + redoCount);
-            }
-        } finally {
-            logReader.close();
-
-            //close all indexes
-            Set<Long> resourceIdList = resourceId2MaxLSNMap.keySet();
-            for (long r : resourceIdList) {
-                datasetLifecycleManager.close(resourcesMap.get(r).getResourcePath());
-            }
-
-            //delete any recovery files after completing recovery
-            deleteRecoveryTemporaryFiles();
-        }
-    }
-
     private class JobEntityCommits {
         private static final String PARTITION_FILE_NAME_SEPARATOR = "_";
         private final int jobId;
@@ -1499,153 +1039,4 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
             }
         }
     }
-}
-
-class TxnId {
-    public boolean isByteArrayPKValue;
-    public int jobId;
-    public int datasetId;
-    public int pkHashValue;
-    public int pkSize;
-    public byte[] byteArrayPKValue;
-    public ITupleReference tupleReferencePKValue;
-
-    public TxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize,
-            boolean isByteArrayPKValue) {
-        this.jobId = jobId;
-        this.datasetId = datasetId;
-        this.pkHashValue = pkHashValue;
-        this.pkSize = pkSize;
-        this.isByteArrayPKValue = isByteArrayPKValue;
-        if (isByteArrayPKValue) {
-            this.byteArrayPKValue = new byte[pkSize];
-            readPKValueIntoByteArray(pkValue, pkSize, byteArrayPKValue);
-        } else {
-            this.tupleReferencePKValue = pkValue;
-        }
-    }
-
-    public TxnId() {
-    }
-
-    private static void readPKValueIntoByteArray(ITupleReference pkValue, int pkSize, byte[] byteArrayPKValue) {
-        int readOffset = pkValue.getFieldStart(0);
-        byte[] readBuffer = pkValue.getFieldData(0);
-        for (int i = 0; i < pkSize; i++) {
-            byteArrayPKValue[i] = readBuffer[readOffset + i];
-        }
-    }
-
-    public void setTxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize) {
-        this.jobId = jobId;
-        this.datasetId = datasetId;
-        this.pkHashValue = pkHashValue;
-        this.tupleReferencePKValue = pkValue;
-        this.pkSize = pkSize;
-        this.isByteArrayPKValue = false;
-    }
-
-    @Override
-    public String toString() {
-        return "[" + jobId + "," + datasetId + "," + pkHashValue + "," + pkSize + "]";
-    }
-
-    @Override
-    public int hashCode() {
-        return pkHashValue;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (o == this) {
-            return true;
-        }
-        if (!(o instanceof TxnId)) {
-            return false;
-        }
-        TxnId txnId = (TxnId) o;
-        return (txnId.pkHashValue == pkHashValue && txnId.datasetId == datasetId && txnId.jobId == jobId
-                && pkSize == txnId.pkSize && isEqualTo(txnId));
-    }
-
-    private boolean isEqualTo(TxnId txnId) {
-        if (isByteArrayPKValue && txnId.isByteArrayPKValue) {
-            return isEqual(byteArrayPKValue, txnId.byteArrayPKValue, pkSize);
-        } else if (isByteArrayPKValue && (!txnId.isByteArrayPKValue)) {
-            return isEqual(byteArrayPKValue, txnId.tupleReferencePKValue, pkSize);
-        } else if ((!isByteArrayPKValue) && txnId.isByteArrayPKValue) {
-            return isEqual(txnId.byteArrayPKValue, tupleReferencePKValue, pkSize);
-        } else {
-            return isEqual(tupleReferencePKValue, txnId.tupleReferencePKValue, pkSize);
-        }
-    }
-
-    private static boolean isEqual(byte[] a, byte[] b, int size) {
-        for (int i = 0; i < size; i++) {
-            if (a[i] != b[i]) {
-                return false;
-            }
-        }
-        return true;
-    }
-
-    private static boolean isEqual(byte[] a, ITupleReference b, int size) {
-        int readOffset = b.getFieldStart(0);
-        byte[] readBuffer = b.getFieldData(0);
-        for (int i = 0; i < size; i++) {
-            if (a[i] != readBuffer[readOffset + i]) {
-                return false;
-            }
-        }
-        return true;
-    }
-
-    private static boolean isEqual(ITupleReference a, ITupleReference b, int size) {
-        int aOffset = a.getFieldStart(0);
-        byte[] aBuffer = a.getFieldData(0);
-        int bOffset = b.getFieldStart(0);
-        byte[] bBuffer = b.getFieldData(0);
-        for (int i = 0; i < size; i++) {
-            if (aBuffer[aOffset + i] != bBuffer[bOffset + i]) {
-                return false;
-            }
-        }
-        return true;
-    }
-
-    public void serialize(ByteBuffer buffer) {
-        buffer.putInt(jobId);
-        buffer.putInt(datasetId);
-        buffer.putInt(pkHashValue);
-        buffer.putInt(pkSize);
-        buffer.put((byte) (isByteArrayPKValue ? 1 : 0));
-        if (isByteArrayPKValue) {
-            buffer.put(byteArrayPKValue);
-        }
-    }
-
-    public static TxnId deserialize(ByteBuffer buffer) {
-        TxnId txnId = new TxnId();
-        txnId.jobId = buffer.getInt();
-        txnId.datasetId = buffer.getInt();
-        txnId.pkHashValue = buffer.getInt();
-        txnId.pkSize = buffer.getInt();
-        txnId.isByteArrayPKValue = (buffer.get() == 1);
-        if (txnId.isByteArrayPKValue) {
-            byte[] byteArrayPKValue = new byte[txnId.pkSize];
-            buffer.get(byteArrayPKValue);
-            txnId.byteArrayPKValue = byteArrayPKValue;
-        }
-        return txnId;
-    }
-
-    public int getCurrentSize() {
-        //job id, dataset id, pkHashValue, arraySize, isByteArrayPKValue
-        int size = JobId.BYTES + DatasetId.BYTES + LogRecord.PKHASH_LEN + LogRecord.PKSZ_LEN + Byte.BYTES;
-        //byte arraySize
-        if (isByteArrayPKValue && byteArrayPKValue != null) {
-            size += byteArrayPKValue.length;
-        }
-        return size;
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/TxnId.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/TxnId.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/TxnId.java
new file mode 100644
index 0000000..bd4a49a
--- /dev/null
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/TxnId.java
@@ -0,0 +1,175 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import org.apache.asterix.common.transactions.DatasetId;
+import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.LogRecord;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class TxnId {
+    public boolean isByteArrayPKValue;
+    public int jobId;
+    public int datasetId;
+    public int pkHashValue;
+    public int pkSize;
+    public byte[] byteArrayPKValue;
+    public ITupleReference tupleReferencePKValue;
+
+    public TxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize,
+            boolean isByteArrayPKValue) {
+        this.jobId = jobId;
+        this.datasetId = datasetId;
+        this.pkHashValue = pkHashValue;
+        this.pkSize = pkSize;
+        this.isByteArrayPKValue = isByteArrayPKValue;
+        if (isByteArrayPKValue) {
+            this.byteArrayPKValue = new byte[pkSize];
+            readPKValueIntoByteArray(pkValue, pkSize, byteArrayPKValue);
+        } else {
+            this.tupleReferencePKValue = pkValue;
+        }
+    }
+
+    public TxnId() {
+    }
+
+    private static void readPKValueIntoByteArray(ITupleReference pkValue, int pkSize, byte[] byteArrayPKValue) {
+        int readOffset = pkValue.getFieldStart(0);
+        byte[] readBuffer = pkValue.getFieldData(0);
+        for (int i = 0; i < pkSize; i++) {
+            byteArrayPKValue[i] = readBuffer[readOffset + i];
+        }
+    }
+
+    public void setTxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize) {
+        this.jobId = jobId;
+        this.datasetId = datasetId;
+        this.pkHashValue = pkHashValue;
+        this.tupleReferencePKValue = pkValue;
+        this.pkSize = pkSize;
+        this.isByteArrayPKValue = false;
+    }
+
+    @Override
+    public String toString() {
+        return "[" + jobId + "," + datasetId + "," + pkHashValue + "," + pkSize + "]";
+    }
+
+    @Override
+    public int hashCode() {
+        return pkHashValue;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof TxnId)) {
+            return false;
+        }
+        TxnId txnId = (TxnId) o;
+        return (txnId.pkHashValue == pkHashValue && txnId.datasetId == datasetId && txnId.jobId == jobId
+                && pkSize == txnId.pkSize && isEqualTo(txnId));
+    }
+
+    private boolean isEqualTo(TxnId txnId) {
+        if (isByteArrayPKValue && txnId.isByteArrayPKValue) {
+            return isEqual(byteArrayPKValue, txnId.byteArrayPKValue, pkSize);
+        } else if (isByteArrayPKValue && (!txnId.isByteArrayPKValue)) {
+            return isEqual(byteArrayPKValue, txnId.tupleReferencePKValue, pkSize);
+        } else if ((!isByteArrayPKValue) && txnId.isByteArrayPKValue) {
+            return isEqual(txnId.byteArrayPKValue, tupleReferencePKValue, pkSize);
+        } else {
+            return isEqual(tupleReferencePKValue, txnId.tupleReferencePKValue, pkSize);
+        }
+    }
+
+    private static boolean isEqual(byte[] a, byte[] b, int size) {
+        for (int i = 0; i < size; i++) {
+            if (a[i] != b[i]) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private static boolean isEqual(byte[] a, ITupleReference b, int size) {
+        int readOffset = b.getFieldStart(0);
+        byte[] readBuffer = b.getFieldData(0);
+        for (int i = 0; i < size; i++) {
+            if (a[i] != readBuffer[readOffset + i]) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private static boolean isEqual(ITupleReference a, ITupleReference b, int size) {
+        int aOffset = a.getFieldStart(0);
+        byte[] aBuffer = a.getFieldData(0);
+        int bOffset = b.getFieldStart(0);
+        byte[] bBuffer = b.getFieldData(0);
+        for (int i = 0; i < size; i++) {
+            if (aBuffer[aOffset + i] != bBuffer[bOffset + i]) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    public void serialize(ByteBuffer buffer) {
+        buffer.putInt(jobId);
+        buffer.putInt(datasetId);
+        buffer.putInt(pkHashValue);
+        buffer.putInt(pkSize);
+        buffer.put((byte) (isByteArrayPKValue ? 1 : 0));
+        if (isByteArrayPKValue) {
+            buffer.put(byteArrayPKValue);
+        }
+    }
+
+    public static TxnId deserialize(ByteBuffer buffer) {
+        TxnId txnId = new TxnId();
+        txnId.jobId = buffer.getInt();
+        txnId.datasetId = buffer.getInt();
+        txnId.pkHashValue = buffer.getInt();
+        txnId.pkSize = buffer.getInt();
+        txnId.isByteArrayPKValue = (buffer.get() == 1);
+        if (txnId.isByteArrayPKValue) {
+            byte[] byteArrayPKValue = new byte[txnId.pkSize];
+            buffer.get(byteArrayPKValue);
+            txnId.byteArrayPKValue = byteArrayPKValue;
+        }
+        return txnId;
+    }
+
+    public int getCurrentSize() {
+        //job id, dataset id, pkHashValue, arraySize, isByteArrayPKValue
+        int size = JobId.BYTES + DatasetId.BYTES + LogRecord.PKHASH_LEN + LogRecord.PKSZ_LEN + Byte.BYTES;
+        //byte arraySize
+        if (isByteArrayPKValue && byteArrayPKValue != null) {
+            size += byteArrayPKValue.length;
+        }
+        return size;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
index 47a92cb..215eb14 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -110,7 +110,6 @@ public class TransactionContext implements ITransactionContext, Serializable {
         primaryIndex = null;
         tempResourceIdForRegister = new MutableLong();
         logRecord = new LogRecord();
-        logRecord.setNodeId(transactionSubsystem.getId());
         transactorNumActiveOperations = new AtomicInteger(0);
     }
 


[3/6] incubator-asterixdb git commit: Asterix NCs Failback Support

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
----------------------------------------------------------------------
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
index e4d94b4..b9447af 100644
--- a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
@@ -30,6 +30,7 @@ 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.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -52,7 +53,6 @@ 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.IReplicationThread;
-import org.apache.asterix.common.replication.Replica;
 import org.apache.asterix.common.replication.ReplicaEvent;
 import org.apache.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import org.apache.asterix.common.transactions.ILogManager;
@@ -72,6 +72,7 @@ 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.hyracks.api.application.INCApplicationContext;
 import org.apache.hyracks.storage.am.common.api.IMetaDataPageManager;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -97,6 +98,7 @@ public class ReplicationChannel extends Thread implements IReplicationChannel {
     private final Map<String, LSMComponentProperties> lsmComponentId2PropertiesMap;
     private final Map<Long, RemoteLogMapping> localLSN2RemoteLSNMap;
     private final LSMComponentsSyncService lsmComponentLSNMappingService;
+    private final Set<Integer> nodeHostedPartitions;
 
     public ReplicationChannel(String nodeId, AsterixReplicationProperties replicationProperties, ILogManager logManager,
             IReplicaResourcesManager replicaResoucesManager, IReplicationManager replicationManager,
@@ -112,6 +114,17 @@ public class ReplicationChannel extends Thread implements IReplicationChannel {
         localLSN2RemoteLSNMap = new ConcurrentHashMap<Long, RemoteLogMapping>();
         lsmComponentLSNMappingService = new LSMComponentsSyncService();
         replicationThreads = Executors.newCachedThreadPool(appContext.getThreadFactory());
+        Map<String, ClusterPartition[]> nodePartitions = ((IAsterixPropertiesProvider) asterixAppRuntimeContextProvider
+                .getAppContext()).getMetadataProperties().getNodePartitions();
+        Set<String> nodeReplicationClients = replicationProperties.getNodeReplicationClients(nodeId);
+        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);
     }
 
     @Override
@@ -193,6 +206,18 @@ public class ReplicationChannel extends Thread implements IReplicationChannel {
         }
     }
 
+    private static void sendRemoteRecoveryLog(ILogRecord logRecord, SocketChannel socketChannel, ByteBuffer outBuffer)
+            throws IOException {
+        logRecord.setLogSource(LogSource.REMOTE_RECOVERY);
+        if (logRecord.getSerializedLogSize() > outBuffer.capacity()) {
+            int requestSize = logRecord.getSerializedLogSize() + ReplicationProtocol.REPLICATION_REQUEST_HEADER_SIZE;
+            outBuffer = ByteBuffer.allocate(requestSize);
+        }
+        //set log source to REMOTE_RECOVERY to avoid re-logging on the recipient side
+        ReplicationProtocol.writeRemoteRecoveryLogRequest(outBuffer, logRecord);
+        NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
+    }
+
     /**
      * A replication thread is created per received replication request.
      */
@@ -232,9 +257,6 @@ public class ReplicationChannel extends Thread implements IReplicationChannel {
                         case REPLICA_EVENT:
                             handleReplicaEvent();
                             break;
-                        case UPDATE_REPLICA:
-                            handleUpdateReplica();
-                            break;
                         case GET_REPLICA_MAX_LSN:
                             handleGetReplicaMaxLSN();
                             break;
@@ -379,29 +401,33 @@ public class ReplicationChannel extends Thread implements IReplicationChannel {
 
             List<String> filesList;
             Set<String> replicaIds = request.getReplicaIds();
+            Set<String> requesterExistingFiles = request.getExistingFiles();
             Map<String, ClusterPartition[]> nodePartitions = ((IAsterixPropertiesProvider) asterixAppRuntimeContextProvider
                     .getAppContext()).getMetadataProperties().getNodePartitions();
             for (String replicaId : replicaIds) {
                 //get replica partitions
                 ClusterPartition[] replicaPatitions = nodePartitions.get(replicaId);
                 for (ClusterPartition partition : replicaPatitions) {
-                    filesList = replicaResourcesManager.getPartitionIndexesFiles(partition.getPartitionId());
-
+                    filesList = replicaResourcesManager.getPartitionIndexesFiles(partition.getPartitionId(), false);
                     //start sending files
                     for (String filePath : filesList) {
-                        try (RandomAccessFile fromFile = new RandomAccessFile(filePath, "r");
-                                FileChannel fileChannel = fromFile.getChannel();) {
-                            long fileSize = fileChannel.size();
-                            fileProperties.initialize(filePath, fileSize, replicaId, false,
-                                    IMetaDataPageManager.INVALID_LSN_OFFSET, false);
-                            outBuffer = ReplicationProtocol.writeFileReplicationRequest(outBuffer, fileProperties,
-                                    ReplicationRequestType.REPLICATE_FILE);
-
-                            //send file info
-                            NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
-
-                            //transfer file
-                            NetworkingUtil.sendFile(fileChannel, socketChannel);
+                        String relativeFilePath = PersistentLocalResourceRepository.getResourceRelativePath(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, replicaId, false,
+                                        IMetaDataPageManager.INVALID_LSN_OFFSET, false);
+                                outBuffer = ReplicationProtocol.writeFileReplicationRequest(outBuffer, fileProperties,
+                                        ReplicationRequestType.REPLICATE_FILE);
+
+                                //send file info
+                                NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
+
+                                //transfer file
+                                NetworkingUtil.sendFile(fileChannel, socketChannel);
+                            }
                         }
                     }
                 }
@@ -416,11 +442,23 @@ public class ReplicationChannel extends Thread implements IReplicationChannel {
             ReplicaLogsRequest request = ReplicationProtocol.readReplicaLogsRequest(inBuffer);
 
             Set<String> replicaIds = request.getReplicaIds();
+            //get list of partitions that belong to the replicas in the request
+            Set<Integer> requestedPartitions = new HashSet<>();
+            Map<String, ClusterPartition[]> nodePartitions = ((IAsterixPropertiesProvider) asterixAppRuntimeContextProvider
+                    .getAppContext()).getMetadataProperties().getNodePartitions();
+            for (String replicaId : replicaIds) {
+                //get replica partitions
+                ClusterPartition[] replicaPatitions = nodePartitions.get(replicaId);
+                for (ClusterPartition partition : replicaPatitions) {
+                    requestedPartitions.add(partition.getPartitionId());
+                }
+            }
+
             long fromLSN = request.getFromLSN();
             long minLocalFirstLSN = asterixAppRuntimeContextProvider.getAppContext().getTransactionSubsystem()
                     .getRecoveryManager().getLocalMinFirstLSN();
 
-            //get Log read
+            //get Log reader
             ILogReader logReader = logManager.getLogReader(true);
             try {
                 if (fromLSN < logManager.getReadableSmallestLSN()) {
@@ -429,25 +467,34 @@ public class ReplicationChannel extends Thread implements IReplicationChannel {
 
                 logReader.initializeScan(fromLSN);
                 ILogRecord logRecord = logReader.next();
+                Set<Integer> requestedPartitionsJobs = new HashSet<>();
                 while (logRecord != null) {
                     //we should not send any local log which has already been converted to disk component
                     if (logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLSN() < minLocalFirstLSN) {
                         logRecord = logReader.next();
                         continue;
                     }
-
-                    //since flush logs are not required for recovery, skip them
-                    if (replicaIds.contains(logRecord.getNodeId()) && logRecord.getLogType() != LogType.FLUSH) {
-                        if (logRecord.getSerializedLogSize() > outBuffer.capacity()) {
-                            int requestSize = logRecord.getSerializedLogSize()
-                                    + ReplicationProtocol.REPLICATION_REQUEST_HEADER_SIZE;
-                            outBuffer = ByteBuffer.allocate(requestSize);
-                        }
-
-                        //set log source to REMOTE_RECOVERY to avoid re-logging on the recipient side
-                        logRecord.setLogSource(LogSource.REMOTE_RECOVERY);
-                        ReplicationProtocol.writeRemoteRecoveryLogRequest(outBuffer, logRecord);
-                        NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
+                    //send only logs that belong to the partitions of the request and required for recovery
+                    switch (logRecord.getLogType()) {
+                        case LogType.UPDATE:
+                        case LogType.ENTITY_COMMIT:
+                        case LogType.UPSERT_ENTITY_COMMIT:
+                            if (requestedPartitions.contains(logRecord.getResourcePartition())) {
+                                sendRemoteRecoveryLog(logRecord, socketChannel, outBuffer);
+                                requestedPartitionsJobs.add(logRecord.getJobId());
+                            }
+                            break;
+                        case LogType.JOB_COMMIT:
+                            if (requestedPartitionsJobs.contains(logRecord.getJobId())) {
+                                sendRemoteRecoveryLog(logRecord, socketChannel, outBuffer);
+                                requestedPartitionsJobs.remove(logRecord.getJobId());
+                            }
+                            break;
+                        case LogType.ABORT:
+                        case LogType.FLUSH:
+                            break;
+                        default:
+                            throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
                     }
                     logRecord = logReader.next();
                 }
@@ -459,12 +506,6 @@ public class ReplicationChannel extends Thread implements IReplicationChannel {
             ReplicationProtocol.sendGoodbye(socketChannel);
         }
 
-        private void handleUpdateReplica() throws IOException {
-            inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
-            Replica replica = ReplicationProtocol.readReplicaUpdateRequest(inBuffer);
-            replicationManager.updateReplicaInfo(replica);
-        }
-
         private void handleReplicaEvent() throws IOException {
             inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
             ReplicaEvent event = ReplicationProtocol.readReplicaEventRequest(inBuffer);
@@ -484,37 +525,45 @@ public class ReplicationChannel extends Thread implements IReplicationChannel {
             inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
 
             //Deserialize log
-            remoteLog.readRemoteLog(inBuffer, false, localNodeID);
+            remoteLog.readRemoteLog(inBuffer, false);
             remoteLog.setLogSource(LogSource.REMOTE);
 
-            if (remoteLog.getLogType() == LogType.JOB_COMMIT) {
-                LogRecord jobCommitLog = new LogRecord();
-                TransactionUtil.formJobTerminateLogRecord(jobCommitLog, remoteLog.getJobId(), true,
-                        remoteLog.getNodeId());
-                jobCommitLog.setReplicationThread(this);
-                jobCommitLog.setLogSource(LogSource.REMOTE);
-                logManager.log(jobCommitLog);
-            } else if (remoteLog.getLogType() == LogType.FLUSH) {
-                LogRecord flushLog = new LogRecord();
-                TransactionUtil.formFlushLogRecord(flushLog, remoteLog.getDatasetId(), null, remoteLog.getNodeId(),
-                        remoteLog.getNumOfFlushedIndexes());
-                flushLog.setReplicationThread(this);
-                flushLog.setLogSource(LogSource.REMOTE);
-                synchronized (localLSN2RemoteLSNMap) {
-                    logManager.log(flushLog);
-
-                    //store mapping information for flush logs to use them in incoming LSM components.
-                    RemoteLogMapping flushLogMap = new RemoteLogMapping();
-                    flushLogMap.setRemoteLSN(remoteLog.getLSN());
-                    flushLogMap.setRemoteNodeID(remoteLog.getNodeId());
-                    flushLogMap.setLocalLSN(flushLog.getLSN());
-                    flushLogMap.numOfFlushedIndexes.set(remoteLog.getNumOfFlushedIndexes());
-                    localLSN2RemoteLSNMap.put(flushLog.getLSN(), flushLogMap);
-                    localLSN2RemoteLSNMap.notifyAll();
-                }
-            } else {
-                //send log to LogManager as a remote log
-                logManager.log(remoteLog);
+            switch (remoteLog.getLogType()) {
+                case LogType.UPDATE:
+                case LogType.ENTITY_COMMIT:
+                case LogType.UPSERT_ENTITY_COMMIT:
+                    //if the log partition belongs to a partitions hosted on this node, replicate it
+                    if (nodeHostedPartitions.contains(remoteLog.getResourcePartition())) {
+                        logManager.log(remoteLog);
+                    }
+                    break;
+                case LogType.JOB_COMMIT:
+                    LogRecord jobCommitLog = new LogRecord();
+                    TransactionUtil.formJobTerminateLogRecord(jobCommitLog, remoteLog.getJobId(), true);
+                    jobCommitLog.setReplicationThread(this);
+                    jobCommitLog.setLogSource(LogSource.REMOTE);
+                    logManager.log(jobCommitLog);
+                    break;
+                case LogType.FLUSH:
+                    LogRecord flushLog = new LogRecord();
+                    TransactionUtil.formFlushLogRecord(flushLog, remoteLog.getDatasetId(), null, remoteLog.getNodeId(),
+                            remoteLog.getNumOfFlushedIndexes());
+                    flushLog.setReplicationThread(this);
+                    flushLog.setLogSource(LogSource.REMOTE);
+                    synchronized (localLSN2RemoteLSNMap) {
+                        logManager.log(flushLog);
+                        //store mapping information for flush logs to use them in incoming LSM components.
+                        RemoteLogMapping flushLogMap = new RemoteLogMapping();
+                        flushLogMap.setRemoteLSN(remoteLog.getLSN());
+                        flushLogMap.setRemoteNodeID(remoteLog.getNodeId());
+                        flushLogMap.setLocalLSN(flushLog.getLSN());
+                        flushLogMap.numOfFlushedIndexes.set(remoteLog.getNumOfFlushedIndexes());
+                        localLSN2RemoteLSNMap.put(flushLog.getLSN(), flushLogMap);
+                        localLSN2RemoteLSNMap.notifyAll();
+                    }
+                    break;
+                default:
+                    throw new ACIDException("Unsupported LogType: " + remoteLog.getLogType());
             }
         }
 
@@ -649,4 +698,4 @@ public class ReplicationChannel extends Thread implements IReplicationChannel {
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationLifecycleListener.java
----------------------------------------------------------------------
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationLifecycleListener.java b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationLifecycleListener.java
deleted file mode 100644
index 39130a4..0000000
--- a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationLifecycleListener.java
+++ /dev/null
@@ -1,77 +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.util.Set;
-
-import org.apache.asterix.common.api.IClusterEventsSubscriber;
-import org.apache.asterix.common.api.IClusterManagementWork;
-import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
-import org.apache.asterix.common.api.IClusterManagementWorkResponse;
-import org.apache.asterix.common.config.AsterixReplicationProperties;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.replication.ReplicaEvent;
-import org.apache.asterix.common.replication.ReplicaEvent.ReplicaEventType;
-
-public class ReplicationLifecycleListener implements IClusterEventsSubscriber {
-
-    private final AsterixReplicationProperties asterixReplicationProperties;
-    public static ReplicationLifecycleListener INSTANCE;
-
-    public ReplicationLifecycleListener(AsterixReplicationProperties asterixReplicationProperties) {
-        this.asterixReplicationProperties = asterixReplicationProperties;
-    }
-
-    @Override
-    public Set<IClusterManagementWork> notifyNodeFailure(Set<String> deadNodeIds) {
-        //notify impacted replicas
-        for (String deadNodeId : deadNodeIds) {
-            Replica eventOnReplica = asterixReplicationProperties.getReplicaById(deadNodeId);
-            ReplicaEvent event = new ReplicaEvent(eventOnReplica, ReplicaEventType.FAIL);
-            ReplicaEventNotifier notifier = new ReplicaEventNotifier(event, asterixReplicationProperties);
-
-            //start notifier
-            new Thread(notifier).start();
-        }
-
-        return null;
-    }
-
-    @Override
-    public Set<IClusterManagementWork> notifyNodeJoin(String joinedNodeId) {
-        Replica eventOnReplica = asterixReplicationProperties.getReplicaById(joinedNodeId);
-        ReplicaEvent event = new ReplicaEvent(eventOnReplica, ReplicaEventType.JOIN);
-        ReplicaEventNotifier notifier = new ReplicaEventNotifier(event, asterixReplicationProperties);
-
-        //start notifier
-        new Thread(notifier).start();
-
-        return null;
-    }
-
-    @Override
-    public void notifyRequestCompletion(IClusterManagementWorkResponse response) {
-        //do nothing
-    }
-
-    @Override
-    public void notifyStateChange(ClusterState previousState, ClusterState newState) {
-        //do nothing
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
----------------------------------------------------------------------
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
index 5c35df4..7243629 100644
--- a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
@@ -37,6 +37,7 @@ 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.Set;
@@ -50,7 +51,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.common.config.IAsterixPropertiesProvider;
 import org.apache.asterix.common.dataflow.AsterixLSMIndexUtil;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.replication.AsterixReplicationJob;
@@ -59,23 +62,25 @@ import org.apache.asterix.common.replication.IReplicationManager;
 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.ReplicaEvent.ReplicaEventType;
 import org.apache.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import org.apache.asterix.common.transactions.ILogManager;
 import org.apache.asterix.common.transactions.ILogRecord;
 import org.apache.asterix.common.transactions.LogRecord;
 import org.apache.asterix.common.transactions.LogType;
 import org.apache.asterix.event.schema.cluster.Node;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
-import org.apache.asterix.replication.functions.ReplicationProtocol.ReplicationRequestType;
 import org.apache.asterix.replication.functions.ReplicaFilesRequest;
 import org.apache.asterix.replication.functions.ReplicaIndexFlushRequest;
 import org.apache.asterix.replication.functions.ReplicaLogsRequest;
+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.ReplicationLogFlusher;
 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.exceptions.HyracksDataException;
 import org.apache.hyracks.api.replication.IReplicationJob;
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationExecutionType;
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationJobType;
@@ -107,6 +112,7 @@ public class ReplicationManager implements IReplicationManager {
     private final IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider;
     private final AsterixReplicationProperties replicationProperties;
     private final Map<String, Replica> replicas;
+    private final Map<String, Set<Integer>> replica2PartitionsMap;
 
     private final AtomicBoolean replicationSuspended;
     private AtomicBoolean terminateJobsReplication;
@@ -117,8 +123,8 @@ public class ReplicationManager implements IReplicationManager {
     private ReplicationJobsProccessor replicationJobsProcessor;
     private final ReplicasEventsMonitor replicationMonitor;
     //dummy job used to stop ReplicationJobsProccessor thread.
-    private static final IReplicationJob replicationJobPoisonPill = new AsterixReplicationJob(
-            ReplicationJobType.METADATA, ReplicationOperation.STOP, ReplicationExecutionType.ASYNC, null);
+    private static final IReplicationJob REPLICATION_JOB_POISON_PILL = new AsterixReplicationJob(
+            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;
 
@@ -128,6 +134,7 @@ public class ReplicationManager implements IReplicationManager {
     private ReplicationLogFlusher txnlogsReplicator;
     private Future<? extends Object> txnLogReplicatorTask;
     private Map<String, SocketChannel> logsReplicaSockets = null;
+
     //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, AsterixReplicationProperties replicationProperties,
@@ -144,7 +151,6 @@ public class ReplicationManager implements IReplicationManager {
         terminateJobsReplication = new AtomicBoolean(false);
         jobsReplicationSuspended = new AtomicBoolean(true);
         replicationSuspended = new AtomicBoolean(true);
-
         replicas = new HashMap<String, Replica>();
         jobCommitAcks = new ConcurrentHashMap<Integer, Set<String>>();
         replicationJobsPendingAcks = new ConcurrentHashMap<Integer, ILogRecord>();
@@ -156,12 +162,25 @@ public class ReplicationManager implements IReplicationManager {
         replicationJobsProcessor = new ReplicationJobsProccessor();
         replicationMonitor = new ReplicasEventsMonitor();
 
+        Map<String, ClusterPartition[]> nodePartitions = ((IAsterixPropertiesProvider) asterixAppRuntimeContextProvider
+                .getAppContext()).getMetadataProperties().getNodePartitions();
         //add list of replicas from configurations (To be read from another source e.g. Zookeeper)
         Set<Replica> replicaNodes = replicationProperties.getRemoteReplicas(nodeId);
-        if (replicaNodes != null) {
-            for (Replica replica : replicaNodes) {
-                replicas.put(replica.getNode().getId(), replica);
+        replica2PartitionsMap = new HashMap<>(replicaNodes.size());
+        for (Replica replica : replicaNodes) {
+            replicas.put(replica.getNode().getId(), replica);
+            //for each remote replica, get the list of replication clients
+            Set<String> nodeReplicationClients = replicationProperties.getNodeReplicationClients(replica.getId());
+            //get the partitions of each client
+            List<Integer> clientPartitions = new ArrayList<>();
+            for (String clientId : nodeReplicationClients) {
+                for (ClusterPartition clusterPartition : nodePartitions.get(clientId)) {
+                    clientPartitions.add(clusterPartition.getPartitionId());
+                }
             }
+            Set<Integer> clientPartitonsSet = new HashSet<>(clientPartitions.size());
+            clientPartitonsSet.addAll(clientPartitions);
+            replica2PartitionsMap.put(replica.getId(), clientPartitonsSet);
         }
         int numLogBuffers = logManager.getNumLogPages();
         emptyLogBuffersQ = new LinkedBlockingQueue<ReplicationLogBuffer>(numLogBuffers);
@@ -173,10 +192,6 @@ public class ReplicationManager implements IReplicationManager {
         }
     }
 
-    /**
-     * Accepts a replication job. If the job execution type is ASYNC, it is queued.
-     * Otherwise, it is processed immediately.
-     */
     @Override
     public void submitJob(IReplicationJob job) throws IOException {
         if (job.getExecutionType() == ReplicationExecutionType.ASYNC) {
@@ -253,24 +268,28 @@ public class ReplicationManager implements IReplicationManager {
      */
     private void processJob(IReplicationJob job, Map<String, SocketChannel> replicasSockets, ByteBuffer requestBuffer)
             throws IOException {
-        boolean isLSMComponentFile;
-        ByteBuffer responseBuffer = null;
-        LSMIndexFileProperties asterixFileProperties = new LSMIndexFileProperties();
-        if (requestBuffer == null) {
-            requestBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
-        }
-
-        isLSMComponentFile = job.getJobType() == ReplicationJobType.LSM_COMPONENT ? true : false;
         try {
-            //if there isn't already a connection, establish a new one
-            if (replicasSockets == null) {
-                replicasSockets = getActiveRemoteReplicasSockets();
+
+            //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();
+            int jobPartitionId = PersistentLocalResourceRepository.getResourcePartition(jobFile);
+
+            ByteBuffer responseBuffer = null;
+            LSMIndexFileProperties asterixFileProperties = new LSMIndexFileProperties();
+            if (requestBuffer == null) {
+                requestBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
             }
 
-            int remainingFiles = job.getJobFiles().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();
+                }
 
-            if (job.getOperation() == ReplicationOperation.REPLICATE) {
-                try {
+                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) {
@@ -316,6 +335,10 @@ public class ReplicationManager implements IReplicationManager {
                             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 {
@@ -338,44 +361,52 @@ public class ReplicationManager implements IReplicationManager {
                             }
                         }
                     }
-                } finally {
-                    if (job instanceof ILSMIndexReplicationJob) {
-                        //exit the replicated LSM components
-                        ILSMIndexReplicationJob aJob = (ILSMIndexReplicationJob) job;
-                        aJob.endReplication();
-                    }
-                }
-            } else if (job.getOperation() == ReplicationOperation.DELETE) {
-                for (String filePath : job.getJobFiles()) {
-                    remainingFiles--;
-                    asterixFileProperties.initialize(filePath, -1, nodeId, isLSMComponentFile,
-                            IMetaDataPageManager.INVALID_LSN_OFFSET, 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();
-                        SocketChannel socketChannel = entry.getValue();
-                        try {
-                            sendRequest(replicasSockets, requestBuffer);
-                            if (asterixFileProperties.requiresAck()) {
-                                waitForResponse(socketChannel, responseBuffer);
+                } else if (job.getOperation() == ReplicationOperation.DELETE) {
+                    for (String filePath : job.getJobFiles()) {
+                        remainingFiles--;
+                        asterixFileProperties.initialize(filePath, -1, nodeId, isLSMComponentFile,
+                                IMetaDataPageManager.INVALID_LSN_OFFSET, 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) {
+                                reportFailedReplica(entry.getKey());
+                                iterator.remove();
+                            } finally {
+                                requestBuffer.position(0);
                             }
-                        } catch (IOException e) {
-                            reportFailedReplica(entry.getKey());
-                            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 {
-            //if sync, close sockets with replicas since they wont be reused
-            if (job.getExecutionType() == ReplicationExecutionType.SYNC) {
-                closeReplicaSockets(replicasSockets);
-            }
+            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();
         }
     }
 
@@ -398,8 +429,7 @@ public class ReplicationManager implements IReplicationManager {
         }
 
         //read response from remote replicas
-        ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(socketChannel,
-                responseBuffer);
+        ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(socketChannel, responseBuffer);
         return responseFunction;
     }
 
@@ -415,17 +445,7 @@ public class ReplicationManager implements IReplicationManager {
         if (replica.getState() == ReplicaState.ACTIVE) {
             return;
         }
-
         replica.getNode().setClusterIp(replicaNode.getNode().getClusterIp());
-
-        /*
-         * This could be used to reconnect to replica without needing the Cluster notifications
-        if (replica.getState() == ReplicaState.DEAD) {
-            reportFailedReplica(replica.getNode().getId());
-        } else if (replica.getState() == ReplicaState.ACTIVE) {
-            checkReplicaState(replica.getNode().getId(), true);
-        }
-        */
     }
 
     /**
@@ -440,7 +460,7 @@ public class ReplicationManager implements IReplicationManager {
             if (force) {
                 terminateJobsReplication.set(true);
             }
-            replicationJobsQ.offer(replicationJobPoisonPill);
+            replicationJobsQ.offer(REPLICATION_JOB_POISON_PILL);
 
             //wait until the jobs are suspended
             synchronized (jobsReplicationSuspended) {
@@ -504,27 +524,6 @@ public class ReplicationManager implements IReplicationManager {
         }
     }
 
-    @Override
-    public void broadcastNewIPAddress() throws IOException {
-        String orignalIPAddress = replicationProperties.getReplicaIPAddress(nodeId);
-        String newAddress = NetworkingUtil.getHostAddress(hostIPAddressFirstOctet);
-
-        //IP Address didn't change after failure
-        if (orignalIPAddress.equals(newAddress)) {
-            return;
-        }
-
-        Node node = new Node();
-        node.setId(nodeId);
-        node.setClusterIp(newAddress);
-        Replica replica = new Replica(node);
-
-        ByteBuffer buffer = ReplicationProtocol.writeUpdateReplicaRequest(replica);
-        Map<String, SocketChannel> replicaSockets = getActiveRemoteReplicasSockets();
-        sendRequest(replicaSockets, buffer);
-        closeReplicaSockets(replicaSockets);
-    }
-
     /**
      * Sends a shutdown event to remote replicas notifying them
      * no more logs/files will be sent from this local replica.
@@ -536,7 +535,7 @@ public class ReplicationManager implements IReplicationManager {
         node.setId(nodeId);
         node.setClusterIp(NetworkingUtil.getHostAddress(hostIPAddressFirstOctet));
         Replica replica = new Replica(node);
-        ReplicaEvent event = new ReplicaEvent(replica, ReplicaEventType.SHUTDOWN);
+        ReplicaEvent event = new ReplicaEvent(replica, ClusterEventType.NODE_SHUTTING_DOWN);
         ByteBuffer buffer = ReplicationProtocol.writeReplicaEventRequest(event);
         Map<String, SocketChannel> replicaSockets = getActiveRemoteReplicasSockets();
         sendRequest(replicaSockets, buffer);
@@ -672,9 +671,6 @@ public class ReplicationManager implements IReplicationManager {
 
         if (newState == ReplicaState.ACTIVE) {
             replicationFactor++;
-            //TODO Extra check: make sure newly added replica is in sync.
-            //Since in the current design the whole cluster becomes UNUSABLE,
-            //no new jobs could start before the failed node rejoins
         } else if (newState == ReplicaState.DEAD) {
             if (replicationFactor > INITIAL_REPLICATION_FACTOR) {
                 replicationFactor--;
@@ -873,7 +869,7 @@ public class ReplicationManager implements IReplicationManager {
         //need to stop processing any new logs or jobs
         terminateJobsReplication.set(true);
 
-        ReplicaEvent event = new ReplicaEvent(replica, ReplicaEventType.FAIL);
+        ReplicaEvent event = new ReplicaEvent(replica, ClusterEventType.NODE_FAILURE);
         reportReplicaEvent(event);
     }
 
@@ -986,9 +982,10 @@ public class ReplicationManager implements IReplicationManager {
 
     //Recovery Method
     @Override
-    public void requestReplicaFiles(String selectedReplicaId, Set<String> replicasDataToRecover) throws IOException {
-        ReplicaFilesRequest request = new ReplicaFilesRequest(replicasDataToRecover);
-        ReplicationProtocol.writeGetReplicaFilesRequest(dataBuffer, request);
+    public void requestReplicaFiles(String selectedReplicaId, Set<String> replicasDataToRecover,
+            Set<String> existingFiles) throws IOException {
+        ReplicaFilesRequest request = new ReplicaFilesRequest(replicasDataToRecover, existingFiles);
+        dataBuffer = ReplicationProtocol.writeGetReplicaFilesRequest(dataBuffer, request);
 
         try (SocketChannel socketChannel = getReplicaSocket(selectedReplicaId)) {
 
@@ -997,8 +994,7 @@ public class ReplicationManager implements IReplicationManager {
 
             String indexPath;
             String destFilePath;
-            ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(socketChannel,
-                    dataBuffer);
+            ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(socketChannel, dataBuffer);
             LSMIndexFileProperties fileProperties;
             while (responseFunction != ReplicationRequestType.GOODBYE) {
                 dataBuffer = ReplicationProtocol.readRequest(socketChannel, dataBuffer);
@@ -1057,11 +1053,10 @@ public class ReplicationManager implements IReplicationManager {
 
     //Recovery Method
     @Override
-    public ArrayList<ILogRecord> requestReplicaLogs(String remoteNode, Set<String> nodeIdsToRecoverFor, long fromLSN)
-            throws IOException, ACIDException {
+    public void requestReplicaLogs(String remoteNode, Set<String> nodeIdsToRecoverFor, long fromLSN,
+            File recoveryLogsFile) throws IOException, ACIDException {
         ReplicaLogsRequest request = new ReplicaLogsRequest(nodeIdsToRecoverFor, fromLSN);
         dataBuffer = ReplicationProtocol.writeGetReplicaLogsRequest(dataBuffer, request);
-
         try (SocketChannel socketChannel = getReplicaSocket(remoteNode)) {
             //transfer request
             NetworkingUtil.transferBufferToChannel(socketChannel, dataBuffer);
@@ -1069,28 +1064,55 @@ public class ReplicationManager implements IReplicationManager {
             //read response type
             ReplicationRequestType responseType = ReplicationProtocol.getRequestType(socketChannel, dataBuffer);
 
-            ArrayList<ILogRecord> recoveryLogs = new ArrayList<ILogRecord>();
             ILogRecord logRecord = new LogRecord();
-            while (responseType != ReplicationRequestType.GOODBYE) {
-                dataBuffer = ReplicationProtocol.readRequest(socketChannel, dataBuffer);
-                logRecord.readRemoteLog(dataBuffer, true, nodeId);
-
-                if (logRecord.getNodeId().equals(nodeId)) {
-                    //store log in memory to replay it for recovery
-                    recoveryLogs.add(logRecord);
-                    //this needs to be a new log object so that it is passed to recovery manager as a different object
-                    logRecord = new LogRecord();
-                } else {
-                    //send log to log manager as a remote recovery log
-                    logManager.log(logRecord);
+            Set<Integer> nodePartitions = ((PersistentLocalResourceRepository) asterixAppRuntimeContextProvider
+                    .getLocalResourceRepository()).getNodeOrignalPartitions();
+            Set<Integer> nodePartitionsJobs = new HashSet<>();
+            try (RandomAccessFile raf = new RandomAccessFile(recoveryLogsFile, "rw");
+                    FileChannel fileChannel = raf.getChannel();) {
+                while (responseType != ReplicationRequestType.GOODBYE) {
+                    dataBuffer = ReplicationProtocol.readRequest(socketChannel, dataBuffer);
+                    logRecord.readRemoteLog(dataBuffer, true);
+                    switch (logRecord.getLogType()) {
+                        case LogType.UPDATE:
+                        case LogType.ENTITY_COMMIT:
+                        case LogType.UPSERT_ENTITY_COMMIT:
+                            if (nodePartitions.contains(logRecord.getResourcePartition())) {
+                                nodePartitionsJobs.add(logRecord.getJobId());
+                                dataBuffer.flip();
+                                while (dataBuffer.hasRemaining()) {
+                                    //store log in temp file to replay it for recovery
+                                    fileChannel.write(dataBuffer);
+                                }
+                            } else {
+                                //send log to log manager as a remote recovery log
+                                logManager.log(logRecord);
+                            }
+                            break;
+                        case LogType.JOB_COMMIT:
+                            if (nodePartitionsJobs.contains(logRecord.getJobId())) {
+                                nodePartitionsJobs.remove(logRecord.getJobId());
+                                dataBuffer.flip();
+                                while (dataBuffer.hasRemaining()) {
+                                    //store log in temp file to replay it for recovery
+                                    fileChannel.write(dataBuffer);
+                                }
+                                break;
+                            }
+                            logManager.log(logRecord);
+                            break;
+                        case LogType.ABORT:
+                        case LogType.FLUSH:
+                            break;
+                        default:
+                            throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
+                    }
+                    responseType = ReplicationProtocol.getRequestType(socketChannel, dataBuffer);
                 }
-
-                responseType = ReplicationProtocol.getRequestType(socketChannel, dataBuffer);
             }
 
             //send goodbye
             ReplicationProtocol.sendGoodbye(socketChannel);
-            return recoveryLogs;
         }
     }
 
@@ -1108,13 +1130,14 @@ public class ReplicationManager implements IReplicationManager {
                     event = replicaEventsQ.take();
 
                     switch (event.getEventType()) {
-                        case FAIL:
+                        case NODE_FAILURE:
                             handleReplicaFailure(event.getReplica().getId());
                             break;
-                        case JOIN:
+                        case NODE_JOIN:
+                            updateReplicaInfo(event.getReplica());
                             checkReplicaState(event.getReplica().getId(), false, true);
                             break;
-                        case SHUTDOWN:
+                        case NODE_SHUTTING_DOWN:
                             handleShutdownEvent(event.getReplica().getId());
                             break;
                         default:
@@ -1168,18 +1191,17 @@ public class ReplicationManager implements IReplicationManager {
                     }
 
                     IReplicationJob job = replicationJobsQ.take();
-                    if (job.getOperation() != ReplicationOperation.STOP) {
-                        //if there isn't already a connection, establish a new one
-                        if (replicaSockets == null) {
-                            replicaSockets = getActiveRemoteReplicasSockets();
-                        }
-
-                        processJob(job, replicaSockets, reusableBuffer);
-                    } else {
+                    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.size() == 0) {
                         LOGGER.log(Level.INFO, "No pending replication jobs. Closing connections to replicas");

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java
----------------------------------------------------------------------
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java b/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java
index ee987f8..c905add 100644
--- a/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java
@@ -18,8 +18,11 @@
  */
 package org.apache.asterix.replication.recovery;
 
+import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -30,56 +33,62 @@ import java.util.logging.Logger;
 
 import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.common.config.IAsterixPropertiesProvider;
 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.transactions.ILogManager;
-import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.ILogReader;
+import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.om.util.AsterixClusterProperties;
+import org.apache.asterix.replication.storage.ReplicaResourcesManager;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.asterix.transaction.management.service.logging.RemoteLogReader;
 
 public class RemoteRecoveryManager implements IRemoteRecoveryManager {
 
+    private static final String RECOVERY_LOGS_FILE_NAME = "recoveryLogs";
     private final IReplicationManager replicationManager;
-    private final ILogManager logManager;
-    public static final boolean IS_DEBUG_MODE = false;//true
     private static final Logger LOGGER = Logger.getLogger(RemoteRecoveryManager.class.getName());
     private final IAsterixAppRuntimeContext runtimeContext;
     private final AsterixReplicationProperties replicationProperties;
+    private final static int REMOTE_RECOVERY_JOB_ID = -1;
+    private Map<String, Set<String>> failbackRecoveryReplicas;
 
     public RemoteRecoveryManager(IReplicationManager replicationManager, IAsterixAppRuntimeContext runtimeContext,
             AsterixReplicationProperties replicationProperties) {
         this.replicationManager = replicationManager;
         this.runtimeContext = runtimeContext;
-        this.logManager = runtimeContext.getTransactionSubsystem().getLogManager();
         this.replicationProperties = replicationProperties;
     }
 
     @Override
     public void performRemoteRecovery() {
-        //TODO this method needs to be adapted to perform failback when autoFailover is enabled.
-        //Currently we will not allow a node to perform remote recovery since another replica
-        //already tookover its workload and might not resync correctly if there are on on-going
-        //jobs on the replica.
-        if (AsterixClusterProperties.INSTANCE.isAutoFailoverEnabled()) {
-            throw new IllegalStateException("Cannot perform remote recovery when auto failover is enabled.");
-        }
         //The whole remote recovery process should be atomic.
-        //Any error happens, we should start the recovery from the start until the recovery is complete or an illegal state is reached (cannot recovery).
-        int maxRecoveryAttempts = 10;
+        //Any error happens, we should start the recovery from the start until the recovery is
+        //complete or an illegal state is reached (cannot recover or max attempts exceed).
+        int maxRecoveryAttempts = replicationProperties.getMaxRemoteRecoveryAttempts();
         PersistentLocalResourceRepository resourceRepository = (PersistentLocalResourceRepository) runtimeContext
                 .getLocalResourceRepository();
+        IRecoveryManager recoveryManager = runtimeContext.getTransactionSubsystem().getRecoveryManager();
+        ILogManager logManager = runtimeContext.getTransactionSubsystem().getLogManager();
         while (true) {
             //start recovery steps
             try {
-                maxRecoveryAttempts--;
-
-                if (maxRecoveryAttempts == 0) {
+                if (maxRecoveryAttempts <= 0) {
                     //to avoid infinite loop in case of unexpected behavior.
                     throw new IllegalStateException("Failed to perform remote recovery.");
                 }
 
+                //delete any existing recovery files from previous failed recovery attempts
+                recoveryManager.deleteRecoveryTemporaryFiles();
+
+                //create temporary file to store recovery logs
+                File recoveryLogsFile = recoveryManager.createJobRecoveryFile(REMOTE_RECOVERY_JOB_ID,
+                        RECOVERY_LOGS_FILE_NAME);
+
                 /*** Prepare for Recovery ***/
                 //1. check remote replicas states
                 replicationManager.initializeReplicasState();
@@ -93,8 +102,9 @@ public class RemoteRecoveryManager implements IRemoteRecoveryManager {
                 IDatasetLifecycleManager datasetLifeCycleManager = runtimeContext.getDatasetLifecycleManager();
                 datasetLifeCycleManager.closeAllDatasets();
 
-                //3. remove any existing storage data
+                //3. remove any existing storage data and initialize storage metadata
                 resourceRepository.deleteStorageData(true);
+                resourceRepository.initializeNewUniverse(AsterixClusterProperties.INSTANCE.getStorageDirectoryName());
 
                 //4. select remote replicas to recover from per lost replica data
                 Map<String, Set<String>> selectedRemoteReplicas = constructRemoteRecoveryPlan();
@@ -110,47 +120,42 @@ public class RemoteRecoveryManager implements IRemoteRecoveryManager {
                     String replicaId = remoteReplica.getKey();
                     Set<String> replicasDataToRecover = remoteReplica.getValue();
 
-                    //1. Request indexes metadata and LSM components
-                    replicationManager.requestReplicaFiles(replicaId, replicasDataToRecover);
-
-                    //2. Initialize local resources based on the newly received files (if we are recovering the primary replica on this node)
-                    if (replicasDataToRecover.contains(logManager.getNodeId())) {
-                        ((PersistentLocalResourceRepository) runtimeContext.getLocalResourceRepository())
-                                .initializeNewUniverse(AsterixClusterProperties.INSTANCE.getStorageDirectoryName());
-                        //initialize resource id factor to correct max resource id
-                        runtimeContext.initializeResourceIdFactory();
-                    }
+                    //Request indexes metadata and LSM components
+                    replicationManager.requestReplicaFiles(replicaId, replicasDataToRecover, new HashSet<String>());
 
-                    //3. Get min LSN to start requesting logs from
+                    //Get min LSN to start requesting logs from
                     long minLSN = replicationManager.requestReplicaMinLSN(replicaId);
 
-                    //4. Request remote logs from selected remote replicas
-                    ArrayList<ILogRecord> remoteRecoveryLogs = replicationManager.requestReplicaLogs(replicaId,
-                            replicasDataToRecover, minLSN);
+                    //Request remote logs from selected remote replicas
+                    replicationManager.requestReplicaLogs(replicaId, replicasDataToRecover, minLSN, recoveryLogsFile);
 
-                    //5. Replay remote logs using recovery manager
+                    //Replay remote logs using recovery manager
                     if (replicasDataToRecover.contains(logManager.getNodeId())) {
-                        if (remoteRecoveryLogs.size() > 0) {
-                            runtimeContext.getTransactionSubsystem().getRecoveryManager()
-                                    .replayRemoteLogs(remoteRecoveryLogs);
+                        //replay logs for local partitions only
+                        Set<Integer> nodePartitions = resourceRepository.getNodeOrignalPartitions();
+                        try (RandomAccessFile raf = new RandomAccessFile(recoveryLogsFile, "r");
+                                FileChannel fileChannel = raf.getChannel();) {
+                            ILogReader logReader = new RemoteLogReader(fileChannel, fileChannel.size(),
+                                    logManager.getLogPageSize());
+                            recoveryManager.replayPartitionsLogs(nodePartitions, logReader, 0);
                         }
-                        remoteRecoveryLogs.clear();
                     }
                 }
-
                 LOGGER.log(Level.INFO, "Completed remote recovery successfully!");
                 break;
             } catch (Exception e) {
                 e.printStackTrace();
                 LOGGER.log(Level.WARNING, "Failed during remote recovery. Attempting again...");
+                maxRecoveryAttempts--;
             }
         }
     }
 
     private Map<String, Set<String>> constructRemoteRecoveryPlan() {
         //1. identify which replicas reside in this node
-        String localNodeId = logManager.getNodeId();
-        Set<String> nodes = replicationProperties.getNodeReplicasIds(localNodeId);
+        String localNodeId = runtimeContext.getTransactionSubsystem().getId();
+
+        Set<String> nodes = replicationProperties.getNodeReplicationClients(localNodeId);
 
         Map<String, Set<String>> recoveryCandidates = new HashMap<String, Set<String>>();
         Map<String, Integer> candidatesScore = new HashMap<String, Integer>();
@@ -187,7 +192,6 @@ public class RemoteRecoveryManager implements IRemoteRecoveryManager {
 
         //3. find best candidate to recover from per lost replica data
         for (Entry<String, Set<String>> entry : recoveryCandidates.entrySet()) {
-
             int winnerScore = -1;
             String winner = "";
             for (String node : entry.getValue()) {
@@ -214,12 +218,133 @@ public class RemoteRecoveryManager implements IRemoteRecoveryManager {
     }
 
     @Override
-    public void takeoverPartitons(String failedNode, Integer[] partitions) throws IOException, ACIDException {
-        long minLSN = runtimeContext.getReplicaResourcesManager().getPartitionsMinLSN(partitions);
-        //reply logs > minLSN that belong to these partitions
-        //TODO (mhubail) currently we assume the logs for these partitions belong to the failed node
-        //this needs to be updated once log formats are updated to include the partition id
-        runtimeContext.getTransactionSubsystem().getRecoveryManager().replayPartitionsLogs(partitions, minLSN,
-                failedNode);
+    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));
+        ILogManager logManager = runtimeContext.getTransactionSubsystem().getLogManager();
+
+        long minLSN = runtimeContext.getReplicaResourcesManager().getPartitionsMinLSN(partitionsToTakeover);
+        long readableSmallestLSN = logManager.getReadableSmallestLSN();
+        if (minLSN < readableSmallestLSN) {
+            minLSN = readableSmallestLSN;
+        }
+
+        //replay logs > minLSN that belong to these partitions
+        IRecoveryManager recoveryManager = runtimeContext.getTransactionSubsystem().getRecoveryManager();
+        recoveryManager.replayPartitionsLogs(partitionsToTakeover, logManager.getLogReader(true), minLSN);
+
+        //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();
+
+        failbackRecoveryReplicas = new HashMap<>();
+        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(true);
+                resourceRepository.initializeNewUniverse(AsterixClusterProperties.INSTANCE.getStorageDirectoryName());
+
+                //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> partitionsToRecover = remoteReplica.getValue();
+
+                    //1. Request indexes metadata and LSM components
+                    replicationManager.requestReplicaFiles(replicaId, partitionsToRecover, new HashSet<String>());
+                }
+                break;
+            } catch (IOException e) {
+                e.printStackTrace();
+                LOGGER.log(Level.WARNING, "Failed during remote recovery. Attempting again...");
+                maxRecoveryAttempts--;
+            }
+        }
+    }
+
+    @Override
+    public void completeFailbackProcess() throws IOException {
+        ILogManager logManager = runtimeContext.getTransactionSubsystem().getLogManager();
+        ReplicaResourcesManager replicaResourcesManager = (ReplicaResourcesManager) runtimeContext
+                .getReplicaResourcesManager();
+        Map<String, ClusterPartition[]> nodePartitions = ((IAsterixPropertiesProvider) 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<>();
+                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));
+                    }
+                }
+
+                //Request remaining indexes files
+                replicationManager.requestReplicaFiles(replicaId, NCsDataToRecover, 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.
+             */
+            e.printStackTrace();
+            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;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java
----------------------------------------------------------------------
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java
index 890d3a2..031aeb6 100644
--- a/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java
@@ -69,11 +69,7 @@ public class LSMIndexFileProperties {
         this.fileName = tokens[arraySize - 1];
         this.idxName = tokens[arraySize - 2];
         this.dataverse = tokens[arraySize - 3];
-        this.partition = getPartitonNumFromName(tokens[arraySize - 4]);
-    }
-
-    private static int getPartitonNumFromName(String name) {
-        return Integer.parseInt(name.substring(StoragePathUtil.PARTITION_DIR_PREFIX.length()));
+        this.partition = StoragePathUtil.getPartitonNumFromName(tokens[arraySize - 4]);
     }
 
     public void serialize(OutputStream out) throws IOException {
@@ -114,10 +110,6 @@ public class LSMIndexFileProperties {
         return nodeId;
     }
 
-    public void setNodeId(String nodeId) {
-        this.nodeId = nodeId;
-    }
-
     public String getDataverse() {
         return dataverse;
     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java
----------------------------------------------------------------------
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java
index b9f7506..55d442d 100644
--- a/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java
@@ -34,7 +34,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -55,7 +54,6 @@ public class ReplicaResourcesManager implements IReplicaResourcesManager {
     public final static String LSM_COMPONENT_MASK_SUFFIX = "_mask";
     private final static String REPLICA_INDEX_LSN_MAP_NAME = ".LSN_MAP";
     public static final long REPLICA_INDEX_CREATION_LSN = -1;
-    private final AtomicLong lastMinRemoteLSN;
     private final PersistentLocalResourceRepository localRepository;
     private final Map<String, ClusterPartition[]> nodePartitions;
 
@@ -63,7 +61,6 @@ public class ReplicaResourcesManager implements IReplicaResourcesManager {
             AsterixMetadataProperties metadataProperties) {
         this.localRepository = (PersistentLocalResourceRepository) localRepository;
         nodePartitions = metadataProperties.getNodePartitions();
-        lastMinRemoteLSN = new AtomicLong(-1);
     }
 
     public void deleteIndexFile(LSMIndexFileProperties afp) {
@@ -126,7 +123,6 @@ public class ReplicaResourcesManager implements IReplicaResourcesManager {
 
         //update map on disk
         updateReplicaIndexLSNMap(lsmComponentProperties.getReplicaComponentPath(this), lsnMap);
-
     }
 
     public Set<File> getReplicaIndexes(String replicaId) {
@@ -139,35 +135,7 @@ public class ReplicaResourcesManager implements IReplicaResourcesManager {
     }
 
     @Override
-    public long getMinRemoteLSN(Set<String> replicaIds) {
-        if (lastMinRemoteLSN.get() != -1) {
-            return lastMinRemoteLSN.get();
-        }
-        long minRemoteLSN = Long.MAX_VALUE;
-        for (String replica : replicaIds) {
-            //for every index in replica
-            Set<File> remoteIndexes = getReplicaIndexes(replica);
-            for (File indexFolder : remoteIndexes) {
-                //read LSN map
-                try {
-                    //get max LSN per index
-                    long remoteIndexMaxLSN = getReplicaIndexMaxLSN(indexFolder);
-
-                    //get min of all maximums
-                    minRemoteLSN = Math.min(minRemoteLSN, remoteIndexMaxLSN);
-                } catch (IOException e) {
-                    LOGGER.log(Level.INFO,
-                            indexFolder.getAbsolutePath() + " Couldn't read LSN map for index " + indexFolder);
-                    continue;
-                }
-            }
-        }
-        lastMinRemoteLSN.set(minRemoteLSN);
-        return minRemoteLSN;
-    }
-
-    @Override
-    public long getPartitionsMinLSN(Integer[] partitions) {
+    public long getPartitionsMinLSN(Set<Integer> partitions) {
         long minRemoteLSN = Long.MAX_VALUE;
         for (Integer partition : partitions) {
             //for every index in replica
@@ -219,7 +187,6 @@ public class ReplicaResourcesManager implements IReplicaResourcesManager {
                 remoteIndexMaxLSN = Math.max(remoteIndexMaxLSN, lsn);
             }
         }
-
         return remoteIndexMaxLSN;
     }
 
@@ -271,7 +238,6 @@ public class ReplicaResourcesManager implements IReplicaResourcesManager {
                 ObjectOutputStream oosToFos = new ObjectOutputStream(fos)) {
             oosToFos.writeObject(lsnMap);
             oosToFos.flush();
-            lastMinRemoteLSN.set(-1);
         }
     }
 
@@ -293,7 +259,9 @@ public class ReplicaResourcesManager implements IReplicaResourcesManager {
                         File[] indexFileList = dataverseFile.listFiles();
                         if (indexFileList != null) {
                             for (File indexFile : indexFileList) {
-                                partitionIndexes.add(indexFile);
+                                if (indexFile.isDirectory()) {
+                                    partitionIndexes.add(indexFile);
+                                }
                             }
                         }
                     }
@@ -307,7 +275,7 @@ public class ReplicaResourcesManager implements IReplicaResourcesManager {
      * @param partition
      * @return Absolute paths to all partition files
      */
-    public List<String> getPartitionIndexesFiles(int partition) {
+    public List<String> getPartitionIndexesFiles(int partition, boolean relativePath) {
         List<String> partitionFiles = new ArrayList<String>();
         Set<File> partitionIndexes = getPartitionIndexes(partition);
         for (File indexDir : partitionIndexes) {
@@ -315,7 +283,12 @@ public class ReplicaResourcesManager implements IReplicaResourcesManager {
                 File[] indexFiles = indexDir.listFiles(LSM_INDEX_FILES_FILTER);
                 if (indexFiles != null) {
                     for (File file : indexFiles) {
-                        partitionFiles.add(file.getAbsolutePath());
+                        if (!relativePath) {
+                            partitionFiles.add(file.getAbsolutePath());
+                        } else {
+                            partitionFiles.add(
+                                    PersistentLocalResourceRepository.getResourceRelativePath(file.getAbsolutePath()));
+                        }
                     }
                 }
             }
@@ -324,18 +297,21 @@ public class ReplicaResourcesManager implements IReplicaResourcesManager {
     }
 
     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(PersistentLocalResourceRepository.METADATA_FILE_NAME) || !name.startsWith(".");
         }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java
index f35f4d6..97683d5 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java
@@ -75,7 +75,7 @@ public class AsterixLSMPrimaryUpsertOperatorNodePushable extends LSMIndexInsertU
 
     public AsterixLSMPrimaryUpsertOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition, int[] fieldPermutation, IRecordDescriptorProvider recordDescProvider, int numOfPrimaryKeys,
-            ARecordType recordType, int filterFieldIndex) throws HyracksDataException {
+            ARecordType recordType, int filterFieldIndex) {
         super(opDesc, ctx, partition, fieldPermutation, recordDescProvider, IndexOperation.UPSERT);
         // initialize nullWriter
         this.nullWriter = opDesc.getNullWriterFactory().createNullWriter();
@@ -125,7 +125,8 @@ public class AsterixLSMPrimaryUpsertOperatorNodePushable extends LSMIndexInsertU
             dos = tb.getDataOutput();
             appender = new FrameTupleAppender(new VSizeFrame(ctx), true);
             modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
-                    indexHelper.getResourcePath(), indexHelper.getResourceID(), lsmIndex, ctx);
+                    indexHelper.getResourcePath(), indexHelper.getResourceID(), indexHelper.getResourcePartition(),
+                    lsmIndex, ctx);
 
             indexAccessor = lsmIndex.createAccessor(modCallback, opDesc.getSearchOpCallbackFactory()
                     .createSearchOperationCallback(indexHelper.getResourceID(), ctx));

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
index 3b5630f..65c9a49 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
@@ -45,7 +45,7 @@ public abstract class AbstractIndexModificationOperationCallback extends Abstrac
 
     protected AbstractIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
             ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
-            byte resourceType, IndexOperation indexOp) {
+            int resourcePartition, byte resourceType, IndexOperation indexOp) {
         super(datasetId, primaryKeyFields, txnCtx, lockManager);
         this.resourceId = resourceId;
         this.resourceType = resourceType;
@@ -58,8 +58,8 @@ public abstract class AbstractIndexModificationOperationCallback extends Abstrac
         logRecord.setJobId(txnCtx.getJobId().getId());
         logRecord.setDatasetId(datasetId);
         logRecord.setResourceId(resourceId);
+        logRecord.setResourcePartition(resourcePartition);
         logRecord.setNewOp((byte) (indexOp.ordinal()));
-        logRecord.setNodeId(txnSubsystem.getId());
     }
 
     protected void log(int PKHash, ITupleReference newValue) throws ACIDException {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
index 3c34153..780f294 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
@@ -37,9 +37,10 @@ public class PrimaryIndexModificationOperationCallback extends AbstractIndexModi
         implements IModificationOperationCallback {
 
     public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
-            ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, byte resourceType,
-            IndexOperation indexOp) {
-        super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
+            ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, int resourcePartition,
+            byte resourceType, IndexOperation indexOp) {
+        super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourcePartition,
+                resourceType, indexOp);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
index 794f867..db68b26 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -37,8 +37,8 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 /**
  * Assumes LSM-BTrees as primary indexes.
  */
-public class PrimaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory implements
-        IModificationOperationCallbackFactory {
+public class PrimaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory
+        implements IModificationOperationCallbackFactory {
 
     private static final long serialVersionUID = 1L;
     private final IndexOperation indexOp;
@@ -51,7 +51,7 @@ public class PrimaryIndexModificationOperationCallbackFactory extends AbstractOp
 
     @Override
     public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
-            Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+            int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
 
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
@@ -64,8 +64,8 @@ public class PrimaryIndexModificationOperationCallbackFactory extends AbstractOp
         try {
             ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
-                    primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
-                    indexOp);
+                    primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId,
+                    resourcePartition, resourceType, indexOp);
             txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, true);
             return modCallback;
         } catch (ACIDException e) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
index 250e28d..8044d90 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
@@ -40,8 +40,9 @@ public class SecondaryIndexModificationOperationCallback extends AbstractIndexMo
 
     public SecondaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
             ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
-            byte resourceType, IndexOperation indexOp) {
-        super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
+            int resourcePartition, byte resourceType, IndexOperation indexOp) {
+        super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourcePartition,
+                resourceType, indexOp);
         oldOp = (indexOp == IndexOperation.DELETE) ? IndexOperation.INSERT : IndexOperation.DELETE;
     }
 


[6/6] incubator-asterixdb git commit: Asterix NCs Failback Support

Posted by mh...@apache.org.
Asterix NCs Failback Support

- Allow Failed NCs to failback and takeover their partitions.
- New cluster API servlet for cluster state description.
- Remove nodeId from txn logs except remote FLUSH_LOG.
- Add partition id in UPDATE and Entity_COMMIT logs.
- Adapt remote recovery to new logs format.
- Refactor RecoveryManager and split Analysis and Redo phase.
- Spill remote recover logs to temporary file.
- Replicate files to remote replicas for partitions of interest only.
- Introduce NC active/inactive partitions concept.
- Test case for failback.

Change-Id: Id17819542d6b9c4e32647e64737c4a467b630f24
Reviewed-on: https://asterix-gerrit.ics.uci.edu/613
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <ba...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/commit/98d38e6a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/tree/98d38e6a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/diff/98d38e6a

Branch: refs/heads/master
Commit: 98d38e6a0a3199a33e06a7665c8a587841b260c3
Parents: c318249
Author: Murtadha Hubail <mh...@uci.edu>
Authored: Wed Feb 17 23:53:09 2016 -0800
Committer: Murtadha Hubail <hu...@gmail.com>
Committed: Thu Feb 18 01:49:33 2016 -0800

----------------------------------------------------------------------
 .gitignore                                      |    1 +
 .../operators/physical/CommitPOperator.java     |   21 +-
 .../operators/physical/CommitRuntime.java       |   17 +-
 .../physical/CommitRuntimeFactory.java          |    9 +-
 .../operators/physical/UpsertCommitRuntime.java |   13 +-
 .../rules/ReplaceSinkOpWithCommitOpRule.java    |   13 +-
 .../api/common/AsterixAppRuntimeContext.java    |   51 +-
 .../api/http/servlet/ClusterAPIServlet.java     |   51 +
 .../bootstrap/CCApplicationEntryPoint.java      |   54 +-
 .../bootstrap/ClusterLifecycleListener.java     |   17 +-
 .../bootstrap/NCApplicationEntryPoint.java      |   70 +-
 .../asterix/messaging/CCMessageBroker.java      |   24 +-
 .../asterix/messaging/NCMessageBroker.java      |  159 ++-
 .../common/api/IAsterixAppRuntimeContext.java   |   11 +-
 .../common/api/IClusterManagementWork.java      |    3 +-
 .../common/api/IDatasetLifecycleManager.java    |    7 +
 .../config/AsterixReplicationProperties.java    |   56 +-
 .../common/context/DatasetLifecycleManager.java |   19 +-
 ...erixLSMInsertDeleteOperatorNodePushable.java |    3 +-
 .../messaging/AbstractFailbackPlanMessage.java  |   39 +
 .../CompleteFailbackRequestMessage.java         |   56 +
 .../CompleteFailbackResponseMessage.java        |   49 +
 ...PreparePartitionsFailbackRequestMessage.java |   65 ++
 ...reparePartitionsFailbackResponseMessage.java |   41 +
 .../common/messaging/ReplicaEventMessage.java   |   52 +
 .../TakeoverPartitionsRequestMessage.java       |   10 +-
 .../messaging/api/IApplicationMessage.java      |    7 +-
 .../replication/IRemoteRecoveryManager.java     |   13 +-
 .../replication/IReplicaResourcesManager.java   |    8 +-
 .../common/replication/IReplicationManager.java |   22 +-
 .../common/replication/NodeFailbackPlan.java    |  209 ++++
 .../common/replication/ReplicaEvent.java        |   23 +-
 .../asterix/common/transactions/ILogRecord.java |   20 +-
 .../common/transactions/IRecoveryManager.java   |   43 +-
 .../asterix/common/transactions/LogRecord.java  |  214 ++--
 .../asterix/common/utils/ServletUtil.java       |   54 +
 .../asterix/common/utils/StoragePathUtil.java   |    4 +
 .../asterix/common/utils/TransactionUtil.java   |   34 +-
 .../apache/asterix/test/aql/TestExecutor.java   |   92 +-
 .../asterix/installer/test/ReplicationIT.java   |   14 +-
 .../node_failback/node_failback.1.ddl.aql       |   59 +
 .../node_failback/node_failback.10.cstate.aql   |   29 +
 .../node_failback/node_failback.11.query.aql    |   33 +
 .../node_failback/node_failback.2.update.aql    |   35 +
 .../node_failback/node_failback.3.vscript.aql   |    1 +
 .../node_failback/node_failback.4.sleep.aql     |    1 +
 .../node_failback/node_failback.5.cstate.aql    |   29 +
 .../node_failback/node_failback.6.query.aql     |   33 +
 .../node_failback/node_failback.7.update.aql    |   51 +
 .../node_failback/node_failback.8.vmgx.aql      |    1 +
 .../node_failback/node_failback.9.sleep.aql     |    1 +
 .../failover/bulkload/bulkload.2.update.aql     |    2 +-
 .../bulkload/bulkload.4.vagrant_script.aql      |    1 -
 .../failover/bulkload/bulkload.4.vscript.aql    |    1 +
 .../mem_component_recovery.2.update.aql         |    2 +-
 .../mem_component_recovery.4.vagrant_script.aql |    1 -
 .../mem_component_recovery.4.vscript.aql        |    1 +
 .../metadata_node.3.vagrant_script.aql          |    1 -
 .../metadata_node/metadata_node.3.vscript.aql   |    1 +
 .../node_failback.cluster_state.10.adm          |    1 +
 .../node_failback.cluster_state.5.adm           |    1 +
 .../node_failback/node_failback.query.11.adm    |    1 +
 .../node_failback/node_failback.query.6.adm     |    1 +
 .../integrationts/replication/testsuite.xml     |   41 +-
 .../apache/asterix/metadata/MetadataNode.java   |   26 +-
 .../om/util/AsterixClusterProperties.java       |  407 +++++--
 .../functions/ReplicaFilesRequest.java          |   22 +-
 .../functions/ReplicationProtocol.java          |   57 +-
 .../management/ReplicaEventNotifier.java        |  109 --
 .../management/ReplicationChannel.java          |  187 +--
 .../ReplicationLifecycleListener.java           |   77 --
 .../management/ReplicationManager.java          |  288 ++---
 .../recovery/RemoteRecoveryManager.java         |  221 +++-
 .../storage/LSMIndexFileProperties.java         |   10 +-
 .../storage/ReplicaResourcesManager.java        |   52 +-
 ...rixLSMPrimaryUpsertOperatorNodePushable.java |    5 +-
 ...tractIndexModificationOperationCallback.java |    4 +-
 ...imaryIndexModificationOperationCallback.java |    7 +-
 ...dexModificationOperationCallbackFactory.java |   10 +-
 ...ndaryIndexModificationOperationCallback.java |    5 +-
 ...dexModificationOperationCallbackFactory.java |   10 +-
 ...tasetIndexModificationOperationCallback.java |    5 +-
 ...dexModificationOperationCallbackFactory.java |    6 +-
 ...dexModificationOperationCallbackFactory.java |    6 +-
 .../opcallbacks/UpsertOperationCallback.java    |    7 +-
 .../UpsertOperationCallbackFactory.java         |    5 +-
 .../PersistentLocalResourceRepository.java      |   58 +-
 .../logging/LogManagerWithReplication.java      |    5 +-
 .../management/service/logging/LogReader.java   |    3 +-
 .../service/logging/RemoteLogReader.java        |  138 +++
 .../service/recovery/RecoveryManager.java       | 1099 ++++--------------
 .../management/service/recovery/TxnId.java      |  175 +++
 .../service/transaction/TransactionContext.java |    1 -
 93 files changed, 3087 insertions(+), 1914 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 90efb11..d6ab1f1 100644
--- a/.gitignore
+++ b/.gitignore
@@ -16,6 +16,7 @@ asterix-app/parserts/
 asterix-app/opt_parserts/
 asterix-app/runtime_parserts/
 asterix-installer/ittest/
+asterix-installer/repliationtest/
 build
 asterix_logs
 bin/

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
index 5112fcf..2d1cf1e 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
@@ -37,20 +37,25 @@ import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirement
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.dataflow.std.file.FileSplit;
 
 public class CommitPOperator extends AbstractPhysicalOperator {
 
     private final List<LogicalVariable> primaryKeyLogicalVars;
     private final JobId jobId;
     private final int datasetId;
+    private final String dataverse;
+    private final String dataset;
     private final LogicalVariable upsertVar;
 
-    public CommitPOperator(JobId jobId, int datasetId, List<LogicalVariable> primaryKeyLogicalVars,
-            LogicalVariable upsertVar) {
+    public CommitPOperator(JobId jobId, String dataverse, String dataset, int datasetId,
+            List<LogicalVariable> primaryKeyLogicalVars, LogicalVariable upsertVar) {
         this.jobId = jobId;
         this.datasetId = datasetId;
         this.primaryKeyLogicalVars = primaryKeyLogicalVars;
         this.upsertVar = upsertVar;
+        this.dataverse = dataverse;
+        this.dataset = dataset;
     }
 
     @Override
@@ -84,13 +89,23 @@ public class CommitPOperator extends AbstractPhysicalOperator {
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema,
                 context);
         int[] primaryKeyFields = JobGenHelper.variablesToFieldIndexes(primaryKeyLogicalVars, inputSchemas[0]);
+
+        //get dataset splits
+        FileSplit[] splitsForDataset = metadataProvider.splitsForDataset(metadataProvider.getMetadataTxnContext(),
+                dataverse, dataset, dataset, metadataProvider.isTemporaryDatasetWriteJob());
+        int[] datasetPartitions = new int[splitsForDataset.length];
+        for (int i = 0; i < splitsForDataset.length; i++) {
+            datasetPartitions[i] = splitsForDataset[i].getPartition();
+        }
+
         int upsertVarIdx = -1;
         CommitRuntimeFactory runtime = null;
         if (upsertVar != null) {
             upsertVarIdx = inputSchemas[0].findVariable(upsertVar);
         }
         runtime = new CommitRuntimeFactory(jobId, datasetId, primaryKeyFields,
-                metadataProvider.isTemporaryDatasetWriteJob(), metadataProvider.isWriteTransaction(), upsertVarIdx);
+                metadataProvider.isTemporaryDatasetWriteJob(), metadataProvider.isWriteTransaction(), upsertVarIdx,
+                datasetPartitions);
         builder.contributeMicroOperator(op, runtime, recDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
index b72018a..94519cf 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
@@ -28,8 +28,8 @@ import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionManager;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.common.transactions.LogRecord;
+import org.apache.asterix.common.transactions.LogType;
 import org.apache.asterix.common.utils.TransactionUtil;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -58,9 +58,10 @@ public class CommitRuntime implements IPushRuntime {
 
     protected ITransactionContext transactionContext;
     protected FrameTupleAccessor frameTupleAccessor;
+    protected final int resourcePartition;
 
     public CommitRuntime(IHyracksTaskContext ctx, JobId jobId, int datasetId, int[] primaryKeyFields,
-            boolean isTemporaryDatasetWriteJob, boolean isWriteTransaction) {
+            boolean isTemporaryDatasetWriteJob, boolean isWriteTransaction, int resourcePartition) {
         IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
                 .getApplicationContext().getApplicationObject();
         this.ctx = ctx;
@@ -72,9 +73,9 @@ public class CommitRuntime implements IPushRuntime {
         this.frameTupleReference = new FrameTupleReference();
         this.isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob;
         this.isWriteTransaction = isWriteTransaction;
-        this.longHashes = new long[2];
-        this.logRecord = new LogRecord();
-        logRecord.setNodeId(logMgr.getNodeId());
+        this.resourcePartition = resourcePartition;
+        longHashes = new long[2];
+        logRecord = new LogRecord();
     }
 
     @Override
@@ -109,17 +110,17 @@ public class CommitRuntime implements IPushRuntime {
                 try {
                     formLogRecord(buffer, t);
                     logMgr.log(logRecord);
-                } catch (ACIDException | AlgebricksException e) {
+                } catch (ACIDException e) {
                     throw new HyracksDataException(e);
                 }
             }
         }
     }
 
-    protected void formLogRecord(ByteBuffer buffer, int t) throws AlgebricksException {
+    protected void formLogRecord(ByteBuffer buffer, int t) {
         int pkHash = computePrimaryKeyHashValue(frameTupleReference, primaryKeyFields);
         TransactionUtil.formEntityCommitLogRecord(logRecord, transactionContext, datasetId, pkHash, frameTupleReference,
-                primaryKeyFields);
+                primaryKeyFields, resourcePartition, LogType.ENTITY_COMMIT);
     }
 
     protected int computePrimaryKeyHashValue(ITupleReference tuple, int[] primaryKeyFields) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntimeFactory.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntimeFactory.java b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntimeFactory.java
index 7d03796..4f28b9d 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntimeFactory.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntimeFactory.java
@@ -35,15 +35,17 @@ public class CommitRuntimeFactory implements IPushRuntimeFactory {
     private final boolean isTemporaryDatasetWriteJob;
     private final boolean isWriteTransaction;
     private final int upsertVarIdx;
+    private int[] datasetPartitions;
 
     public CommitRuntimeFactory(JobId jobId, int datasetId, int[] primaryKeyFields, boolean isTemporaryDatasetWriteJob,
-            boolean isWriteTransaction, int upsertVarIdx) {
+            boolean isWriteTransaction, int upsertVarIdx, int[] datasetPartitions) {
         this.jobId = jobId;
         this.datasetId = datasetId;
         this.primaryKeyFields = primaryKeyFields;
         this.isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob;
         this.isWriteTransaction = isWriteTransaction;
         this.upsertVarIdx = upsertVarIdx;
+        this.datasetPartitions = datasetPartitions;
     }
 
     @Override
@@ -55,10 +57,11 @@ public class CommitRuntimeFactory implements IPushRuntimeFactory {
     public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException {
         if (upsertVarIdx >= 0) {
             return new UpsertCommitRuntime(ctx, jobId, datasetId, primaryKeyFields, isTemporaryDatasetWriteJob,
-                    isWriteTransaction, upsertVarIdx);
+                    isWriteTransaction, datasetPartitions[ctx.getTaskAttemptId().getTaskId().getPartition()],
+                    upsertVarIdx);
         } else {
             return new CommitRuntime(ctx, jobId, datasetId, primaryKeyFields, isTemporaryDatasetWriteJob,
-                    isWriteTransaction);
+                    isWriteTransaction, datasetPartitions[ctx.getTaskAttemptId().getTaskId().getPartition()]);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/UpsertCommitRuntime.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/UpsertCommitRuntime.java b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/UpsertCommitRuntime.java
index 81996d1..7358700 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/UpsertCommitRuntime.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/UpsertCommitRuntime.java
@@ -21,22 +21,23 @@ package org.apache.asterix.algebra.operators.physical;
 import java.nio.ByteBuffer;
 
 import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.LogType;
 import org.apache.asterix.common.utils.TransactionUtil;
 import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 
 public class UpsertCommitRuntime extends CommitRuntime {
     private final int upsertIdx;
 
     public UpsertCommitRuntime(IHyracksTaskContext ctx, JobId jobId, int datasetId, int[] primaryKeyFields,
-            boolean isTemporaryDatasetWriteJob, boolean isWriteTransaction, int upsertIdx) throws AlgebricksException {
-        super(ctx, jobId, datasetId, primaryKeyFields, isTemporaryDatasetWriteJob, isWriteTransaction);
+            boolean isTemporaryDatasetWriteJob, boolean isWriteTransaction, int resourcePartition, int upsertIdx) {
+        super(ctx, jobId, datasetId, primaryKeyFields, isTemporaryDatasetWriteJob, isWriteTransaction,
+                resourcePartition);
         this.upsertIdx = upsertIdx;
     }
 
     @Override
-    protected void formLogRecord(ByteBuffer buffer, int t) throws AlgebricksException {
+    protected void formLogRecord(ByteBuffer buffer, int t) {
         boolean isNull = ABooleanSerializerDeserializer.getBoolean(buffer.array(),
                 frameTupleAccessor.getFieldSlotsLength() + frameTupleAccessor.getTupleStartOffset(t)
                         + frameTupleAccessor.getFieldStartOffset(t, upsertIdx) + 1);
@@ -46,8 +47,8 @@ public class UpsertCommitRuntime extends CommitRuntime {
         } else {
             // Previous record found (delete + insert)
             int pkHash = computePrimaryKeyHashValue(frameTupleReference, primaryKeyFields);
-            TransactionUtil.formEntityUpsertCommitLogRecord(logRecord, transactionContext, datasetId, pkHash,
-                    frameTupleReference, primaryKeyFields);
+            TransactionUtil.formEntityCommitLogRecord(logRecord, transactionContext, datasetId, pkHash,
+                    frameTupleReference, primaryKeyFields, resourcePartition, LogType.UPSERT_ENTITY_COMMIT);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
index f8df183..ef8b4a3 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
@@ -68,6 +68,8 @@ public class ReplaceSinkOpWithCommitOpRule implements IAlgebraicRewriteRule {
 
         List<Mutable<ILogicalExpression>> primaryKeyExprs = null;
         int datasetId = 0;
+        String dataverse = null;
+        String datasetName = null;
         AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) sinkOperator.getInputs().get(0).getValue();
         LogicalVariable upsertVar = null;
         AssignOperator upsertFlagAssign = null;
@@ -79,6 +81,10 @@ public class ReplaceSinkOpWithCommitOpRule implements IAlgebraicRewriteRule {
                     primaryKeyExprs = indexInsertDeleteUpsertOperator.getPrimaryKeyExpressions();
                     datasetId = ((DatasetDataSource) indexInsertDeleteUpsertOperator.getDataSourceIndex()
                             .getDataSource()).getDataset().getDatasetId();
+                    dataverse = ((DatasetDataSource) indexInsertDeleteUpsertOperator.getDataSourceIndex()
+                            .getDataSource()).getDataset().getDataverseName();
+                    datasetName = ((DatasetDataSource) indexInsertDeleteUpsertOperator.getDataSourceIndex()
+                            .getDataSource()).getDataset().getDatasetName();
                     break;
                 }
             } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.INSERT_DELETE_UPSERT) {
@@ -87,6 +93,10 @@ public class ReplaceSinkOpWithCommitOpRule implements IAlgebraicRewriteRule {
                     primaryKeyExprs = insertDeleteUpsertOperator.getPrimaryKeyExpressions();
                     datasetId = ((DatasetDataSource) insertDeleteUpsertOperator.getDataSource()).getDataset()
                             .getDatasetId();
+                    dataverse = ((DatasetDataSource) insertDeleteUpsertOperator.getDataSource()).getDataset()
+                            .getDataverseName();
+                    datasetName = ((DatasetDataSource) insertDeleteUpsertOperator.getDataSource()).getDataset()
+                            .getDatasetName();
                     if (insertDeleteUpsertOperator.getOperation() == Kind.UPSERT) {
                         //we need to add a function that checks if previous record was found
                         upsertVar = context.newVar();
@@ -132,7 +142,8 @@ public class ReplaceSinkOpWithCommitOpRule implements IAlgebraicRewriteRule {
 
         //create the logical and physical operator
         CommitOperator commitOperator = new CommitOperator(primaryKeyLogicalVars, upsertVar);
-        CommitPOperator commitPOperator = new CommitPOperator(jobId, datasetId, primaryKeyLogicalVars, upsertVar);
+        CommitPOperator commitPOperator = new CommitPOperator(jobId, dataverse, datasetName, datasetId,
+                primaryKeyLogicalVars, upsertVar);
         commitOperator.setPhysicalOperator(commitPOperator);
 
         //create ExtensionOperator and put the commitOperator in it.

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java b/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
index 8a40876..5532e79 100644
--- a/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
+++ b/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
@@ -22,12 +22,14 @@ import java.io.IOException;
 import java.rmi.RemoteException;
 import java.rmi.server.UnicastRemoteObject;
 import java.util.List;
+import java.util.Set;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import org.apache.asterix.common.api.AsterixThreadExecutor;
 import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.config.AsterixBuildProperties;
 import org.apache.asterix.common.config.AsterixCompilerProperties;
 import org.apache.asterix.common.config.AsterixExternalProperties;
@@ -105,8 +107,6 @@ public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAst
         }
     }
 
-    private static final int METADATA_IO_DEVICE_ID = 0;
-
     private ILSMMergePolicyFactory metadataMergePolicyFactory;
     private final INCApplicationContext ncApplicationContext;
 
@@ -126,7 +126,7 @@ public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAst
     private ITransactionSubsystem txnSubsystem;
 
     private ILSMIOOperationScheduler lsmIOScheduler;
-    private ILocalResourceRepository localResourceRepository;
+    private PersistentLocalResourceRepository localResourceRepository;
     private IResourceIdFactory resourceIdFactory;
     private IIOManager ioManager;
     private boolean isShuttingdown;
@@ -153,6 +153,7 @@ public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAst
         this.metadataRmiPort = metadataRmiPort;
     }
 
+    @Override
     public void initialize(boolean initialRun) throws IOException, ACIDException, AsterixException {
         Logger.getLogger("org.apache").setLevel(externalProperties.getLogLevel());
 
@@ -172,7 +173,7 @@ public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAst
         ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
                 ioManager, ncApplicationContext.getNodeId(), metadataProperties);
 
-        localResourceRepository = persistentLocalResourceRepositoryFactory.createRepository();
+        localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
 
         IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider = new AsterixAppRuntimeContextProdiverForRecovery(
                 this);
@@ -183,7 +184,7 @@ public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAst
         SystemState systemState = recoveryMgr.getSystemState();
         if (initialRun || systemState == SystemState.NEW_UNIVERSE) {
             //delete any storage data before the resource factory is initialized
-            ((PersistentLocalResourceRepository) localResourceRepository).deleteStorageData(true);
+            localResourceRepository.deleteStorageData(true);
         }
         initializeResourceIdFactory();
 
@@ -208,7 +209,22 @@ public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAst
             txnSubsystem.getLogManager().setReplicationManager(replicationManager);
 
             //PersistentLocalResourceRepository to replicate metadata files and delete backups on drop index
-            ((PersistentLocalResourceRepository) localResourceRepository).setReplicationManager(replicationManager);
+            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> replicationClients = replicationProperties.getNodeReplicationClients(nodeId);
+            //remove the node itself
+            replicationClients.remove(nodeId);
+            for (String clientId : replicationClients) {
+                //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(),
@@ -220,7 +236,6 @@ public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAst
             bufferCache = new BufferCache(ioManager, prs, pcp, fileMapManager,
                     storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory(),
                     replicationManager);
-
         } else {
             bufferCache = new BufferCache(ioManager, prs, pcp, fileMapManager,
                     storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory());
@@ -251,57 +266,65 @@ public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAst
         lccm.register((ILifeCycleComponent) txnSubsystem.getLockManager());
     }
 
+    @Override
     public boolean isShuttingdown() {
         return isShuttingdown;
     }
 
+    @Override
     public void setShuttingdown(boolean isShuttingdown) {
         this.isShuttingdown = isShuttingdown;
     }
 
+    @Override
     public void deinitialize() throws HyracksDataException {
     }
 
+    @Override
     public IBufferCache getBufferCache() {
         return bufferCache;
     }
 
+    @Override
     public IFileMapProvider getFileMapManager() {
         return fileMapManager;
     }
 
+    @Override
     public ITransactionSubsystem getTransactionSubsystem() {
         return txnSubsystem;
     }
 
+    @Override
     public IDatasetLifecycleManager getDatasetLifecycleManager() {
         return datasetLifecycleManager;
     }
 
+    @Override
     public double getBloomFilterFalsePositiveRate() {
         return storageProperties.getBloomFilterFalsePositiveRate();
     }
 
+    @Override
     public ILSMIOOperationScheduler getLSMIOScheduler() {
         return lsmIOScheduler;
     }
 
+    @Override
     public ILocalResourceRepository getLocalResourceRepository() {
         return localResourceRepository;
     }
 
+    @Override
     public IResourceIdFactory getResourceIdFactory() {
         return resourceIdFactory;
     }
 
+    @Override
     public IIOManager getIOManager() {
         return ioManager;
     }
 
-    public int getMetaDataIODeviceId() {
-        return METADATA_IO_DEVICE_ID;
-    }
-
     @Override
     public AsterixStorageProperties getStorageProperties() {
         return storageProperties;
@@ -352,6 +375,7 @@ public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAst
         return threadExecutor;
     }
 
+    @Override
     public ILSMMergePolicyFactory getMetadataMergePolicyFactory() {
         return metadataMergePolicyFactory;
     }
@@ -421,4 +445,9 @@ public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAst
         IMetadataNode stub = (IMetadataNode) UnicastRemoteObject.exportObject(MetadataNode.INSTANCE, metadataRmiPort);
         ((IAsterixStateProxy) ncApplicationContext.getDistributedState()).setMetadataNode(stub);
     }
+
+    @Override
+    public void unexportMetadataNodeStub() throws RemoteException {
+        UnicastRemoteObject.unexportObject(MetadataNode.INSTANCE, false);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java b/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
new file mode 100644
index 0000000..dce10ab
--- /dev/null
+++ b/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
@@ -0,0 +1,51 @@
+/*
+ * 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.api.http.servlet;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.asterix.om.util.AsterixClusterProperties;
+import org.apache.asterix.result.ResultUtils;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+public class ClusterAPIServlet extends HttpServlet {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+        response.setContentType("application/json");
+        response.setCharacterEncoding("utf-8");
+        PrintWriter responseWriter = response.getWriter();
+        try {
+            JSONObject responseObject = AsterixClusterProperties.INSTANCE.getClusterStateDescription();
+            responseWriter.write(responseObject.toString());
+            response.setStatus(HttpServletResponse.SC_OK);
+        } catch (JSONException e) {
+            ResultUtils.apiErrorHandler(responseWriter, e);
+            response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+        }
+        responseWriter.flush();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index bee284d..adf0a4d 100644
--- a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -23,6 +23,7 @@ import java.util.logging.Logger;
 
 import org.apache.asterix.api.http.servlet.APIServlet;
 import org.apache.asterix.api.http.servlet.AQLAPIServlet;
+import org.apache.asterix.api.http.servlet.ClusterAPIServlet;
 import org.apache.asterix.api.http.servlet.ConnectorAPIServlet;
 import org.apache.asterix.api.http.servlet.DDLAPIServlet;
 import org.apache.asterix.api.http.servlet.FeedServlet;
@@ -37,7 +38,7 @@ import org.apache.asterix.common.api.AsterixThreadFactory;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.config.AsterixExternalProperties;
 import org.apache.asterix.common.config.AsterixMetadataProperties;
-import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.common.utils.ServletUtil.Servlets;
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
 import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
 import org.apache.asterix.event.service.ILookupService;
@@ -51,7 +52,6 @@ import org.apache.asterix.metadata.bootstrap.AsterixStateProxy;
 import org.apache.asterix.metadata.cluster.ClusterManager;
 import org.apache.asterix.om.util.AsterixAppContextInfo;
 import org.apache.asterix.om.util.AsterixClusterProperties;
-import org.apache.asterix.replication.management.ReplicationLifecycleListener;
 import org.apache.hyracks.api.application.ICCApplicationContext;
 import org.apache.hyracks.api.application.ICCApplicationEntryPoint;
 import org.apache.hyracks.api.client.HyracksConnection;
@@ -88,11 +88,9 @@ public class CCApplicationEntryPoint implements ICCApplicationEntryPoint {
         }
 
         appCtx.setThreadFactory(new AsterixThreadFactory(new LifeCycleComponentManager()));
-        GlobalRecoveryManager.INSTANCE = new GlobalRecoveryManager(
-                (HyracksConnection) getNewHyracksClientConnection());
+        GlobalRecoveryManager.INSTANCE = new GlobalRecoveryManager((HyracksConnection) getNewHyracksClientConnection());
 
-        AsterixAppContextInfo.initialize(appCtx, getNewHyracksClientConnection(),
-                GlobalRecoveryManager.INSTANCE);
+        AsterixAppContextInfo.initialize(appCtx, getNewHyracksClientConnection(), GlobalRecoveryManager.INSTANCE);
 
         proxy = AsterixStateProxy.registerRemoteObject();
         appCtx.setDistributedState(proxy);
@@ -118,13 +116,6 @@ public class CCApplicationEntryPoint implements ICCApplicationEntryPoint {
 
         ClusterManager.INSTANCE.registerSubscriber(GlobalRecoveryManager.INSTANCE);
 
-        AsterixReplicationProperties asterixRepliactionProperties = AsterixAppContextInfo.getInstance()
-                .getReplicationProperties();
-        if (asterixRepliactionProperties.isReplicationEnabled()) {
-            ReplicationLifecycleListener.INSTANCE = new ReplicationLifecycleListener(asterixRepliactionProperties);
-            ClusterManager.INSTANCE.registerSubscriber(ReplicationLifecycleListener.INSTANCE);
-        }
-
         ccAppCtx.addClusterLifecycleListener(ClusterLifecycleListener.INSTANCE);
         ccAppCtx.setMessageBroker(messageBroker);
     }
@@ -178,25 +169,32 @@ public class CCApplicationEntryPoint implements ICCApplicationEntryPoint {
         jsonAPIServer.setHandler(context);
 
         // AQL rest APIs.
-        context.addServlet(new ServletHolder(new QueryAPIServlet(new AqlCompilationProvider())), "/query");
-        context.addServlet(new ServletHolder(new UpdateAPIServlet(new AqlCompilationProvider())), "/update");
-        context.addServlet(new ServletHolder(new DDLAPIServlet(new AqlCompilationProvider())), "/ddl");
-        context.addServlet(new ServletHolder(new AQLAPIServlet(new AqlCompilationProvider())), "/aql");
+        context.addServlet(new ServletHolder(new QueryAPIServlet(new AqlCompilationProvider())),
+                Servlets.AQL_QUERY.getPath());
+        context.addServlet(new ServletHolder(new UpdateAPIServlet(new AqlCompilationProvider())),
+                Servlets.AQL_UPDATE.getPath());
+        context.addServlet(new ServletHolder(new DDLAPIServlet(new AqlCompilationProvider())),
+                Servlets.AQL_DDL.getPath());
+        context.addServlet(new ServletHolder(new AQLAPIServlet(new AqlCompilationProvider())), Servlets.AQL.getPath());
 
         // SQL++ rest APIs.
-        context.addServlet(new ServletHolder(new QueryAPIServlet(new SqlppCompilationProvider())), "/query/sqlpp");
-        context.addServlet(new ServletHolder(new UpdateAPIServlet(new SqlppCompilationProvider())), "/update/sqlpp");
-        context.addServlet(new ServletHolder(new DDLAPIServlet(new SqlppCompilationProvider())), "/ddl/sqlpp");
-        context.addServlet(new ServletHolder(new AQLAPIServlet(new SqlppCompilationProvider())), "/sqlpp");
+        context.addServlet(new ServletHolder(new QueryAPIServlet(new SqlppCompilationProvider())),
+                Servlets.SQLPP_QUERY.getPath());
+        context.addServlet(new ServletHolder(new UpdateAPIServlet(new SqlppCompilationProvider())),
+                Servlets.SQLPP_UPDATE.getPath());
+        context.addServlet(new ServletHolder(new DDLAPIServlet(new SqlppCompilationProvider())),
+                Servlets.SQLPP_DDL.getPath());
+        context.addServlet(new ServletHolder(new AQLAPIServlet(new SqlppCompilationProvider())),
+                Servlets.SQLPP.getPath());
 
         // Other APIs.
-        context.addServlet(new ServletHolder(new QueryStatusAPIServlet()), "/query/status");
-        context.addServlet(new ServletHolder(new QueryResultAPIServlet()), "/query/result");
-        context.addServlet(new ServletHolder(new ConnectorAPIServlet()), "/connector");
-        context.addServlet(new ServletHolder(new ShutdownAPIServlet()), "/admin/shutdown");
-        context.addServlet(new ServletHolder(new VersionAPIServlet()), "/admin/version");
-
-        context.addServlet(new ServletHolder(new QueryServiceServlet()), "/query/service");
+        context.addServlet(new ServletHolder(new QueryStatusAPIServlet()), Servlets.QUERY_STATUS.getPath());
+        context.addServlet(new ServletHolder(new QueryResultAPIServlet()), Servlets.QUERY_RESULT.getPath());
+        context.addServlet(new ServletHolder(new QueryServiceServlet()), Servlets.QUERY_SERVICE.getPath());
+        context.addServlet(new ServletHolder(new ConnectorAPIServlet()), Servlets.CONNECTOR.getPath());
+        context.addServlet(new ServletHolder(new ShutdownAPIServlet()), Servlets.SHUTDOWN.getPath());
+        context.addServlet(new ServletHolder(new VersionAPIServlet()), Servlets.VERSION.getPath());
+        context.addServlet(new ServletHolder(new ClusterAPIServlet()), Servlets.CLUSTER_STATE.getPath());
     }
 
     private void setupFeedServer(AsterixExternalProperties externalProperties) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
index 00b7391..4514fee 100644
--- a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
+++ b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
@@ -62,17 +62,15 @@ public class ClusterLifecycleListener implements IClusterLifecycleListener {
         t.start();
     }
 
-    public enum ClusterEventType {
-        NODE_JOIN,
-        NODE_FAILURE
-    }
-
     @Override
     public void notifyNodeJoin(String nodeId, Map<String, String> ncConfiguration) {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("NC: " + nodeId + " joined");
         }
         AsterixClusterProperties.INSTANCE.addNCConfiguration(nodeId, ncConfiguration);
+        //if metadata node rejoining, we need to rebind the proxy connection when it is active again.
+        MetadataManager.INSTANCE.rebindMetadataNode = !AsterixClusterProperties.INSTANCE.isMetadataNodeActive();
+
         Set<String> nodeAddition = new HashSet<String>();
         nodeAddition.add(nodeId);
         updateProgress(ClusterEventType.NODE_JOIN, nodeAddition);
@@ -90,17 +88,16 @@ public class ClusterLifecycleListener implements IClusterLifecycleListener {
 
     }
 
+    @Override
     public void notifyNodeFailure(Set<String> deadNodeIds) {
         for (String deadNode : deadNodeIds) {
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("NC: " + deadNode + " left");
             }
-            //if metadata node failed, we need to rebind the proxy connection when it joins again.
-            String metadataNode = AsterixClusterProperties.INSTANCE.getCurrentMetadataNode();
-            if (deadNode.equals(metadataNode)) {
-                MetadataManager.INSTANCE.rebindMetadataNode = true;
-            }
             AsterixClusterProperties.INSTANCE.removeNCConfiguration(deadNode);
+
+            //if metadata node failed, we need to rebind the proxy connection when it is active again
+            MetadataManager.INSTANCE.rebindMetadataNode = !AsterixClusterProperties.INSTANCE.isMetadataNodeActive();
         }
         updateProgress(ClusterEventType.NODE_FAILURE, deadNodeIds);
         Set<IClusterEventsSubscriber> subscribers = ClusterManager.INSTANCE.getRegisteredClusterEventSubscribers();

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
index dac9af5..fcb196d 100644
--- a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.hyracks.bootstrap;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -30,7 +29,6 @@ import org.apache.asterix.api.common.AsterixAppRuntimeContext;
 import org.apache.asterix.common.api.AsterixThreadFactory;
 import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
 import org.apache.asterix.common.config.AsterixMetadataProperties;
-import org.apache.asterix.common.config.AsterixReplicationProperties;
 import org.apache.asterix.common.config.AsterixTransactionProperties;
 import org.apache.asterix.common.config.IAsterixPropertiesProvider;
 import org.apache.asterix.common.messaging.api.INCMessageBroker;
@@ -71,8 +69,7 @@ public class NCApplicationEntryPoint implements INCApplicationEntryPoint {
     private boolean isMetadataNode = false;
     private boolean stopInitiated = false;
     private SystemState systemState = SystemState.NEW_UNIVERSE;
-    private boolean performedRemoteRecovery = false;
-    private boolean replicationEnabled = false;
+    private boolean pendingFailbackCompletion = false;
     private IMessageBroker messageBroker;
 
     @Override
@@ -90,8 +87,7 @@ public class NCApplicationEntryPoint implements INCApplicationEntryPoint {
 
         ncAppCtx.setThreadFactory(new AsterixThreadFactory(ncAppCtx.getLifeCycleComponentManager()));
         ncApplicationContext = ncAppCtx;
-        messageBroker = new NCMessageBroker((NodeControllerService) ncAppCtx.getControllerService());
-        ncApplicationContext.setMessageBroker(messageBroker);
+
         nodeId = ncApplicationContext.getNodeId();
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting Asterix node controller: " + nodeId);
@@ -108,17 +104,14 @@ public class NCApplicationEntryPoint implements INCApplicationEntryPoint {
         }
         runtimeContext.initialize(initialRun);
         ncApplicationContext.setApplicationObject(runtimeContext);
+        messageBroker = new NCMessageBroker((NodeControllerService) ncAppCtx.getControllerService());
+        ncApplicationContext.setMessageBroker(messageBroker);
 
-        //If replication is enabled, check if there is a replica for this node
-        AsterixReplicationProperties asterixReplicationProperties = ((IAsterixPropertiesProvider) runtimeContext)
-                .getReplicationProperties();
-
-        replicationEnabled = asterixReplicationProperties.isReplicationEnabled();
-
+        boolean replicationEnabled = AsterixClusterProperties.INSTANCE.isReplicationEnabled();
+        boolean autoFailover = AsterixClusterProperties.INSTANCE.isAutoFailoverEnabled();
         if (initialRun) {
             LOGGER.info("System is being initialized. (first run)");
         } else {
-            //#. recover if the system is corrupted by checking system state.
             IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
             systemState = recoveryMgr.getSystemState();
 
@@ -130,36 +123,41 @@ public class NCApplicationEntryPoint implements INCApplicationEntryPoint {
                 if (systemState == SystemState.NEW_UNIVERSE || systemState == SystemState.CORRUPTED) {
                     //Try to perform remote recovery
                     IRemoteRecoveryManager remoteRecoveryMgr = runtimeContext.getRemoteRecoveryManager();
-                    remoteRecoveryMgr.performRemoteRecovery();
-                    performedRemoteRecovery = true;
-                    systemState = SystemState.HEALTHY;
+                    if (autoFailover) {
+                        remoteRecoveryMgr.startFailbackProcess();
+                        systemState = SystemState.RECOVERING;
+                        pendingFailbackCompletion = true;
+                    } else {
+                        remoteRecoveryMgr.performRemoteRecovery();
+                        systemState = SystemState.HEALTHY;
+                    }
+                }
+            } else {
+                //recover if the system is corrupted by checking system state.
+                if (systemState == SystemState.CORRUPTED) {
+                    recoveryMgr.startRecovery(true);
                 }
-            }
-
-            if (systemState == SystemState.CORRUPTED) {
-                recoveryMgr.startRecovery(true);
             }
         }
 
-        if (replicationEnabled) {
+        /**
+         * if the node pending failback completion, the replication channel
+         * should not be opened to avoid other nodes connecting to it before
+         * the node completes its failback. CC will notify other replicas once
+         * this node is ready to receive replication requests.
+         */
+        if (replicationEnabled && !pendingFailbackCompletion) {
             startReplicationService();
         }
     }
 
-    private void startReplicationService() throws IOException {
+    private void startReplicationService() {
         //Open replication channel
         runtimeContext.getReplicationChannel().start();
 
         //Check the state of remote replicas
         runtimeContext.getReplicationManager().initializeReplicasState();
 
-        if (performedRemoteRecovery) {
-            //Notify remote replicas about the new IP Address if changed
-            //Note: this is a hack since each node right now maintains its own copy of the cluster configuration.
-            //Once the configuration is centralized on the CC, this step wont be needed.
-            runtimeContext.getReplicationManager().broadcastNewIPAddress();
-        }
-
         //Start replication after the state of remote replicas has been initialized.
         runtimeContext.getReplicationManager().startReplicationThreads();
     }
@@ -211,10 +209,10 @@ public class NCApplicationEntryPoint implements INCApplicationEntryPoint {
         }
 
         isMetadataNode = nodeId.equals(metadataProperties.getMetadataNodeName());
-        if (isMetadataNode) {
+        if (isMetadataNode && !pendingFailbackCompletion) {
             runtimeContext.initializeMetadata(systemState == SystemState.NEW_UNIVERSE);
         }
-        ExternalLibraryBootstrap.setUpExternaLibraries(isMetadataNode);
+        ExternalLibraryBootstrap.setUpExternaLibraries(isMetadataNode && !pendingFailbackCompletion);
 
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting lifecycle components");
@@ -237,11 +235,13 @@ public class NCApplicationEntryPoint implements INCApplicationEntryPoint {
 
         lccm.startAll();
 
-        IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
-        recoveryMgr.checkpoint(true, RecoveryManager.NON_SHARP_CHECKPOINT_TARGET_LSN);
+        if (!pendingFailbackCompletion) {
+            IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
+            recoveryMgr.checkpoint(true, RecoveryManager.NON_SHARP_CHECKPOINT_TARGET_LSN);
 
-        if (isMetadataNode) {
-            runtimeContext.exportMetadataNodeStub();
+            if (isMetadataNode) {
+                runtimeContext.exportMetadataNodeStub();
+            }
         }
 
         //Clean any temporary files

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java b/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
index aeaef59..258bc35 100644
--- a/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
+++ b/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
@@ -22,9 +22,12 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import org.apache.asterix.common.messaging.AbstractApplicationMessage;
+import org.apache.asterix.common.messaging.PreparePartitionsFailbackResponseMessage;
+import org.apache.asterix.common.messaging.CompleteFailbackResponseMessage;
 import org.apache.asterix.common.messaging.ReportMaxResourceIdMessage;
 import org.apache.asterix.common.messaging.ReportMaxResourceIdRequestMessage;
 import org.apache.asterix.common.messaging.ResourceIdRequestMessage;
@@ -54,6 +57,9 @@ public class CCMessageBroker implements ICCMessageBroker {
     @Override
     public void receivedMessage(IMessage message, String nodeId) throws Exception {
         AbstractApplicationMessage absMessage = (AbstractApplicationMessage) message;
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Received message: " + absMessage.getMessageType().name());
+        }
         switch (absMessage.getMessageType()) {
             case RESOURCE_ID_REQUEST:
                 handleResourceIdRequest(message, nodeId);
@@ -67,6 +73,12 @@ public class CCMessageBroker implements ICCMessageBroker {
             case TAKEOVER_METADATA_NODE_RESPONSE:
                 handleTakeoverMetadataNodeResponse(message);
                 break;
+            case PREPARE_PARTITIONS_FAILBACK_RESPONSE:
+                handleClosePartitionsResponse(message);
+                break;
+            case COMPLETE_FAILBACK_RESPONSE:
+                handleCompleteFailbcakResponse(message);
+                break;
             default:
                 LOGGER.warning("Unknown message: " + absMessage.getMessageType());
                 break;
@@ -78,7 +90,7 @@ public class CCMessageBroker implements ICCMessageBroker {
         ResourceIdRequestResponseMessage reponse = new ResourceIdRequestResponseMessage();
         reponse.setId(msg.getId());
         //cluster is not active
-        if (!AsterixClusterProperties.isClusterActive()) {
+        if (!AsterixClusterProperties.INSTANCE.isClusterActive()) {
             reponse.setResourceId(-1);
             reponse.setException(new Exception("Cannot generate global resource id when cluster is not active."));
         } else if (nodesReportedMaxResourceId.size() < AsterixClusterProperties.getNumberOfNodes()) {
@@ -126,4 +138,14 @@ public class CCMessageBroker implements ICCMessageBroker {
         TakeoverMetadataNodeResponseMessage msg = (TakeoverMetadataNodeResponseMessage) message;
         AsterixClusterProperties.INSTANCE.processMetadataNodeTakeoverResponse(msg);
     }
+
+    private void handleCompleteFailbcakResponse(IMessage message) {
+        CompleteFailbackResponseMessage msg = (CompleteFailbackResponseMessage) message;
+        AsterixClusterProperties.INSTANCE.processCompleteFailbackResponse(msg);
+    }
+
+    private void handleClosePartitionsResponse(IMessage message) {
+        PreparePartitionsFailbackResponseMessage msg = (PreparePartitionsFailbackResponseMessage) message;
+        AsterixClusterProperties.INSTANCE.processPreparePartitionsFailbackResponse(msg);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-app/src/main/java/org/apache/asterix/messaging/NCMessageBroker.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/org/apache/asterix/messaging/NCMessageBroker.java b/asterix-app/src/main/java/org/apache/asterix/messaging/NCMessageBroker.java
index 8f8723e..0a0a917 100644
--- a/asterix-app/src/main/java/org/apache/asterix/messaging/NCMessageBroker.java
+++ b/asterix-app/src/main/java/org/apache/asterix/messaging/NCMessageBroker.java
@@ -21,9 +21,16 @@ package org.apache.asterix.messaging;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
 import org.apache.asterix.common.messaging.AbstractApplicationMessage;
+import org.apache.asterix.common.messaging.CompleteFailbackRequestMessage;
+import org.apache.asterix.common.messaging.CompleteFailbackResponseMessage;
+import org.apache.asterix.common.messaging.PreparePartitionsFailbackRequestMessage;
+import org.apache.asterix.common.messaging.PreparePartitionsFailbackResponseMessage;
+import org.apache.asterix.common.messaging.ReplicaEventMessage;
 import org.apache.asterix.common.messaging.ReportMaxResourceIdMessage;
 import org.apache.asterix.common.messaging.TakeoverMetadataNodeResponseMessage;
 import org.apache.asterix.common.messaging.TakeoverPartitionsRequestMessage;
@@ -32,18 +39,26 @@ import org.apache.asterix.common.messaging.api.IApplicationMessage;
 import org.apache.asterix.common.messaging.api.IApplicationMessageCallback;
 import org.apache.asterix.common.messaging.api.INCMessageBroker;
 import org.apache.asterix.common.replication.IRemoteRecoveryManager;
+import org.apache.asterix.common.replication.Replica;
+import org.apache.asterix.common.replication.ReplicaEvent;
+import org.apache.asterix.event.schema.cluster.Node;
 import org.apache.asterix.metadata.bootstrap.MetadataIndexImmutableProperties;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import org.apache.hyracks.api.messages.IMessage;
 import org.apache.hyracks.api.util.JavaSerializationUtils;
 import org.apache.hyracks.control.nc.NodeControllerService;
 
 public class NCMessageBroker implements INCMessageBroker {
+    private final static Logger LOGGER = Logger.getLogger(NCMessageBroker.class.getName());
+
     private final NodeControllerService ncs;
     private final AtomicLong messageId = new AtomicLong(0);
     private final Map<Long, IApplicationMessageCallback> callbacks;
+    private final IAsterixAppRuntimeContext appContext;
 
     public NCMessageBroker(NodeControllerService ncs) {
         this.ncs = ncs;
+        appContext = (IAsterixAppRuntimeContext) ncs.getApplicationContext().getApplicationObject();
         callbacks = new ConcurrentHashMap<Long, IApplicationMessageCallback>();
     }
 
@@ -67,56 +82,72 @@ public class NCMessageBroker implements INCMessageBroker {
 
     @Override
     public void receivedMessage(IMessage message, String nodeId) throws Exception {
-        AbstractApplicationMessage absMessage = (AbstractApplicationMessage) message;
-        //if the received message is a response to a sent message, deliver it to the sender
-        IApplicationMessageCallback callback = callbacks.remove(absMessage.getId());
-        if (callback != null) {
-            callback.deliverMessageResponse(absMessage);
-        }
+        try {
+            AbstractApplicationMessage absMessage = (AbstractApplicationMessage) message;
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Received message: " + absMessage.getMessageType().name());
+            }
+            //if the received message is a response to a sent message, deliver it to the sender
+            IApplicationMessageCallback callback = callbacks.remove(absMessage.getId());
+            if (callback != null) {
+                callback.deliverMessageResponse(absMessage);
+            }
 
-        //handle requests from CC
-        switch (absMessage.getMessageType()) {
-            case REPORT_MAX_RESOURCE_ID_REQUEST:
-                reportMaxResourceId();
-                break;
-            case TAKEOVER_PARTITIONS_REQUEST:
-                handleTakeoverPartitons(message);
-                break;
-            case TAKEOVER_METADATA_NODE_REQUEST:
-                handleTakeoverMetadataNode(message);
-                break;
-            default:
-                break;
+            //handle requests from CC
+            switch (absMessage.getMessageType()) {
+                case REPORT_MAX_RESOURCE_ID_REQUEST:
+                    reportMaxResourceId();
+                    break;
+                case TAKEOVER_PARTITIONS_REQUEST:
+                    handleTakeoverPartitons(message);
+                    break;
+                case TAKEOVER_METADATA_NODE_REQUEST:
+                    handleTakeoverMetadataNode(message);
+                    break;
+                case PREPARE_PARTITIONS_FAILBACK_REQUEST:
+                    handlePreparePartitionsFailback(message);
+                    break;
+                case COMPLETE_FAILBACK_REQUEST:
+                    handleCompleteFailbackRequest(message);
+                    break;
+                case REPLICA_EVENT:
+                    handleReplicaEvent(message);
+                    break;
+                default:
+                    break;
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw e;
         }
     }
 
     private void handleTakeoverPartitons(IMessage message) throws Exception {
         TakeoverPartitionsRequestMessage msg = (TakeoverPartitionsRequestMessage) message;
-        IAsterixAppRuntimeContext appContext = (IAsterixAppRuntimeContext) ncs.getApplicationContext()
-                .getApplicationObject();
-        IRemoteRecoveryManager remoteRecoeryManager = appContext.getRemoteRecoveryManager();
-        remoteRecoeryManager.takeoverPartitons(msg.getFailedNode(), msg.getPartitions());
-        //send response after takeover is completed
-        TakeoverPartitionsResponseMessage reponse = new TakeoverPartitionsResponseMessage(msg.getRequestId(),
-                appContext.getTransactionSubsystem().getId(), msg.getPartitions());
-        sendMessage(reponse, null);
+        try {
+            IRemoteRecoveryManager remoteRecoeryManager = appContext.getRemoteRecoveryManager();
+            remoteRecoeryManager.takeoverPartitons(msg.getPartitions());
+        } finally {
+            //send response after takeover is completed
+            TakeoverPartitionsResponseMessage reponse = new TakeoverPartitionsResponseMessage(msg.getRequestId(),
+                    appContext.getTransactionSubsystem().getId(), msg.getPartitions());
+            sendMessage(reponse, null);
+        }
     }
 
     private void handleTakeoverMetadataNode(IMessage message) throws Exception {
-        IAsterixAppRuntimeContext appContext = (IAsterixAppRuntimeContext) ncs.getApplicationContext()
-                .getApplicationObject();
-        appContext.initializeMetadata(false);
-        appContext.exportMetadataNodeStub();
-        //send response after takeover is completed
-        TakeoverMetadataNodeResponseMessage reponse = new TakeoverMetadataNodeResponseMessage(
-                appContext.getTransactionSubsystem().getId());
-        sendMessage(reponse, null);
+        try {
+            appContext.initializeMetadata(false);
+            appContext.exportMetadataNodeStub();
+        } finally {
+            TakeoverMetadataNodeResponseMessage reponse = new TakeoverMetadataNodeResponseMessage(
+                    appContext.getTransactionSubsystem().getId());
+            sendMessage(reponse, null);
+        }
     }
 
     @Override
     public void reportMaxResourceId() throws Exception {
-        IAsterixAppRuntimeContext appContext = (IAsterixAppRuntimeContext) ncs.getApplicationContext()
-                .getApplicationObject();
         ReportMaxResourceIdMessage maxResourceIdMsg = new ReportMaxResourceIdMessage();
         //resource ids < FIRST_AVAILABLE_USER_DATASET_ID are reserved for metadata indexes.
         long maxResourceId = Math.max(appContext.getLocalResourceRepository().getMaxResourceID(),
@@ -124,4 +155,58 @@ public class NCMessageBroker implements INCMessageBroker {
         maxResourceIdMsg.setMaxResourceId(maxResourceId);
         sendMessage(maxResourceIdMsg, null);
     }
+
+    private void handleReplicaEvent(IMessage message) {
+        ReplicaEventMessage msg = (ReplicaEventMessage) message;
+        Node node = new Node();
+        node.setId(msg.getNodeId());
+        node.setClusterIp(msg.getNodeIPAddress());
+        Replica replica = new Replica(node);
+        ReplicaEvent event = new ReplicaEvent(replica, msg.getEvent());
+        appContext.getReplicationManager().reportReplicaEvent(event);
+    }
+
+    private void handlePreparePartitionsFailback(IMessage message) throws Exception {
+        PreparePartitionsFailbackRequestMessage msg = (PreparePartitionsFailbackRequestMessage) message;
+        /**
+         * 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 (msg.isReleaseMetadataNode()) {
+            appContext.getDatasetLifecycleManager().closeAllDatasets();
+            //remove the metadata node stub from RMI registry
+            appContext.unexportMetadataNodeStub();
+        } 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 : msg.getPartitions()) {
+            localResourceRepo.addInactivePartition(partitionId);
+        }
+
+        //send response after partitions prepared for failback
+        PreparePartitionsFailbackResponseMessage reponse = new PreparePartitionsFailbackResponseMessage(msg.getPlanId(),
+                msg.getRequestId(), msg.getPartitions());
+        sendMessage(reponse, null);
+    }
+
+    private void handleCompleteFailbackRequest(IMessage message) throws Exception {
+        CompleteFailbackRequestMessage msg = (CompleteFailbackRequestMessage) message;
+        try {
+            IRemoteRecoveryManager remoteRecoeryManager = appContext.getRemoteRecoveryManager();
+            remoteRecoeryManager.completeFailbackProcess();
+        } finally {
+            CompleteFailbackResponseMessage reponse = new CompleteFailbackResponseMessage(msg.getPlanId(),
+                    msg.getRequestId(), msg.getPartitions());
+            sendMessage(reponse, null);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java b/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java
index 975180b..496a10e 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java
@@ -55,8 +55,6 @@ public interface IAsterixAppRuntimeContext {
 
     public ILSMMergePolicyFactory getMetadataMergePolicyFactory();
 
-    public int getMetaDataIODeviceId();
-
     public IBufferCache getBufferCache();
 
     public IFileMapProvider getFileMapManager();
@@ -93,14 +91,23 @@ public interface IAsterixAppRuntimeContext {
 
     /**
      * Exports the metadata node to the metadata RMI port.
+     *
      * @throws RemoteException
      */
     public void exportMetadataNodeStub() throws RemoteException;
 
     /**
      * Initializes the metadata node and bootstraps the metadata.
+     *
      * @param newUniverse
      * @throws Exception
      */
     public void initializeMetadata(boolean newUniverse) throws Exception;
+
+    /**
+     * Unexports the metadata node from the RMI registry
+     *
+     * @throws RemoteException
+     */
+    public void unexportMetadataNodeStub() throws RemoteException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java b/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
index 75f1f82..adf8e38 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
@@ -28,7 +28,8 @@ public interface IClusterManagementWork {
     public enum ClusterState {
         STARTING,
         ACTIVE,
-        UNUSABLE
+        UNUSABLE,
+        REBALANCING
     }
 
     public WorkType getClusterManagementWorkType();

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java b/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
index 7e02faf..3b4617a 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
@@ -94,4 +94,11 @@ public interface IDatasetLifecycleManager extends IIndexLifecycleManager {
      * @return a list of all indexes that are open at the time of the call.
      */
     List<IndexInfo> getOpenIndexesInfo();
+
+    /**
+     * Flushes and closes all user datasets (non-metadata datasets)
+     *
+     * @throws HyracksDataException
+     */
+    void closeUserDatasets() throws HyracksDataException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixReplicationProperties.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixReplicationProperties.java b/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixReplicationProperties.java
index fa5b503..019c168 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixReplicationProperties.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixReplicationProperties.java
@@ -34,7 +34,7 @@ public class AsterixReplicationProperties extends AbstractAsterixProperties {
     private static int REPLICATION_DATAPORT_DEFAULT = 2000;
     private static int REPLICATION_FACTOR_DEFAULT = 1;
     private static int REPLICATION_TIME_OUT_DEFAULT = 15;
-
+    private static final int MAX_REMOTE_RECOVERY_ATTEMPTS = 5;
     private static final String NODE_IP_ADDRESS_DEFAULT = "127.0.0.1";
     private final String NODE_NAME_PREFIX;
     private final Cluster cluster;
@@ -88,10 +88,10 @@ public class AsterixReplicationProperties extends AbstractAsterixProperties {
         Set<Replica> remoteReplicas = new HashSet<Replica>();;
 
         int numberOfRemoteReplicas = getReplicationFactor() - 1;
-
         //Using chained-declustering
         if (cluster != null) {
             int nodeIndex = -1;
+            //find the node index in the cluster config
             for (int i = 0; i < cluster.getNode().size(); i++) {
                 Node node = cluster.getNode().get(i);
                 if (getRealCluserNodeID(node.getId()).equals(nodeId)) {
@@ -106,19 +106,18 @@ public class AsterixReplicationProperties extends AbstractAsterixProperties {
                 return null;
             }
 
+            //find nodes to the right of this node
             for (int i = nodeIndex + 1; i < cluster.getNode().size(); i++) {
                 remoteReplicas.add(getReplicaByNodeIndex(i));
-
                 if (remoteReplicas.size() == numberOfRemoteReplicas) {
                     break;
                 }
             }
 
+            //if not all remote replicas have been found, start from the beginning
             if (remoteReplicas.size() != numberOfRemoteReplicas) {
                 for (int i = 0; i < cluster.getNode().size(); i++) {
-
                     remoteReplicas.add(getReplicaByNodeIndex(i));
-
                     if (remoteReplicas.size() == numberOfRemoteReplicas) {
                         break;
                     }
@@ -194,4 +193,51 @@ public class AsterixReplicationProperties extends AbstractAsterixProperties {
         }
         return REPLICATION_TIME_OUT_DEFAULT;
     }
+
+    /**
+     * @param nodeId
+     * @return The set of nodes which replicate to this node, including the node itself
+     */
+    public Set<String> getNodeReplicationClients(String nodeId) {
+        Set<String> clientReplicas = new HashSet<>();
+        clientReplicas.add(nodeId);
+
+        int clientsCount = getReplicationFactor();
+
+        //Using chained-declustering backwards
+        if (cluster != null) {
+            int nodeIndex = -1;
+            //find the node index in the cluster config
+            for (int i = 0; i < cluster.getNode().size(); i++) {
+                Node node = cluster.getNode().get(i);
+                if (getRealCluserNodeID(node.getId()).equals(nodeId)) {
+                    nodeIndex = i;
+                    break;
+                }
+            }
+
+            //find nodes to the left of this node
+            for (int i = nodeIndex - 1; i >= 0; i--) {
+                clientReplicas.add(getReplicaByNodeIndex(i).getId());
+                if (clientReplicas.size() == clientsCount) {
+                    break;
+                }
+            }
+
+            //if not all client replicas have been found, start from the end
+            if (clientReplicas.size() != clientsCount) {
+                for (int i = cluster.getNode().size() - 1; i >= 0; i--) {
+                    clientReplicas.add(getReplicaByNodeIndex(i).getId());
+                    if (clientReplicas.size() == clientsCount) {
+                        break;
+                    }
+                }
+            }
+        }
+        return clientReplicas;
+    }
+
+    public int getMaxRemoteRecoveryAttempts() {
+        return MAX_REMOTE_RECOVERY_ATTEMPTS;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java b/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
index 53902e1..0cd88d6 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
@@ -75,7 +75,6 @@ public class DatasetLifecycleManager implements IDatasetLifecycleManager, ILifeC
         capacity = storageProperties.getMemoryComponentGlobalBudget();
         used = 0;
         logRecord = new LogRecord();
-        logRecord.setNodeId(logManager.getNodeId());
     }
 
     @Override
@@ -543,7 +542,8 @@ public class DatasetLifecycleManager implements IDatasetLifecycleManager, ILifeC
     private void flushDatasetOpenIndexes(DatasetInfo dsInfo, boolean asyncFlush) throws HyracksDataException {
         if (!dsInfo.isExternal) {
             synchronized (logRecord) {
-                TransactionUtil.formFlushLogRecord(logRecord, dsInfo.datasetID, null, dsInfo.indexes.size());
+                TransactionUtil.formFlushLogRecord(logRecord, dsInfo.datasetID, null, logManager.getNodeId(),
+                        dsInfo.indexes.size());
                 try {
                     logManager.log(logRecord);
                 } catch (ACIDException e) {
@@ -612,13 +612,24 @@ public class DatasetLifecycleManager implements IDatasetLifecycleManager, ILifeC
     }
 
     @Override
-    public void closeAllDatasets() throws HyracksDataException {
-        for (DatasetInfo dsInfo : datasetInfos.values()) {
+    public synchronized void closeAllDatasets() throws HyracksDataException {
+        List<DatasetInfo> openDatasets = new ArrayList<>(datasetInfos.values());
+        for (DatasetInfo dsInfo : openDatasets) {
             closeDataset(dsInfo);
         }
     }
 
     @Override
+    public synchronized void closeUserDatasets() throws HyracksDataException {
+        List<DatasetInfo> openDatasets = new ArrayList<>(datasetInfos.values());
+        for (DatasetInfo dsInfo : openDatasets) {
+            if (dsInfo.datasetID >= firstAvilableUserDatasetID) {
+                closeDataset(dsInfo);
+            }
+        }
+    }
+
+    @Override
     public synchronized void stop(boolean dumpState, OutputStream outputStream) throws IOException {
         if (dumpState) {
             dumpState(outputStream);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java b/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
index d25e51f..afdbb31 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
@@ -65,7 +65,8 @@ public class AsterixLSMInsertDeleteOperatorNodePushable extends LSMIndexInsertUp
         try {
             writer.open();
             modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
-                    indexHelper.getResourcePath(), indexHelper.getResourceID(), lsmIndex, ctx);
+                    indexHelper.getResourcePath(), indexHelper.getResourceID(), indexHelper.getResourcePartition(),
+                    lsmIndex, ctx);
             indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
             ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
             if (tupleFilterFactory != null) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/messaging/AbstractFailbackPlanMessage.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/AbstractFailbackPlanMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/AbstractFailbackPlanMessage.java
new file mode 100644
index 0000000..2c72051
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/AbstractFailbackPlanMessage.java
@@ -0,0 +1,39 @@
+/*
+ * 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.messaging;
+
+public abstract class AbstractFailbackPlanMessage extends AbstractApplicationMessage {
+
+    private static final long serialVersionUID = 1L;
+    protected final long planId;
+    protected final int requestId;
+
+    public AbstractFailbackPlanMessage(long planId, int requestId) {
+        this.planId = planId;
+        this.requestId = requestId;
+    }
+
+    public long getPlanId() {
+        return planId;
+    }
+
+    public int getRequestId() {
+        return requestId;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackRequestMessage.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackRequestMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackRequestMessage.java
new file mode 100644
index 0000000..510817a
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackRequestMessage.java
@@ -0,0 +1,56 @@
+/*
+ * 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.messaging;
+
+import java.util.Set;
+
+public class CompleteFailbackRequestMessage extends AbstractFailbackPlanMessage {
+
+    private static final long serialVersionUID = 1L;
+    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;
+    }
+
+    @Override
+    public ApplicationMessageType getMessageType() {
+        return ApplicationMessageType.COMPLETE_FAILBACK_REQUEST;
+    }
+
+    public Set<Integer> getPartitions() {
+        return partitions;
+    }
+
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("Plan ID: " + planId);
+        sb.append(" Node ID: " + nodeId);
+        sb.append(" Partitions: " + partitions);
+        return sb.toString();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackResponseMessage.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackResponseMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackResponseMessage.java
new file mode 100644
index 0000000..6d77920
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackResponseMessage.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.common.messaging;
+
+import java.util.Set;
+
+public class CompleteFailbackResponseMessage extends AbstractFailbackPlanMessage {
+
+    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;
+    }
+
+    @Override
+    public ApplicationMessageType getMessageType() {
+        return ApplicationMessageType.COMPLETE_FAILBACK_RESPONSE;
+    }
+
+    public Set<Integer> getPartitions() {
+        return partitions;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("Plan ID: " + planId);
+        sb.append(" Partitions: " + partitions);
+        return sb.toString();
+    }
+}
\ No newline at end of file


[5/6] incubator-asterixdb git commit: Asterix NCs Failback Support

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackRequestMessage.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackRequestMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackRequestMessage.java
new file mode 100644
index 0000000..7909a35
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackRequestMessage.java
@@ -0,0 +1,65 @@
+/*
+ * 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.messaging;
+
+import java.util.Set;
+
+public class PreparePartitionsFailbackRequestMessage extends AbstractFailbackPlanMessage {
+
+    private static final long serialVersionUID = 1L;
+    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;
+    }
+
+    @Override
+    public ApplicationMessageType getMessageType() {
+        return ApplicationMessageType.PREPARE_PARTITIONS_FAILBACK_REQUEST;
+    }
+
+    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("Plan ID: " + planId);
+        sb.append(" Partitions: " + partitions);
+        sb.append(" releaseMetadataNode: " + releaseMetadataNode);
+        return sb.toString();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackResponseMessage.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackResponseMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackResponseMessage.java
new file mode 100644
index 0000000..6b058c7
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackResponseMessage.java
@@ -0,0 +1,41 @@
+/*
+ * 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.messaging;
+
+import java.util.Set;
+
+public class PreparePartitionsFailbackResponseMessage extends AbstractFailbackPlanMessage {
+
+    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;
+    }
+
+    @Override
+    public ApplicationMessageType getMessageType() {
+        return ApplicationMessageType.PREPARE_PARTITIONS_FAILBACK_RESPONSE;
+    }
+
+    public Set<Integer> getPartitions() {
+        return partitions;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/messaging/ReplicaEventMessage.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/ReplicaEventMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/ReplicaEventMessage.java
new file mode 100644
index 0000000..28fd36f
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/ReplicaEventMessage.java
@@ -0,0 +1,52 @@
+/*
+ * 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.messaging;
+
+import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
+
+public class ReplicaEventMessage extends AbstractApplicationMessage {
+
+    private static final long serialVersionUID = 1L;
+    private final String nodeId;
+    private final ClusterEventType event;
+    private final String nodeIPAddress;
+
+    public ReplicaEventMessage(String nodeId, String nodeIPAddress, ClusterEventType event) {
+        this.nodeId = nodeId;
+        this.nodeIPAddress = nodeIPAddress;
+        this.event = event;
+    }
+
+    @Override
+    public ApplicationMessageType getMessageType() {
+        return ApplicationMessageType.REPLICA_EVENT;
+    }
+
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public ClusterEventType getEvent() {
+        return event;
+    }
+
+    public String getNodeIPAddress() {
+        return nodeIPAddress;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/messaging/TakeoverPartitionsRequestMessage.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/TakeoverPartitionsRequestMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/TakeoverPartitionsRequestMessage.java
index abfa7d2..ad5234b 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/messaging/TakeoverPartitionsRequestMessage.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/TakeoverPartitionsRequestMessage.java
@@ -22,15 +22,12 @@ public class TakeoverPartitionsRequestMessage extends AbstractApplicationMessage
 
     private static final long serialVersionUID = 1L;
     private final Integer[] partitions;
-    private final String failedNode;
     private final long requestId;
     private final String nodeId;
 
-    public TakeoverPartitionsRequestMessage(long requestId, String nodeId, String failedNode,
-            Integer[] partitionsToTakeover) {
+    public TakeoverPartitionsRequestMessage(long requestId, String nodeId, Integer[] partitionsToTakeover) {
         this.requestId = requestId;
         this.nodeId = nodeId;
-        this.failedNode = failedNode;
         this.partitions = partitionsToTakeover;
     }
 
@@ -47,10 +44,6 @@ public class TakeoverPartitionsRequestMessage extends AbstractApplicationMessage
         return requestId;
     }
 
-    public String getFailedNode() {
-        return failedNode;
-    }
-
     public String getNodeId() {
         return nodeId;
     }
@@ -60,7 +53,6 @@ public class TakeoverPartitionsRequestMessage extends AbstractApplicationMessage
         StringBuilder sb = new StringBuilder();
         sb.append("Request ID: " + requestId);
         sb.append(" Node ID: " + nodeId);
-        sb.append(" Failed Node: " + failedNode);
         sb.append(" Partitions: ");
         for (Integer partitionId : partitions) {
             sb.append(partitionId + ",");

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/messaging/api/IApplicationMessage.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/api/IApplicationMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/api/IApplicationMessage.java
index 57a0dae..5d2e263 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/messaging/api/IApplicationMessage.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/api/IApplicationMessage.java
@@ -30,7 +30,12 @@ public interface IApplicationMessage extends IMessage {
         TAKEOVER_PARTITIONS_REQUEST,
         TAKEOVER_PARTITIONS_RESPONSE,
         TAKEOVER_METADATA_NODE_REQUEST,
-        TAKEOVER_METADATA_NODE_RESPONSE
+        TAKEOVER_METADATA_NODE_RESPONSE,
+        PREPARE_PARTITIONS_FAILBACK_REQUEST,
+        PREPARE_PARTITIONS_FAILBACK_RESPONSE,
+        COMPLETE_FAILBACK_REQUEST,
+        COMPLETE_FAILBACK_RESPONSE,
+        REPLICA_EVENT
     }
 
     public abstract ApplicationMessageType getMessageType();

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java
index ecc9494..6d5918f 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java
@@ -36,6 +36,17 @@ public interface IRemoteRecoveryManager {
      * @throws IOException
      * @throws ACIDException
      */
-    public void takeoverPartitons(String failedNode, Integer[] partitions) throws IOException, 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
+     */
+    public void completeFailbackProcess() throws IOException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java
index f13d300..e22fafc 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java
@@ -23,14 +23,8 @@ import java.util.Set;
 public interface IReplicaResourcesManager {
 
     /**
-     * @param remoteNodes
-     * @return The minimum LSN of all indexes that belong to {@code remoteNodes}.
-     */
-    public long getMinRemoteLSN(Set<String> remoteNodes);
-
-    /**
      * @param partitions
      * @return the minimum LSN of all indexes that belong to {@code partitions}.
      */
-    public long getPartitionsMinLSN(Integer[] partitions);
+    public long getPartitionsMinLSN(Set<Integer> partitions);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java
index 76f8767..3fc2af0 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java
@@ -18,8 +18,8 @@
  */
 package org.apache.asterix.common.replication;
 
+import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.ACIDException;
@@ -54,12 +54,13 @@ public interface IReplicationManager extends IIOReplicationManager {
      *            Get logs that belong to those replicas.
      * @param fromLSN
      *            Low water mark for logs to be requested.
-     * @return The logs received that belong to the local node.
+     * @param recoveryLogsFile
+     *            a temporary file to store the logs required for recovery
      * @throws IOException
      * @throws ACIDException
      */
-    public ArrayList<ILogRecord> requestReplicaLogs(String remoteReplicaId, Set<String> replicasDataToRecover,
-            long fromLSN) throws IOException, ACIDException;
+    public void requestReplicaLogs(String remoteReplicaId, Set<String> replicasDataToRecover, long fromLSN,
+            File recoveryLogsFile) throws IOException, ACIDException;
 
     /**
      * Requests LSM components files from a remote replica.
@@ -68,9 +69,12 @@ public interface IReplicationManager extends IIOReplicationManager {
      *            The replica id to send the request to.
      * @param replicasDataToRecover
      *            Get files that belong to those replicas.
+     * @param existingFiles
+     *            a list of already existing files on the requester
      * @throws IOException
      */
-    public void requestReplicaFiles(String remoteReplicaId, Set<String> replicasDataToRecover) throws IOException;
+    public void requestReplicaFiles(String remoteReplicaId, Set<String> replicasDataToRecover,
+            Set<String> existingFiles) throws IOException;
 
     /**
      * Requests current maximum LSN from remote replicas.
@@ -83,13 +87,6 @@ public interface IReplicationManager extends IIOReplicationManager {
     public long getMaxRemoteLSN(Set<String> remoteReplicaIds) throws IOException;
 
     /**
-     * Sends the IP address of the local replica to all remote replicas.
-     *
-     * @throws IOException
-     */
-    public void broadcastNewIPAddress() throws IOException;
-
-    /**
      * @return The number of remote replicas that are in ACTIVE state.
      */
     public int getActiveReplicasCount();
@@ -146,5 +143,4 @@ public interface IReplicationManager extends IIOReplicationManager {
      * @throws IOException
      */
     public void requestFlushLaggingReplicaIndexes(long nonSharpCheckpointTargetLSN) throws IOException;
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/replication/NodeFailbackPlan.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/NodeFailbackPlan.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/NodeFailbackPlan.java
new file mode 100644
index 0000000..0591644
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/NodeFailbackPlan.java
@@ -0,0 +1,209 @@
+/*
+ * 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.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.messaging.CompleteFailbackRequestMessage;
+import org.apache.asterix.common.messaging.PreparePartitionsFailbackRequestMessage;
+
+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.size() > 0) {
+                    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<>();
+            for (Map.Entry<Integer, String> entry : partition2nodeMap.entrySet()) {
+                if (entry.getValue().equals(participant)) {
+                    partitionToPrepareForFailback.add(entry.getKey());
+                }
+            }
+            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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java
index 0797a02..ae02ca9 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java
@@ -23,23 +23,14 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 
-public class ReplicaEvent {
+import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
 
-    /*
-     * FAIL: remote replica failed.
-     * JOIN: remote replica rejoined the cluster.
-     * SHUTDOWN: remote replica is shutting down normally
-     * */
-    public enum ReplicaEventType {
-        FAIL,
-        JOIN,
-        SHUTDOWN
-    }
+public class ReplicaEvent {
 
     Replica replica;
-    ReplicaEventType eventType;
+    ClusterEventType eventType;
 
-    public ReplicaEvent(Replica replica, ReplicaEventType eventType) {
+    public ReplicaEvent(Replica replica, ClusterEventType eventType) {
         this.replica = replica;
         this.eventType = eventType;
     }
@@ -52,11 +43,11 @@ public class ReplicaEvent {
         this.replica = replica;
     }
 
-    public ReplicaEventType getEventType() {
+    public ClusterEventType getEventType() {
         return eventType;
     }
 
-    public void setEventType(ReplicaEventType eventType) {
+    public void setEventType(ClusterEventType eventType) {
         this.eventType = eventType;
     }
 
@@ -68,7 +59,7 @@ public class ReplicaEvent {
 
     public static ReplicaEvent create(DataInput input) throws IOException {
         Replica replica = Replica.create(input);
-        ReplicaEventType eventType = ReplicaEventType.values()[input.readInt()];
+        ClusterEventType eventType = ClusterEventType.values()[input.readInt()];
         ReplicaEvent event = new ReplicaEvent(replica, eventType);
         return event;
     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java b/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
index a88c985..7e27c54 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
@@ -31,10 +31,10 @@ public interface ILogRecord {
         OK
     }
 
-    public static final int JOB_TERMINATE_LOG_SIZE = 18; //JOB_COMMIT or ABORT log type
+    public static final int JOB_TERMINATE_LOG_SIZE = 14; //JOB_COMMIT or ABORT log type
     public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 30;
     public static final int UPDATE_LOG_BASE_SIZE = 59;
-    public static final int FLUSH_LOG_SIZE = 22;
+    public static final int FLUSH_LOG_SIZE = 18;
 
     public LogRecord.RECORD_STATUS readLogRecord(ByteBuffer buffer);
 
@@ -112,11 +112,9 @@ public interface ILogRecord {
 
     public String getNodeId();
 
-    public void setNodeId(String nodeId);
-
     public int writeRemoteRecoveryLog(ByteBuffer buffer);
 
-    public void readRemoteLog(ByteBuffer buffer, boolean remoteRecoveryLog, String localNodeId);
+    public RECORD_STATUS readRemoteLog(ByteBuffer buffer, boolean remoteRecoveryLog);
 
     public void setReplicationThread(IReplicationThread replicationThread);
 
@@ -130,4 +128,16 @@ public interface ILogRecord {
 
     public ByteBuffer getSerializedLog();
 
+    public void setNodeId(String nodeId);
+
+    public int getResourcePartition();
+
+    public void setResourcePartition(int resourcePartition);
+
+    public void setReplicated(boolean replicated);
+
+    /**
+     * @return a flag indicating whether the log record should be sent to remote replicas
+     */
+    public boolean isReplicated();
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java b/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
index cb3f8c3..a3115e7 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
@@ -18,8 +18,9 @@
  */
 package org.apache.asterix.common.transactions;
 
+import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
+import java.util.Set;
 
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -82,31 +83,25 @@ public interface IRecoveryManager {
 
     /**
      * Makes a system checkpoint.
-     * @param isSharpCheckpoint a flag indicating whether to perform a sharp or non-sharp checkpoint.
-     * @param nonSharpCheckpointTargetLSN if a non-sharp checkpoint to be performed, what is the minimum LSN it should target.
+     *
+     * @param isSharpCheckpoint
+     *            a flag indicating whether to perform a sharp or non-sharp checkpoint.
+     * @param nonSharpCheckpointTargetLSN
+     *            if a non-sharp checkpoint to be performed, what is the minimum LSN it should target.
      * @return the LSN at which the checkpoint was performed.
      * @throws ACIDException
      * @throws HyracksDataException
      */
-    public long checkpoint(boolean isSharpCheckpoint, long nonSharpCheckpointTargetLSN) throws ACIDException, HyracksDataException;
+    public long checkpoint(boolean isSharpCheckpoint, long nonSharpCheckpointTargetLSN)
+            throws ACIDException, HyracksDataException;
 
     /**
-     * Performs recovery based on the passed logs
-     * @param remoteLogs the remote logs to be replayed
-     * @throws HyracksDataException
-     * @throws ACIDException
-     */
-    public void replayRemoteLogs(ArrayList<ILogRecord> remoteLogs) throws HyracksDataException, ACIDException;
-
-    /**
-     *
      * @return min first LSN of the open indexes (including remote indexes if replication is enabled)
      * @throws HyracksDataException
      */
     public long getMinFirstLSN() throws HyracksDataException;
 
     /**
-     *
      * @return min first LSN of the open indexes
      * @throws HyracksDataException
      */
@@ -114,11 +109,29 @@ public interface IRecoveryManager {
 
     /**
      * Replay the logs that belong to the passed {@code partitions} starting from the {@code lowWaterMarkLSN}
+     *
      * @param partitions
      * @param lowWaterMarkLSN
      * @param failedNode
      * @throws IOException
      * @throws ACIDException
      */
-    public void replayPartitionsLogs(Integer[] partitions, long lowWaterMarkLSN, String failedNode) throws IOException, ACIDException;
+    public void replayPartitionsLogs(Set<Integer> partitions, ILogReader logReader, long lowWaterMarkLSN)
+            throws IOException, ACIDException;
+
+    /**
+     * Creates a temporary file to be used during recovery
+     *
+     * @param jobId
+     * @param fileName
+     * @return A file to the created temporary file
+     * @throws IOException
+     *             if the file for the specified {@code jobId} with the {@code fileName} already exists
+     */
+    public File createJobRecoveryFile(int jobId, String fileName) throws IOException;
+
+    /**
+     * Deletes all temporary recovery files
+     */
+    public void deleteRecoveryTemporaryFiles();
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java b/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
index d4a96a5..d09f6ca 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
@@ -20,8 +20,6 @@ package org.apache.asterix.common.transactions;
 
 import java.nio.BufferUnderflowException;
 import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.zip.CRC32;
 
@@ -34,15 +32,14 @@ import org.apache.hyracks.storage.am.common.tuples.SimpleTupleWriter;
 /*
  * == LogRecordFormat ==
  * ---------------------------
- * [Header1] (10 bytes + NodeId Length) : for all log types
+ * [Header1] (6 bytes) : for all log types
  * LogSource(1)
  * LogType(1)
  * JobId(4)
- * NodeIdLength(4)
- * NodeId(?)
  * ---------------------------
- * [Header2] (12 bytes + PKValueSize) : for entity_commit, upsert_entity_commit, and update log types
+ * [Header2] (16 bytes + PKValueSize) : for entity_commit, upsert_entity_commit, and update log types
  * DatasetId(4) //stored in dataset_dataset in Metadata Node
+ * ResourcePartition(4)
  * PKHashValue(4)
  * PKValueSize(4)
  * PKValue(PKValueSize)
@@ -52,7 +49,7 @@ import org.apache.hyracks.storage.am.common.tuples.SimpleTupleWriter;
  * ResourceId(8) //stored in .metadata of the corresponding index in NC node
  * LogRecordSize(4)
  * ---------------------------
- * [Body] (Variable size) : only for update log type
+ * [Body] (9 bytes + NewValueSize) : only for update log type
  * FieldCnt(4)
  * NewOp(1)
  * NewValueSize(4)
@@ -62,19 +59,18 @@ import org.apache.hyracks.storage.am.common.tuples.SimpleTupleWriter;
  * Checksum(8)
  * ---------------------------
  * = LogSize =
- * 1) JOB_COMMIT_LOG_SIZE: 13 bytes (5 + 8)
- * 2) ENTITY_COMMIT || UPSERT_ENTITY_COMMIT: 25 + PKSize (5 + 12 + PKSize + 8)
- *    --> ENTITY_COMMIT_LOG_BASE_SIZE = 25
- * 3) UPDATE: 54 + PKValueSize + NewValueSize (5 + 12 + PKValueSize + 20 + 9 + NewValueSize + 8)
- * 4) FLUSH: 5 + 8 + DatasetId(4) (In case of serialize: + (8 bytes for LSN) + (4 bytes for number of flushed indexes)
+ * 1) JOB_COMMIT_LOG_SIZE: 14 bytes (Header1(6) + Tail(8))
+ * 2) ENTITY_COMMIT || UPSERT_ENTITY_COMMIT: (Header1(6) + Header2(16) + Tail(8)) + PKValueSize
+ *    --> ENTITY_COMMIT_LOG_BASE_SIZE = 30
+ * 3) UPDATE: (Header1(6) + Header2(16) + + Header3(20) + Body(9) + Tail(8)) + PKValueSize + NewValueSize
+ *    --> UPDATE_LOG_BASE_SIZE = 59
+ * 4) FLUSH: 18 bytes (Header1(6) + DatasetId(4) + Tail(8))
  */
 
 public class LogRecord implements ILogRecord {
 
     // ------------- fields in a log record (begin) ------------//
     private byte logSource;
-    private String nodeId;
-    private int nodeIdLength;
     private byte logType;
     private int jobId;
     private int datasetId;
@@ -83,6 +79,7 @@ public class LogRecord implements ILogRecord {
     private ITupleReference PKValue;
     private long prevLSN;
     private long resourceId;
+    private int resourcePartition;
     private int logSize;
     private int fieldCnt;
     private byte newOp;
@@ -103,9 +100,13 @@ public class LogRecord implements ILogRecord {
     private PrimaryIndexOperationTracker opTracker;
     private IReplicationThread replicationThread;
     private ByteBuffer serializedLog;
-    private final Map<String, byte[]> nodeIdsMap;
-    // this field is used for serialized flush logs only to indicate how many indexes were flushed using its LSN.
+    /**
+     * The fields (numOfFlushedIndexes and nodeId) are used for serialized 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;
 
     public LogRecord() {
         isFlushed = new AtomicBoolean(false);
@@ -113,16 +114,15 @@ public class LogRecord implements ILogRecord {
         readPKValue = new PrimaryKeyTupleReference();
         readNewValue = (SimpleTupleReference) tupleWriter.createTupleReference();
         checksumGen = new CRC32();
-        this.nodeIdsMap = new HashMap<String, byte[]>();
         logSource = LogSource.LOCAL;
     }
 
     private final static int LOG_SOURCE_LEN = Byte.BYTES;
-    private final static int NODE_ID_STRING_LENGTH = Integer.BYTES;
     private final static int TYPE_LEN = Byte.BYTES;
     public final static int PKHASH_LEN = Integer.BYTES;
     public final static int PKSZ_LEN = Integer.BYTES;
     private final static int PRVLSN_LEN = Long.BYTES;
+    private final static int RS_PARTITION_LEN = Integer.BYTES;
     private final static int RSID_LEN = Long.BYTES;
     private final static int LOGRCD_SZ_LEN = Integer.BYTES;
     private final static int FLDCNT_LEN = Integer.BYTES;
@@ -130,27 +130,19 @@ public class LogRecord implements ILogRecord {
     private final static int NEWVALSZ_LEN = Integer.BYTES;
     private final static int CHKSUM_LEN = Long.BYTES;
 
-    private final static int ALL_RECORD_HEADER_LEN = LOG_SOURCE_LEN + TYPE_LEN + JobId.BYTES + NODE_ID_STRING_LENGTH;
-    private final static int ENTITYCOMMIT_UPDATE_HEADER_LEN = DatasetId.BYTES + PKHASH_LEN + PKSZ_LEN;
+    private final static int ALL_RECORD_HEADER_LEN = LOG_SOURCE_LEN + TYPE_LEN + JobId.BYTES;
+    private final static int ENTITYCOMMIT_UPDATE_HEADER_LEN = RS_PARTITION_LEN + DatasetId.BYTES + PKHASH_LEN
+            + PKSZ_LEN;
     private final static int UPDATE_LSN_HEADER = PRVLSN_LEN + RSID_LEN + LOGRCD_SZ_LEN;
     private final static int UPDATE_BODY_HEADER = FLDCNT_LEN + NEWOP_LEN + NEWVALSZ_LEN;
+    private final static int REMOTE_FLUSH_LOG_EXTRA_FIELDS_LEN = Long.BYTES + Integer.BYTES + Integer.BYTES;
 
     private void writeLogRecordCommonFields(ByteBuffer buffer) {
         buffer.put(logSource);
         buffer.put(logType);
         buffer.putInt(jobId);
-        if (nodeIdsMap.containsKey(nodeId)) {
-            buffer.put(nodeIdsMap.get(nodeId));
-        } else {
-            // byte array for node id length and string
-            byte[] bytes = new byte[(Integer.SIZE / 8) + nodeId.length()];
-            buffer.putInt(nodeId.length());
-            buffer.put(nodeId.getBytes(java.nio.charset.StandardCharsets.UTF_8));
-            buffer.position(buffer.position() - bytes.length);
-            buffer.get(bytes, 0, bytes.length);
-            nodeIdsMap.put(nodeId, bytes);
-        }
         if (logType == LogType.UPDATE || logType == LogType.ENTITY_COMMIT || logType == LogType.UPSERT_ENTITY_COMMIT) {
+            buffer.putInt(resourcePartition);
             buffer.putInt(datasetId);
             buffer.putInt(PKHashValue);
             if (PKValueSize <= 0) {
@@ -168,7 +160,6 @@ public class LogRecord implements ILogRecord {
             buffer.putInt(newValueSize);
             writeTuple(buffer, newValue, newValueSize);
         }
-
         if (logType == LogType.FLUSH) {
             buffer.putInt(datasetId);
         }
@@ -188,10 +179,9 @@ public class LogRecord implements ILogRecord {
         int beginOffset = buffer.position();
         writeLogRecordCommonFields(buffer);
 
-        if (logSource == LogSource.LOCAL) {
-            // copy the serialized log to send it to replicas
-            int serializedLogSize = getSerializedLogSize(logType, logSize);
-
+        if (replicated) {
+            //copy the serialized log to send it to replicas
+            int serializedLogSize = getSerializedLogSize();
             if (serializedLog == null || serializedLog.capacity() < serializedLogSize) {
                 serializedLog = ByteBuffer.allocate(serializedLogSize);
             } else {
@@ -207,6 +197,8 @@ public class LogRecord implements ILogRecord {
             if (logType == LogType.FLUSH) {
                 serializedLog.putLong(appendLSN);
                 serializedLog.putInt(numOfFlushedIndexes);
+                serializedLog.putInt(nodeId.length());
+                serializedLog.put(nodeId.getBytes());
             }
             serializedLog.flip();
             buffer.position(currentPosition);
@@ -244,15 +236,8 @@ public class LogRecord implements ILogRecord {
     public RECORD_STATUS readLogRecord(ByteBuffer buffer) {
         int beginOffset = buffer.position();
 
-        // read header
-        RECORD_STATUS status = readLogHeader(buffer);
-        if (status != RECORD_STATUS.OK) {
-            buffer.position(beginOffset);
-            return status;
-        }
-
-        // read body
-        status = readLogBody(buffer, false);
+        //read common fields
+        RECORD_STATUS status = readLogCommonFields(buffer);
         if (status != RECORD_STATUS.OK) {
             buffer.position(beginOffset);
             return status;
@@ -271,38 +256,25 @@ public class LogRecord implements ILogRecord {
         return RECORD_STATUS.OK;
     }
 
-    private RECORD_STATUS readLogHeader(ByteBuffer buffer) {
-        // first we need the logtype and Job ID, if the buffer isn't that big, then no dice.
+    private RECORD_STATUS readLogCommonFields(ByteBuffer buffer) {
+        //first we need the logtype and Job ID, if the buffer isn't that big, then no dice.
         if (buffer.remaining() < ALL_RECORD_HEADER_LEN) {
             return RECORD_STATUS.TRUNCATED;
         }
         logSource = buffer.get();
         logType = buffer.get();
         jobId = buffer.getInt();
-        nodeIdLength = buffer.getInt();
-        // attempt to read node id
-        if (buffer.remaining() < nodeIdLength) {
-            return RECORD_STATUS.TRUNCATED;
-        }
-        // read node id string
-        nodeId = new String(buffer.array(), buffer.position() + buffer.arrayOffset(), nodeIdLength,
-                java.nio.charset.StandardCharsets.UTF_8);
-        // skip node id string bytes
-        buffer.position(buffer.position() + nodeIdLength);
 
-        return RECORD_STATUS.OK;
-    }
-
-    private RECORD_STATUS readLogBody(ByteBuffer buffer, boolean allocateTupleBuffer) {
         if (logType != LogType.FLUSH) {
             if (logType == LogType.JOB_COMMIT || logType == LogType.ABORT) {
                 datasetId = -1;
                 PKHashValue = -1;
             } else {
-                // attempt to read in the dsid, PK hash and PK length
+                //attempt to read in the resourcePartition, dsid, PK hash and PK length
                 if (buffer.remaining() < ENTITYCOMMIT_UPDATE_HEADER_LEN) {
                     return RECORD_STATUS.TRUNCATED;
                 }
+                resourcePartition = buffer.getInt();
                 datasetId = buffer.getInt();
                 PKHashValue = buffer.getInt();
                 PKValueSize = buffer.getInt();
@@ -330,16 +302,7 @@ public class LogRecord implements ILogRecord {
                 if (buffer.remaining() < newValueSize) {
                     return RECORD_STATUS.TRUNCATED;
                 }
-                if (!allocateTupleBuffer) {
-                    newValue = readTuple(buffer, readNewValue, fieldCnt, newValueSize);
-                } else {
-                    ByteBuffer tupleBuffer = ByteBuffer.allocate(newValueSize);
-                    tupleBuffer.put(buffer.array(), buffer.position(), newValueSize);
-                    tupleBuffer.flip();
-                    newValue = readTuple(tupleBuffer, readNewValue, fieldCnt, newValueSize);
-                    // skip tuple bytes
-                    buffer.position(buffer.position() + newValueSize);
-                }
+                newValue = readTuple(buffer, readNewValue, fieldCnt, newValueSize);
             } else {
                 computeAndSetLogSize();
             }
@@ -356,25 +319,47 @@ public class LogRecord implements ILogRecord {
     }
 
     @Override
-    public void readRemoteLog(ByteBuffer buffer, boolean remoteRecoveryLog, String localNodeId) {
-        readLogHeader(buffer);
-        if (!remoteRecoveryLog || !nodeId.equals(localNodeId)) {
-            readLogBody(buffer, false);
-        } else {
-            // need to allocate buffer for tuple since the logs will be kept in memory to use during remote recovery
-            // TODO when this is redesigned to spill remote recovery logs to disk, this will not be needed
-            readLogBody(buffer, true);
+    public RECORD_STATUS readRemoteLog(ByteBuffer buffer, boolean remoteRecoveryLog) {
+        int beginOffset = buffer.position();
+
+        //read common fields
+        RECORD_STATUS status = readLogCommonFields(buffer);
+        if (status != RECORD_STATUS.OK) {
+            buffer.position(beginOffset);
+            return status;
         }
 
         if (logType == LogType.FLUSH) {
-            LSN = buffer.getLong();
-            numOfFlushedIndexes = buffer.getInt();
+            if (buffer.remaining() >= REMOTE_FLUSH_LOG_EXTRA_FIELDS_LEN) {
+                LSN = buffer.getLong();
+                numOfFlushedIndexes = buffer.getInt();
+                //read serialized node id
+                int nodeIdLength = buffer.getInt();
+                if (buffer.remaining() >= nodeIdLength) {
+                    byte[] nodeIdBytes = new byte[nodeIdLength];
+                    buffer.get(nodeIdBytes);
+                    nodeId = new String(nodeIdBytes);
+                } else {
+                    buffer.position(beginOffset);
+                    return RECORD_STATUS.TRUNCATED;
+                }
+            } else {
+                buffer.position(beginOffset);
+                return RECORD_STATUS.TRUNCATED;
+            }
         }
 
-        // remote recovery logs need to have the LSN to check which should be replayed
-        if (remoteRecoveryLog && nodeId.equals(localNodeId)) {
-            LSN = buffer.getLong();
+        //remote recovery logs need to have the LSN to check which should be replayed
+        if (remoteRecoveryLog) {
+            if (buffer.remaining() >= Long.BYTES) {
+                LSN = buffer.getLong();
+            } else {
+                buffer.position(beginOffset);
+                return RECORD_STATUS.TRUNCATED;
+            }
         }
+
+        return RECORD_STATUS.OK;
     }
 
     private ITupleReference readPKValue(ByteBuffer buffer) {
@@ -430,21 +415,19 @@ public class LogRecord implements ILogRecord {
             default:
                 throw new IllegalStateException("Unsupported Log Type");
         }
-
-        logSize += nodeIdLength;
     }
 
     @Override
     public String getLogRecordForDisplay() {
         StringBuilder builder = new StringBuilder();
         builder.append(" Source : ").append(LogSource.toString(logSource));
-        builder.append(" NodeID : ").append(nodeId);
         builder.append(" LSN : ").append(LSN);
         builder.append(" LogType : ").append(LogType.toString(logType));
         builder.append(" LogSize : ").append(logSize);
         builder.append(" JobId : ").append(jobId);
         if (logType == LogType.ENTITY_COMMIT || logType == LogType.UPDATE) {
             builder.append(" DatasetId : ").append(datasetId);
+            builder.append(" ResourcePartition : ").append(resourcePartition);
             builder.append(" PKHashValue : ").append(PKHashValue);
             builder.append(" PKFieldCnt : ").append(PKFieldCnt);
             builder.append(" PKSize: ").append(PKValueSize);
@@ -460,11 +443,8 @@ public class LogRecord implements ILogRecord {
     public int writeRemoteRecoveryLog(ByteBuffer buffer) {
         int bufferBegin = buffer.position();
         writeLogRecordCommonFields(buffer);
-        if (logType == LogType.FLUSH) {
-            buffer.putLong(LSN);
-            buffer.putInt(numOfFlushedIndexes);
-        }
-        // LSN must be included in all remote recovery logs (not only FLUSH)
+        //FLUSH logs should not included in remote recovery
+        //LSN must be included in all remote recovery logs
         buffer.putLong(LSN);
         return buffer.position() - bufferBegin;
     }
@@ -560,21 +540,21 @@ public class LogRecord implements ILogRecord {
 
     @Override
     public int getSerializedLogSize() {
-        return getSerializedLogSize(logType, logSize);
-    }
-
-    private static int getSerializedLogSize(Byte logType, int logSize) {
+        int serilizedSize = logSize;
         if (logType == LogType.FLUSH) {
-            // LSN
-            logSize += (Long.SIZE / 8);
-            // num of indexes
-            logSize += (Integer.SIZE / 8);
+            //LSN
+            serilizedSize += Long.BYTES;
+            //num of indexes
+            serilizedSize += Integer.BYTES;
+            //serialized node id String
+            serilizedSize += Integer.BYTES + nodeId.length();
         }
-
-        // checksum not included in serialized version
-        logSize -= CHKSUM_LEN;
-
-        return logSize;
+        if (logSource == LogSource.REMOTE_RECOVERY) {
+            //for LSN;
+            serilizedSize += Long.BYTES;
+        }
+        serilizedSize -= CHKSUM_LEN;
+        return serilizedSize;
     }
 
     @Override
@@ -675,7 +655,6 @@ public class LogRecord implements ILogRecord {
     @Override
     public void setNodeId(String nodeId) {
         this.nodeId = nodeId;
-        this.nodeIdLength = nodeId.length();
     }
 
     public IReplicationThread getReplicationThread() {
@@ -713,4 +692,23 @@ public class LogRecord implements ILogRecord {
         this.opTracker = opTracker;
     }
 
-}
+    @Override
+    public int getResourcePartition() {
+        return resourcePartition;
+    }
+
+    @Override
+    public void setResourcePartition(int resourcePartition) {
+        this.resourcePartition = resourcePartition;
+    }
+
+    @Override
+    public void setReplicated(boolean replicate) {
+        this.replicated = replicate;
+    }
+
+    @Override
+    public boolean isReplicated() {
+        return replicated;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/utils/ServletUtil.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/utils/ServletUtil.java b/asterix-common/src/main/java/org/apache/asterix/common/utils/ServletUtil.java
new file mode 100644
index 0000000..b75d16c
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/utils/ServletUtil.java
@@ -0,0 +1,54 @@
+/*
+ * 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.utils;
+
+public class ServletUtil {
+
+    public enum Servlets {
+        AQL("/aql"),
+        AQL_QUERY("/query"),
+        AQL_UPDATE("/update"),
+        AQL_DDL("/ddl"),
+        SQLPP("/sqlpp"),
+        SQLPP_QUERY("/query/sqlpp"),
+        SQLPP_UPDATE("/update/sqlpp"),
+        SQLPP_DDL("/ddl/sqlpp"),
+        QUERY_STATUS("/query/status"),
+        QUERY_RESULT("/query/result"),
+        QUERY_SERVICE("/query/service"),
+        CONNECTOR("/connector"),
+        SHUTDOWN("/admin/shutdown"),
+        VERSION("/admin/version"),
+        CLUSTER_STATE("/admin/cluster");
+
+        private final String path;
+
+        private Servlets(String path) {
+            this.path = path;
+        }
+
+        public String getPath() {
+            return path;
+        }
+    }
+
+    private ServletUtil() {
+        throw new AssertionError("No objects of this class should be created.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java b/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
index 48e42bd..5b4035c 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
@@ -66,4 +66,8 @@ public class StoragePathUtil {
     private static String prepareFullIndexName(String datasetName, String idxName) {
         return (datasetName + DATASET_INDEX_NAME_SEPARATOR + idxName);
     }
+
+    public static int getPartitonNumFromName(String name) {
+        return Integer.parseInt(name.substring(PARTITION_DIR_PREFIX.length()));
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java b/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java
index 97674e6..1d5b15e 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java
@@ -28,61 +28,39 @@ public class TransactionUtil {
 
     public static void formJobTerminateLogRecord(ITransactionContext txnCtx, LogRecord logRecord, boolean isCommit) {
         logRecord.setTxnCtx(txnCtx);
-        TransactionUtil.formJobTerminateLogRecord(logRecord, txnCtx.getJobId().getId(), isCommit,
-                logRecord.getNodeId());
+        TransactionUtil.formJobTerminateLogRecord(logRecord, txnCtx.getJobId().getId(), isCommit);
     }
 
-    public static void formJobTerminateLogRecord(LogRecord logRecord, int jobId, boolean isCommit, String nodeId) {
+    public static void formJobTerminateLogRecord(LogRecord logRecord, int jobId, boolean isCommit) {
         logRecord.setLogType(isCommit ? LogType.JOB_COMMIT : LogType.ABORT);
         logRecord.setDatasetId(-1);
         logRecord.setPKHashValue(-1);
         logRecord.setJobId(jobId);
-        logRecord.setNodeId(nodeId);
         logRecord.computeAndSetLogSize();
     }
 
     public static void formFlushLogRecord(LogRecord logRecord, int datasetId, PrimaryIndexOperationTracker opTracker,
-            int numOfFlushedIndexes) {
-        formFlushLogRecord(logRecord, datasetId, opTracker, null, numOfFlushedIndexes);
-    }
-
-    public static void formFlushLogRecord(LogRecord logRecord, int datasetId, PrimaryIndexOperationTracker opTracker,
             String nodeId, int numberOfIndexes) {
         logRecord.setLogType(LogType.FLUSH);
         logRecord.setJobId(-1);
         logRecord.setDatasetId(datasetId);
         logRecord.setOpTracker(opTracker);
         logRecord.setNumOfFlushedIndexes(numberOfIndexes);
-        if (nodeId != null) {
-            logRecord.setNodeId(nodeId);
-        }
+        logRecord.setNodeId(nodeId);
         logRecord.computeAndSetLogSize();
     }
 
     public static void formEntityCommitLogRecord(LogRecord logRecord, ITransactionContext txnCtx, int datasetId,
-            int PKHashValue, ITupleReference PKValue, int[] PKFields) {
-        logRecord.setTxnCtx(txnCtx);
-        logRecord.setLogType(LogType.ENTITY_COMMIT);
-        logRecord.setJobId(txnCtx.getJobId().getId());
-        logRecord.setDatasetId(datasetId);
-        logRecord.setPKHashValue(PKHashValue);
-        logRecord.setPKFieldCnt(PKFields.length);
-        logRecord.setPKValue(PKValue);
-        logRecord.setPKFields(PKFields);
-        logRecord.computeAndSetPKValueSize();
-        logRecord.computeAndSetLogSize();
-    }
-
-    public static void formEntityUpsertCommitLogRecord(LogRecord logRecord, ITransactionContext txnCtx, int datasetId,
-            int PKHashValue, ITupleReference PKValue, int[] PKFields) {
+            int PKHashValue, ITupleReference PKValue, int[] PKFields, int resourcePartition, byte entityCommitType) {
         logRecord.setTxnCtx(txnCtx);
-        logRecord.setLogType(LogType.UPSERT_ENTITY_COMMIT);
+        logRecord.setLogType(entityCommitType);
         logRecord.setJobId(txnCtx.getJobId().getId());
         logRecord.setDatasetId(datasetId);
         logRecord.setPKHashValue(PKHashValue);
         logRecord.setPKFieldCnt(PKFields.length);
         logRecord.setPKValue(PKValue);
         logRecord.setPKFields(PKFields);
+        logRecord.setResourcePartition(resourcePartition);
         logRecord.computeAndSetPKValueSize();
         logRecord.computeAndSetLogSize();
     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java b/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
index f54db63..29e08b5 100644
--- a/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
+++ b/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
@@ -24,7 +24,6 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.FileReader;
-import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.PrintWriter;
@@ -39,6 +38,7 @@ import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.utils.ServletUtil.Servlets;
 import org.apache.asterix.testframework.context.TestCaseContext;
 import org.apache.asterix.testframework.context.TestCaseContext.OutputFormat;
 import org.apache.asterix.testframework.context.TestFileContext;
@@ -56,7 +56,6 @@ import org.apache.commons.httpclient.methods.PostMethod;
 import org.apache.commons.httpclient.methods.StringRequestEntity;
 import org.apache.commons.httpclient.params.HttpMethodParams;
 import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.json.JSONObject;
 
 public class TestExecutor {
@@ -262,6 +261,17 @@ public class TestExecutor {
         return method.getResponseBodyAsStream();
     }
 
+    public InputStream executeClusterStateQuery(OutputFormat fmt, String url) throws Exception {
+        HttpMethodBase method = new GetMethod(url);
+
+        //Set accepted output response type
+        method.setRequestHeader("Accept", fmt.mimeType());
+        // Provide custom retry handler is necessary
+        method.getParams().setParameter(HttpMethodParams.RETRY_HANDLER, new DefaultHttpMethodRetryHandler(3, false));
+        executeHttpMethod(method);
+        return method.getResponseBodyAsStream();
+    }
+
     // To execute Update statements
     // Insert and Delete statements are executed here
     public void executeUpdate(String str, String url) throws Exception {
@@ -301,7 +311,7 @@ public class TestExecutor {
     }
 
     private InputStream getHandleResult(String handle, OutputFormat fmt) throws Exception {
-        final String url = "http://" + host + ":" + port + "/query/result";
+        final String url = "http://" + host + ":" + port + Servlets.QUERY_RESULT.getPath();
 
         // Create a method instance.
         GetMethod method = new GetMethod(url);
@@ -372,6 +382,14 @@ public class TestExecutor {
         return IOUtils.toString(input, StandardCharsets.UTF_8.name());
     }
 
+    private static String executeVagrantManagix(ProcessBuilder pb, String command) throws Exception {
+        pb.command("vagrant", "ssh", "cc", "--", pb.environment().get("MANAGIX_HOME") + command);
+        Process p = pb.start();
+        p.waitFor();
+        InputStream input = p.getInputStream();
+        return IOUtils.toString(input, StandardCharsets.UTF_8.name());
+    }
+
     private static String getScriptPath(String queryPath, String scriptBasePath, String scriptFileName) {
         String targetWord = "queries" + File.separator;
         int targetWordSize = targetWord.lastIndexOf(File.separator);
@@ -439,9 +457,9 @@ public class TestExecutor {
                     switch (ctx.getType()) {
                         case "ddl":
                             if (ctx.getFile().getName().endsWith("aql")) {
-                                executeDDL(statement, "http://" + host + ":" + port + "/ddl");
+                                executeDDL(statement, "http://" + host + ":" + port + Servlets.AQL_DDL.getPath());
                             } else {
-                                executeDDL(statement, "http://" + host + ":" + port + "/ddl/sqlpp");
+                                executeDDL(statement, "http://" + host + ":" + port + Servlets.SQLPP_DDL.getPath());
                             }
                             break;
                         case "update":
@@ -451,9 +469,10 @@ public class TestExecutor {
                                         "127.0.0.1://../../../../../../asterix-app/");
                             }
                             if (ctx.getFile().getName().endsWith("aql")) {
-                                executeUpdate(statement, "http://" + host + ":" + port + "/update");
+                                executeUpdate(statement, "http://" + host + ":" + port + Servlets.AQL_UPDATE.getPath());
                             } else {
-                                executeUpdate(statement, "http://" + host + ":" + port + "/update/sqlpp");
+                                executeUpdate(statement,
+                                        "http://" + host + ":" + port + Servlets.SQLPP_UPDATE.getPath());
                             }
                             break;
                         case "query":
@@ -471,24 +490,26 @@ public class TestExecutor {
                             if (ctx.getFile().getName().endsWith("aql")) {
                                 if (ctx.getType().equalsIgnoreCase("query")) {
                                     resultStream = executeQuery(statement, fmt,
-                                            "http://" + host + ":" + port + "/query", cUnit.getParameter());
+                                            "http://" + host + ":" + port + Servlets.AQL_QUERY.getPath(),
+                                            cUnit.getParameter());
                                 } else if (ctx.getType().equalsIgnoreCase("async")) {
                                     resultStream = executeAnyAQLAsync(statement, false, fmt,
-                                            "http://" + host + ":" + port + "/aql");
+                                            "http://" + host + ":" + port + Servlets.AQL.getPath());
                                 } else if (ctx.getType().equalsIgnoreCase("asyncdefer")) {
                                     resultStream = executeAnyAQLAsync(statement, true, fmt,
-                                            "http://" + host + ":" + port + "/aql");
+                                            "http://" + host + ":" + port + Servlets.AQL.getPath());
                                 }
                             } else {
                                 if (ctx.getType().equalsIgnoreCase("query")) {
                                     resultStream = executeQuery(statement, fmt,
-                                            "http://" + host + ":" + port + "/query/sqlpp", cUnit.getParameter());
+                                            "http://" + host + ":" + port + Servlets.SQLPP_QUERY.getPath(),
+                                            cUnit.getParameter());
                                 } else if (ctx.getType().equalsIgnoreCase("async")) {
                                     resultStream = executeAnyAQLAsync(statement, false, fmt,
-                                            "http://" + host + ":" + port + "/sqlpp");
+                                            "http://" + host + ":" + port + Servlets.SQLPP.getPath());
                                 } else if (ctx.getType().equalsIgnoreCase("asyncdefer")) {
                                     resultStream = executeAnyAQLAsync(statement, true, fmt,
-                                            "http://" + host + ":" + port + "/sqlpp");
+                                            "http://" + host + ":" + port + Servlets.SQLPP.getPath());
                                 }
                             }
 
@@ -505,9 +526,6 @@ public class TestExecutor {
 
                             runScriptAndCompareWithResult(testFile, new PrintWriter(System.err), expectedResultFile,
                                     actualResultFile);
-                            LOGGER.info("[TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/" + cUnit.getName()
-                                    + " PASSED ");
-
                             queryCount++;
                             break;
                         case "mgx":
@@ -515,7 +533,7 @@ public class TestExecutor {
                             break;
                         case "txnqbc": //qbc represents query before crash
                             resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
-                                    "http://" + host + ":" + port + "/query", cUnit.getParameter());
+                                    "http://" + host + ":" + port + Servlets.AQL_QUERY.getPath(), cUnit.getParameter());
                             qbcFile = new File(actualPath + File.separator
                                     + testCaseCtx.getTestCase().getFilePath().replace(File.separator, "_") + "_"
                                     + cUnit.getName() + "_qbc.adm");
@@ -524,20 +542,17 @@ public class TestExecutor {
                             break;
                         case "txnqar": //qar represents query after recovery
                             resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
-                                    "http://" + host + ":" + port + "/query", cUnit.getParameter());
+                                    "http://" + host + ":" + port + Servlets.AQL_QUERY.getPath(), cUnit.getParameter());
                             qarFile = new File(actualPath + File.separator
                                     + testCaseCtx.getTestCase().getFilePath().replace(File.separator, "_") + "_"
                                     + cUnit.getName() + "_qar.adm");
                             qarFile.getParentFile().mkdirs();
                             writeOutputToFile(qarFile, resultStream);
                             runScriptAndCompareWithResult(testFile, new PrintWriter(System.err), qbcFile, qarFile);
-
-                            LOGGER.info("[TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/" + cUnit.getName()
-                                    + " PASSED ");
                             break;
                         case "txneu": //eu represents erroneous update
                             try {
-                                executeUpdate(statement, "http://" + host + ":" + port + "/update");
+                                executeUpdate(statement, "http://" + host + ":" + port + Servlets.AQL_UPDATE.getPath());
                             } catch (Exception e) {
                                 //An exception is expected.
                                 failed = true;
@@ -565,7 +580,7 @@ public class TestExecutor {
                             break;
                         case "errddl": // a ddlquery that expects error
                             try {
-                                executeDDL(statement, "http://" + host + ":" + port + "/ddl");
+                                executeDDL(statement, "http://" + host + ":" + port + Servlets.AQL_DDL.getPath());
                             } catch (Exception e) {
                                 // expected error happens
                                 failed = true;
@@ -576,7 +591,7 @@ public class TestExecutor {
                             }
                             System.err.println("...but that was expected.");
                             break;
-                        case "vagrant_script":
+                        case "vscript": //a script that will be executed on a vagrant virtual node
                             try {
                                 String[] command = statement.trim().split(" ");
                                 if (command.length != 2) {
@@ -592,6 +607,32 @@ public class TestExecutor {
                                 throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
                             }
                             break;
+                        case "vmgx": //a managix command that will be executed on vagrant cc node
+                            try {
+                                String output = executeVagrantManagix(pb, statement);
+                                if (output.contains("ERROR")) {
+                                    throw new Exception(output);
+                                }
+                            } catch (Exception e) {
+                                throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
+                            }
+                            break;
+                        case "cstate": //cluster state query
+                            try {
+                                fmt = OutputFormat.forCompilationUnit(cUnit);
+                                resultStream = executeClusterStateQuery(fmt,
+                                        "http://" + host + ":" + port + Servlets.CLUSTER_STATE.getPath());
+                                expectedResultFile = expectedResultFileCtxs.get(queryCount).getFile();
+                                actualResultFile = testCaseCtx.getActualResultFile(cUnit, new File(actualPath));
+                                actualResultFile.getParentFile().mkdirs();
+                                writeOutputToFile(actualResultFile, resultStream);
+                                runScriptAndCompareWithResult(testFile, new PrintWriter(System.err), expectedResultFile,
+                                        actualResultFile);
+                                queryCount++;
+                            } catch (Exception e) {
+                                throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
+                            }
+                            break;
                         default:
                             throw new IllegalArgumentException("No statements of type " + ctx.getType());
                     }
@@ -626,6 +667,9 @@ public class TestExecutor {
                                 "Test \"" + cUnit.getName() + "\" FAILED!\nExpected error was not thrown...");
                         e.printStackTrace();
                         throw e;
+                    } else if (numOfFiles == testFileCtxs.size()) {
+                        LOGGER.info("[TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/" + cUnit.getName()
+                                + " PASSED ");
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java b/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java
index 86c15ae..62ca9bf 100644
--- a/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java
+++ b/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java
@@ -58,7 +58,9 @@ public class ReplicationIT {
     private static ProcessBuilder pb;
     private static Map<String, String> env;
     private final static TestExecutor testExecutor = new TestExecutor(CLUSTER_CC_ADDRESS, CLUSTER_CC_API_PORT);
-    private static String SCRIPT_HOME;
+    private static String SCRIPT_HOME = "/vagrant/scripts/";
+    private static String MANAGIX_HOME = "/tmp/asterix/bin/managix ";
+    private static final String INSTANCE_NAME = "asterix";
     protected TestCaseContext tcCtx;
 
     public ReplicationIT(TestCaseContext tcCtx) {
@@ -107,10 +109,10 @@ public class ReplicationIT {
 
         remoteInvoke("cp -r /vagrant/" + managixFolderName + " /tmp/asterix");
 
-        SCRIPT_HOME = "/vagrant/scripts/";
         pb = new ProcessBuilder();
         env = pb.environment();
         env.put("SCRIPT_HOME", SCRIPT_HOME);
+        env.put("MANAGIX_HOME", MANAGIX_HOME);
         File cwd = new File(asterixProjectDir.toString() + "/" + CLUSTER_BASE);
         pb.directory(cwd);
         pb.redirectErrorStream(true);
@@ -141,13 +143,13 @@ public class ReplicationIT {
     @Before
     public void beforeTest() throws Exception {
         //create instance
-        managixInvoke("create -n vagrant-ssh -c /vagrant/cluster_with_replication.xml").getInputStream();
+        managixInvoke("create -n " + INSTANCE_NAME + " -c /vagrant/cluster_with_replication.xml").getInputStream();
     }
 
     @After
     public void afterTest() throws Exception {
         //stop instance
-        managixInvoke("stop -n vagrant-ssh");
+        managixInvoke("stop -n " + INSTANCE_NAME);
 
         //verify that all processes have been stopped
         String killProcesses = "kill_cc_and_nc.sh";
@@ -162,7 +164,7 @@ public class ReplicationIT {
         executeVagrantScript("nc2", deleteStorage);
 
         //delete instance
-        managixInvoke("delete -n vagrant-ssh");
+        managixInvoke("delete -n " + INSTANCE_NAME);
     }
 
     @Test
@@ -244,7 +246,7 @@ public class ReplicationIT {
     }
 
     private static Process managixInvoke(String cmd) throws Exception {
-        return remoteInvoke("/tmp/asterix/bin/managix " + cmd);
+        return remoteInvoke(MANAGIX_HOME + cmd);
     }
 
     private static String executeVagrantScript(String node, String scriptName) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql
new file mode 100644
index 0000000..2c49a01
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql
@@ -0,0 +1,59 @@
+/*
+ * 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

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.cstate.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.cstate.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.cstate.aql
new file mode 100644
index 0000000..bd01d99
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.cstate.aql
@@ -0,0 +1,29 @@
+/*
+ * 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
+ */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql
new file mode 100644
index 0000000..b09c3d3
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql
@@ -0,0 +1,33 @@
+/*
+ * 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

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
new file mode 100644
index 0000000..47f5975
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
@@ -0,0 +1,35 @@
+/*
+ * 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:///vagrant/data/fbu.adm"),("format"="adm"));
+
+insert into dataset TinySocial.FacebookUsersInMemory(for $x in dataset TinySocial.FacebookUsers return $x);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.vscript.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.vscript.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.vscript.aql
new file mode 100644
index 0000000..5eec164
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.vscript.aql
@@ -0,0 +1 @@
+nc1 kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.4.sleep.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.4.sleep.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.4.sleep.aql
new file mode 100644
index 0000000..51b0e76
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.4.sleep.aql
@@ -0,0 +1 @@
+60000
\ No newline at end of file


[2/6] incubator-asterixdb git commit: Asterix NCs Failback Support

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
index 06a1957..ef2b498 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -34,8 +34,8 @@ import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFa
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 
-public class SecondaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory implements
-        IModificationOperationCallbackFactory {
+public class SecondaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory
+        implements IModificationOperationCallbackFactory {
 
     private static final long serialVersionUID = 1L;
     private final IndexOperation indexOp;
@@ -48,7 +48,7 @@ public class SecondaryIndexModificationOperationCallbackFactory extends Abstract
 
     @Override
     public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
-            Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+            int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getDatasetLifecycleManager();
@@ -60,8 +60,8 @@ public class SecondaryIndexModificationOperationCallbackFactory extends Abstract
         try {
             ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             IModificationOperationCallback modCallback = new SecondaryIndexModificationOperationCallback(datasetId,
-                    primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
-                    indexOp);
+                    primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId,
+                    resourcePartition, resourceType, indexOp);
             txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, false);
             return modCallback;
         } catch (ACIDException e) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetIndexModificationOperationCallback.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetIndexModificationOperationCallback.java
index 69aad24..32d3461 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetIndexModificationOperationCallback.java
@@ -39,8 +39,9 @@ public class TempDatasetIndexModificationOperationCallback extends AbstractIndex
 
     public TempDatasetIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
             ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
-            byte resourceType, IndexOperation indexOp) {
-        super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
+            int resourcePartition, byte resourceType, IndexOperation indexOp) {
+        super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourcePartition,
+                resourceType, indexOp);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
index f2a6820..b08798c 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
@@ -49,7 +49,7 @@ public class TempDatasetPrimaryIndexModificationOperationCallbackFactory extends
 
     @Override
     public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
-            Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+            int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getDatasetLifecycleManager();
@@ -61,8 +61,8 @@ public class TempDatasetPrimaryIndexModificationOperationCallbackFactory extends
         try {
             ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             IModificationOperationCallback modCallback = new TempDatasetIndexModificationOperationCallback(datasetId,
-                    primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
-                    indexOp);
+                    primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId,
+                    resourcePartition, resourceType, indexOp);
             txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, true);
             return modCallback;
         } catch (ACIDException e) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
index 8d838a3..403d68d 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
@@ -49,7 +49,7 @@ public class TempDatasetSecondaryIndexModificationOperationCallbackFactory exten
 
     @Override
     public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
-            Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+            int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getDatasetLifecycleManager();
@@ -61,8 +61,8 @@ public class TempDatasetSecondaryIndexModificationOperationCallbackFactory exten
         try {
             ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             IModificationOperationCallback modCallback = new TempDatasetIndexModificationOperationCallback(datasetId,
-                    primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
-                    indexOp);
+                    primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId,
+                    resourcePartition, resourceType, indexOp);
             txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, false);
             return modCallback;
         } catch (ACIDException e) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallback.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallback.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallback.java
index dfc622a..f98083a 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallback.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallback.java
@@ -31,9 +31,10 @@ public class UpsertOperationCallback extends AbstractIndexModificationOperationC
         implements IModificationOperationCallback {
 
     public UpsertOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
-            ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, byte resourceType,
-            IndexOperation indexOp) {
-        super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
+            ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, int resourcePartition,
+            byte resourceType, IndexOperation indexOp) {
+        super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourcePartition,
+                resourceType, indexOp);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java
index 0c83ab5..707f986 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java
@@ -47,7 +47,7 @@ public class UpsertOperationCallbackFactory extends AbstractOperationCallbackFac
 
     @Override
     public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
-            Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+            int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
 
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
@@ -60,7 +60,8 @@ public class UpsertOperationCallbackFactory extends AbstractOperationCallbackFac
         try {
             ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             IModificationOperationCallback modCallback = new UpsertOperationCallback(datasetId, primaryKeyFields,
-                    txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType, indexOp);
+                    txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourcePartition, resourceType,
+                    indexOp);
             txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, true);
             return modCallback;
         } catch (ACIDException e) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 655fd2a..9cb456f 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -28,11 +28,13 @@ import java.io.ObjectOutputStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.SortedMap;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -40,6 +42,7 @@ import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.config.AsterixMetadataProperties;
 import org.apache.asterix.common.replication.AsterixReplicationJob;
 import org.apache.asterix.common.replication.IReplicationManager;
+import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IODeviceHandle;
@@ -67,6 +70,9 @@ public class PersistentLocalResourceRepository implements ILocalResourceReposito
     private boolean isReplicationEnabled = false;
     private Set<String> filesToBeReplicated;
     private final SortedMap<Integer, ClusterPartition> clusterPartitions;
+    private final Set<Integer> nodeOriginalPartitions;
+    private final Set<Integer> nodeActivePartitions;
+    private Set<Integer> nodeInactivePartitions;
 
     public PersistentLocalResourceRepository(List<IODeviceHandle> devices, String nodeId,
             AsterixMetadataProperties metadataProperties) throws HyracksDataException {
@@ -86,6 +92,15 @@ public class PersistentLocalResourceRepository implements ILocalResourceReposito
             }
         }
         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);
+        for (ClusterPartition partition : nodePartitions) {
+            nodeOriginalPartitions.add(partition.getPartitionId());
+            nodeActivePartitions.add(partition.getPartitionId());
+        }
     }
 
     private static String getStorageMetadataDirPath(String mountPoint, String nodeId, int ioDeviceId) {
@@ -301,6 +316,7 @@ public class PersistentLocalResourceRepository implements ILocalResourceReposito
     }
 
     private static final FilenameFilter METADATA_FILES_FILTER = new FilenameFilter() {
+        @Override
         public boolean accept(File dir, String name) {
             if (name.equalsIgnoreCase(METADATA_FILE_NAME)) {
                 return true;
@@ -316,6 +332,7 @@ public class PersistentLocalResourceRepository implements ILocalResourceReposito
 
         if (isReplicationEnabled) {
             filesToBeReplicated = new HashSet<String>();
+            nodeInactivePartitions = ConcurrentHashMap.newKeySet();
         }
     }
 
@@ -404,4 +421,43 @@ public class PersistentLocalResourceRepository implements ILocalResourceReposito
         //currently each partition is replicated on the same IO device number on all NCs.
         return mountPoints[clusterPartitions.get(partition).getIODeviceNum()];
     }
-}
+
+    public Set<Integer> getActivePartitions() {
+        return Collections.unmodifiableSet(nodeActivePartitions);
+    }
+
+    public Set<Integer> getInactivePartitions() {
+        return Collections.unmodifiableSet(nodeInactivePartitions);
+    }
+
+    public Set<Integer> getNodeOrignalPartitions() {
+        return Collections.unmodifiableSet(nodeOriginalPartitions);
+    }
+
+    public synchronized void addActivePartition(int partitonId) {
+        nodeActivePartitions.add(partitonId);
+        nodeInactivePartitions.remove(partitonId);
+    }
+
+    public synchronized void addInactivePartition(int partitonId) {
+        nodeInactivePartitions.add(partitonId);
+        nodeActivePartitions.remove(partitonId);
+    }
+
+    /**
+     * @param resourceAbsolutePath
+     * @return the resource relative path starting from the partition directory
+     */
+    public static String getResourceRelativePath(String resourceAbsolutePath) {
+        String[] tokens = resourceAbsolutePath.split(File.separator);
+        //partiton/dataverse/idx/fileName
+        return tokens[tokens.length - 4] + File.separator + tokens[tokens.length - 3] + File.separator
+                + tokens[tokens.length - 2] + File.separator + tokens[tokens.length - 1];
+    }
+
+    public static int getResourcePartition(String resourceAbsolutePath) {
+        String[] tokens = resourceAbsolutePath.split(File.separator);
+        //partiton/dataverse/idx/fileName
+        return StoragePathUtil.getPartitonNumFromName(tokens[tokens.length - 4]);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
index 1966c39..5649710 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
@@ -41,6 +41,9 @@ public class LogManagerWithReplication extends LogManager {
             throw new IllegalStateException();
         }
 
+        //only locally generated logs should be replicated
+        logRecord.setReplicated(logRecord.getLogSource() == LogSource.LOCAL);
+
         //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) {
             flushLogsQ.offer(logRecord);
@@ -54,7 +57,7 @@ public class LogManagerWithReplication extends LogManager {
     protected void appendToLogTail(ILogRecord logRecord) throws ACIDException {
         syncAppendToLogTail(logRecord);
 
-        if (logRecord.getLogSource() == LogSource.LOCAL) {
+        if (logRecord.isReplicated()) {
             replicationManager.replicateLog(logRecord);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogReader.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogReader.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogReader.java
index e0cddee..a018dc2 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogReader.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogReader.java
@@ -26,11 +26,10 @@ import java.util.logging.Logger;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.transactions.ILogReader;
 import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.ILogRecord.RECORD_STATUS;
 import org.apache.asterix.common.transactions.LogRecord;
 import org.apache.asterix.common.transactions.MutableLong;
 
-import static org.apache.asterix.common.transactions.LogRecord.*;
-
 /**
  * NOTE: Many method calls of this class are not thread safe.
  * Be very cautious using it in a multithreaded context.

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/RemoteLogReader.java
----------------------------------------------------------------------
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/RemoteLogReader.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/RemoteLogReader.java
new file mode 100644
index 0000000..8f88321
--- /dev/null
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/RemoteLogReader.java
@@ -0,0 +1,138 @@
+/*
+ * 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.logging;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.transactions.ILogReader;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.ILogRecord.RECORD_STATUS;
+import org.apache.asterix.common.transactions.LogRecord;
+
+public class RemoteLogReader implements ILogReader {
+
+    private final FileChannel fileChannel;
+    private final ILogRecord logRecord;
+    private final ByteBuffer readBuffer;
+    private long readLSN;
+    private final int logPageSize;
+
+    public RemoteLogReader(FileChannel fileChannel, long logFileSize, int logPageSize) {
+        this.fileChannel = fileChannel;
+        this.logPageSize = logPageSize;
+        logRecord = new LogRecord();
+        readBuffer = ByteBuffer.allocate(logPageSize);
+    }
+
+    @Override
+    public void initializeScan(long beginLSN) throws ACIDException {
+        readLSN = beginLSN;
+        fillLogReadBuffer();
+    }
+
+    private boolean fillLogReadBuffer() throws ACIDException {
+        int size = 0;
+        int read = 0;
+        readBuffer.position(0);
+        readBuffer.limit(logPageSize);
+        try {
+            fileChannel.position(readLSN);
+            //We loop here because read() may return 0, but this simply means we are waiting on IO.
+            //Therefore we want to break out only when either the buffer is full, or we reach EOF.
+            while (size < logPageSize && read != -1) {
+                read = fileChannel.read(readBuffer);
+                if (read > 0) {
+                    size += read;
+                }
+            }
+        } catch (IOException e) {
+            throw new ACIDException(e);
+        }
+        readBuffer.position(0);
+        readBuffer.limit(size);
+        if (size == 0 && read == -1) {
+            return false; //EOF
+        }
+        return true;
+    }
+
+    @Override
+    public ILogRecord read(long LSN) throws ACIDException {
+        throw new UnsupportedOperationException("Random read is not supported.");
+    }
+
+    @Override
+    public ILogRecord next() throws ACIDException {
+        if (readBuffer.position() == readBuffer.limit()) {
+            boolean hasRemaining = fillLogReadBuffer();
+            if (!hasRemaining) {
+                return null;
+            }
+        }
+
+        RECORD_STATUS status = logRecord.readRemoteLog(readBuffer, true);
+        switch (status) {
+            case TRUNCATED: {
+                //we may have just read off the end of the buffer, so try refiling it
+                if (!fillLogReadBuffer()) {
+                    return null;
+                }
+                //now see what we have in the refilled buffer
+                status = logRecord.readRemoteLog(readBuffer, true);
+                switch (status) {
+                    case TRUNCATED: {
+                        return null;
+                    }
+                    case OK:
+                        break;
+                    default:
+                        break;
+                }
+                //if we have exited the inner switch,
+                // this means status is really "OK" after buffer refill
+                break;
+            }
+            case BAD_CHKSUM: {
+                return null;
+            }
+            case OK:
+                break;
+        }
+
+        readLSN += logRecord.getSerializedLogSize();
+        return logRecord;
+    }
+
+    @Override
+    public void close() throws ACIDException {
+        try {
+            if (fileChannel != null) {
+                if (fileChannel.isOpen()) {
+                    fileChannel.close();
+                }
+            }
+        } catch (IOException e) {
+            throw new ACIDException(e);
+        }
+    }
+
+}


[4/6] incubator-asterixdb git commit: Asterix NCs Failback Support

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.cstate.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.cstate.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.cstate.aql
new file mode 100644
index 0000000..bd01d99
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.cstate.aql
@@ -0,0 +1,29 @@
+/*
+ * 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
+ */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.6.query.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.6.query.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.6.query.aql
new file mode 100644
index 0000000..b09c3d3
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.6.query.aql
@@ -0,0 +1,33 @@
+/*
+ * 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

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.7.update.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.7.update.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.7.update.aql
new file mode 100644
index 0000000..56a88a2
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.7.update.aql
@@ -0,0 +1,51 @@
+/*
+ * 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

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.vmgx.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.vmgx.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.vmgx.aql
new file mode 100644
index 0000000..67b492c
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.vmgx.aql
@@ -0,0 +1 @@
+startnode -n asterix -nodes nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql
new file mode 100644
index 0000000..1746da6
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql
@@ -0,0 +1 @@
+10000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
index ae14ad0..94ecc27 100644
--- a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
@@ -29,4 +29,4 @@
 use dataverse TinySocial;
 
 load dataset FacebookUsers using localfs
-(("path"="vagrant-ssh_nc1:///vagrant/data/fbu.adm"),("format"="adm"));
\ No newline at end of file
+(("path"="asterix_nc1:///vagrant/data/fbu.adm"),("format"="adm"));
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vagrant_script.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vagrant_script.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vagrant_script.aql
deleted file mode 100644
index 5695ed7..0000000
--- a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vagrant_script.aql
+++ /dev/null
@@ -1 +0,0 @@
-nc2 kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vscript.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vscript.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vscript.aql
new file mode 100644
index 0000000..5695ed7
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vscript.aql
@@ -0,0 +1 @@
+nc2 kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
index 8087689..d97f786 100644
--- a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
@@ -29,6 +29,6 @@
 use dataverse TinySocial;
 
 load dataset FacebookUsers using localfs
-(("path"="vagrant-ssh_nc1:///vagrant/data/fbu.adm"),("format"="adm"));
+(("path"="asterix_nc1:///vagrant/data/fbu.adm"),("format"="adm"));
 
 insert into dataset TinySocial.FacebookUsersInMemory(for $x in dataset TinySocial.FacebookUsers return $x);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vagrant_script.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vagrant_script.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vagrant_script.aql
deleted file mode 100644
index 5695ed7..0000000
--- a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vagrant_script.aql
+++ /dev/null
@@ -1 +0,0 @@
-nc2 kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vscript.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vscript.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vscript.aql
new file mode 100644
index 0000000..5695ed7
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vscript.aql
@@ -0,0 +1 @@
+nc2 kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vagrant_script.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vagrant_script.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vagrant_script.aql
deleted file mode 100644
index 5eec164..0000000
--- a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vagrant_script.aql
+++ /dev/null
@@ -1 +0,0 @@
-nc1 kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vscript.aql
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vscript.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vscript.aql
new file mode 100644
index 0000000..5eec164
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vscript.aql
@@ -0,0 +1 @@
+nc1 kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.10.adm
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.10.adm b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.10.adm
new file mode 100644
index 0000000..61322c9
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.10.adm
@@ -0,0 +1 @@
+{"State":"ACTIVE","Metadata_Node":"asterix_nc1","partition_0":"asterix_nc1","partition_1":"asterix_nc1","partition_2":"asterix_nc2","partition_3":"asterix_nc2"}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm
new file mode 100644
index 0000000..587a97a
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm
@@ -0,0 +1 @@
+{"State":"ACTIVE","Metadata_Node":"asterix_nc2","partition_0":"asterix_nc2","partition_1":"asterix_nc2","partition_2":"asterix_nc2","partition_3":"asterix_nc2"}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.11.adm
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.11.adm b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.11.adm
new file mode 100644
index 0000000..2edeafb
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.11.adm
@@ -0,0 +1 @@
+20
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.6.adm
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.6.adm b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.6.adm
new file mode 100644
index 0000000..9a03714
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.6.adm
@@ -0,0 +1 @@
+10
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-installer/src/test/resources/integrationts/replication/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterix-installer/src/test/resources/integrationts/replication/testsuite.xml b/asterix-installer/src/test/resources/integrationts/replication/testsuite.xml
index f033086..36c3992 100644
--- a/asterix-installer/src/test/resources/integrationts/replication/testsuite.xml
+++ b/asterix-installer/src/test/resources/integrationts/replication/testsuite.xml
@@ -17,21 +17,28 @@
  ! under the License.
  !-->
 <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="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>
+        </test-case>
+    </test-group>
 </test-suite>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
----------------------------------------------------------------------
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 4e6a3df..088a85b 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -28,6 +28,7 @@ import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.config.IAsterixPropertiesProvider;
 import org.apache.asterix.common.dataflow.AsterixLSMIndexUtil;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.functions.FunctionSignature;
@@ -113,6 +114,7 @@ public class MetadataNode implements IMetadataNode {
 
     private IDatasetLifecycleManager datasetLifecycleManager;
     private ITransactionSubsystem transactionSubsystem;
+    private int metadataStoragePartition;
 
     public static final MetadataNode INSTANCE = new MetadataNode();
 
@@ -123,6 +125,8 @@ public class MetadataNode implements IMetadataNode {
     public void initialize(IAsterixAppRuntimeContext runtimeContext) {
         this.transactionSubsystem = runtimeContext.getTransactionSubsystem();
         this.datasetLifecycleManager = runtimeContext.getDatasetLifecycleManager();
+        this.metadataStoragePartition = ((IAsterixPropertiesProvider) runtimeContext).getMetadataProperties()
+                .getMetadataPartition().getPartitionId();
     }
 
     @Override
@@ -305,11 +309,11 @@ public class MetadataNode implements IMetadataNode {
         if (metadataIndex.isPrimaryIndex()) {
             return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
                     metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
-                    transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
+                    transactionSubsystem, resourceId, metadataStoragePartition, ResourceType.LSM_BTREE, indexOp);
         } else {
             return new SecondaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
                     metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
-                    transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
+                    transactionSubsystem, resourceId, metadataStoragePartition, ResourceType.LSM_BTREE, indexOp);
         }
     }
 
@@ -641,8 +645,7 @@ public class MetadataNode implements IMetadataNode {
         }
     }
 
-    private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName)
-            throws MetadataException, RemoteException {
+    private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName) throws MetadataException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
             DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
@@ -673,7 +676,7 @@ public class MetadataNode implements IMetadataNode {
         }
     }
 
-    public List<Dataset> getAllDatasets(JobId jobId) throws MetadataException, RemoteException {
+    public List<Dataset> getAllDatasets(JobId jobId) throws MetadataException {
         try {
             ITupleReference searchKey = null;
             DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
@@ -686,7 +689,7 @@ public class MetadataNode implements IMetadataNode {
         }
     }
 
-    public List<Datatype> getAllDatatypes(JobId jobId) throws MetadataException, RemoteException {
+    public List<Datatype> getAllDatatypes(JobId jobId) throws MetadataException {
         try {
             ITupleReference searchKey = null;
             DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
@@ -699,8 +702,7 @@ public class MetadataNode implements IMetadataNode {
         }
     }
 
-    private void confirmDataverseCanBeDeleted(JobId jobId, String dataverseName)
-            throws MetadataException, RemoteException {
+    private void confirmDataverseCanBeDeleted(JobId jobId, String dataverseName) throws MetadataException {
         //If a dataset from a DIFFERENT dataverse
         //uses a type from this dataverse
         //throw an error
@@ -717,13 +719,13 @@ public class MetadataNode implements IMetadataNode {
     }
 
     private void confirmDatatypeIsUnused(JobId jobId, String dataverseName, String datatypeName)
-            throws MetadataException, RemoteException {
+            throws MetadataException {
         confirmDatatypeIsUnusedByDatatypes(jobId, dataverseName, datatypeName);
         confirmDatatypeIsUnusedByDatasets(jobId, dataverseName, datatypeName);
     }
 
     private void confirmDatatypeIsUnusedByDatasets(JobId jobId, String dataverseName, String datatypeName)
-            throws MetadataException, RemoteException {
+            throws MetadataException {
         //If any dataset uses this type, throw an error
         List<Dataset> datasets = getAllDatasets(jobId);
         for (Dataset set : datasets) {
@@ -735,7 +737,7 @@ public class MetadataNode implements IMetadataNode {
     }
 
     private void confirmDatatypeIsUnusedByDatatypes(JobId jobId, String dataverseName, String datatypeName)
-            throws MetadataException, RemoteException {
+            throws MetadataException {
         //If any datatype uses this type, throw an error
         //TODO: Currently this loads all types into memory. This will need to be fixed for large numbers of types
         List<Datatype> datatypes = getAllDatatypes(jobId);
@@ -768,7 +770,7 @@ public class MetadataNode implements IMetadataNode {
     }
 
     public List<String> getDatasetNamesPartitionedOnThisNodeGroup(JobId jobId, String nodegroup)
-            throws MetadataException, RemoteException {
+            throws MetadataException {
         //this needs to scan the datasets and return the datasets that use this nodegroup
         List<String> nodeGroupDatasets = new ArrayList<String>();
         List<Dataset> datasets = getAllDatasets(jobId);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java
----------------------------------------------------------------------
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java b/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java
index 2744630..f79385c 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java
@@ -22,6 +22,8 @@ import java.io.InputStream;
 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;
@@ -36,14 +38,24 @@ import javax.xml.bind.Unmarshaller;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.common.messaging.CompleteFailbackRequestMessage;
+import org.apache.asterix.common.messaging.CompleteFailbackResponseMessage;
+import org.apache.asterix.common.messaging.PreparePartitionsFailbackRequestMessage;
+import org.apache.asterix.common.messaging.PreparePartitionsFailbackResponseMessage;
+import org.apache.asterix.common.messaging.ReplicaEventMessage;
 import org.apache.asterix.common.messaging.TakeoverMetadataNodeRequestMessage;
 import org.apache.asterix.common.messaging.TakeoverMetadataNodeResponseMessage;
 import org.apache.asterix.common.messaging.TakeoverPartitionsRequestMessage;
 import org.apache.asterix.common.messaging.TakeoverPartitionsResponseMessage;
 import org.apache.asterix.common.messaging.api.ICCMessageBroker;
+import org.apache.asterix.common.replication.NodeFailbackPlan;
+import org.apache.asterix.common.replication.NodeFailbackPlan.FailbackPlanState;
 import org.apache.asterix.event.schema.cluster.Cluster;
 import org.apache.asterix.event.schema.cluster.Node;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
+import org.json.JSONException;
+import org.json.JSONObject;
 
 /**
  * A holder class for properties related to the Asterix cluster.
@@ -57,15 +69,16 @@ public class AsterixClusterProperties {
      */
 
     private static final Logger LOGGER = Logger.getLogger(AsterixClusterProperties.class.getName());
-
     public static final AsterixClusterProperties INSTANCE = new AsterixClusterProperties();
     public static final String CLUSTER_CONFIGURATION_FILE = "cluster.xml";
 
+    private static final String CLUSTER_NET_IP_ADDRESS_KEY = "cluster-net-ip-address";
     private static final String IO_DEVICES = "iodevices";
     private static final String DEFAULT_STORAGE_DIR_NAME = "storage";
-    private Map<String, Map<String, String>> ncConfiguration = new HashMap<String, Map<String, String>>();
+    private Map<String, Map<String, String>> activeNcConfiguration = new HashMap<String, Map<String, String>>();
 
     private final Cluster cluster;
+    private ClusterState state = ClusterState.UNUSABLE;
 
     private AlgebricksAbsolutePartitionConstraint clusterPartitionConstraint;
 
@@ -75,10 +88,14 @@ public class AsterixClusterProperties {
     private SortedMap<Integer, ClusterPartition> clusterPartitions = null;
     private Map<Long, TakeoverPartitionsRequestMessage> pendingTakeoverRequests = null;
 
-    private long takeoverRequestId = 0;
+    private long clusterRequestId = 0;
     private String currentMetadataNode = null;
-    private boolean isMetadataNodeActive = false;
+    private boolean metadataNodeActive = false;
     private boolean autoFailover = false;
+    private boolean replicationEnabled = false;
+    private Set<String> failedNodes = new HashSet<>();
+    private LinkedList<NodeFailbackPlan> pendingProcessingFailbackPlans;
+    private Map<Long, NodeFailbackPlan> planId2FailbackPlanMap;
 
     private AsterixClusterProperties() {
         InputStream is = this.getClass().getClassLoader().getResourceAsStream(CLUSTER_CONFIGURATION_FILE);
@@ -99,43 +116,73 @@ public class AsterixClusterProperties {
                 node2PartitionsMap = AsterixAppContextInfo.getInstance().getMetadataProperties().getNodePartitions();
                 clusterPartitions = AsterixAppContextInfo.getInstance().getMetadataProperties().getClusterPartitions();
                 currentMetadataNode = AsterixAppContextInfo.getInstance().getMetadataProperties().getMetadataNodeName();
-                if (isAutoFailoverEnabled()) {
-                    autoFailover = cluster.getDataReplication().isAutoFailover();
-                }
+                replicationEnabled = isReplicationEnabled();
+                autoFailover = isAutoFailoverEnabled();
                 if (autoFailover) {
                     pendingTakeoverRequests = new HashMap<>();
+                    pendingProcessingFailbackPlans = new LinkedList<>();
+                    planId2FailbackPlanMap = new HashMap<>();
                 }
             }
         }
     }
 
-    private ClusterState state = ClusterState.UNUSABLE;
-
     public synchronized void removeNCConfiguration(String nodeId) {
-        updateNodePartitions(nodeId, false);
-        ncConfiguration.remove(nodeId);
-        if (nodeId.equals(currentMetadataNode)) {
-            isMetadataNodeActive = false;
-            LOGGER.info("Metadata node is now inactive");
-        }
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Removing configuration parameters for node id " + nodeId);
         }
-        if (autoFailover) {
-            requestPartitionsTakeover(nodeId);
+        activeNcConfiguration.remove(nodeId);
+
+        //if this node was waiting for failback and failed before it completed
+        if (failedNodes.contains(nodeId)) {
+            if (autoFailover) {
+                notifyFailbackPlansNodeFailure(nodeId);
+                revertFailedFailbackPlanEffects();
+            }
+        } else {
+            //an active node failed
+            failedNodes.add(nodeId);
+            if (nodeId.equals(currentMetadataNode)) {
+                metadataNodeActive = false;
+                LOGGER.info("Metadata node is now inactive");
+            }
+            updateNodePartitions(nodeId, false);
+            if (replicationEnabled) {
+                notifyImpactedReplicas(nodeId, ClusterEventType.NODE_FAILURE);
+                if (autoFailover) {
+                    notifyFailbackPlansNodeFailure(nodeId);
+                    requestPartitionsTakeover(nodeId);
+                }
+            }
         }
     }
 
     public synchronized void addNCConfiguration(String nodeId, Map<String, String> configuration) {
-        ncConfiguration.put(nodeId, configuration);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Registering configuration parameters for node id " + nodeId);
+        }
+        activeNcConfiguration.put(nodeId, configuration);
+
+        //a node trying to come back after failure
+        if (failedNodes.contains(nodeId)) {
+            if (autoFailover) {
+                prepareFailbackPlan(nodeId);
+                return;
+            } else {
+                //a node completed local or remote recovery and rejoined
+                failedNodes.remove(nodeId);
+                if (replicationEnabled) {
+                    //notify other replica to reconnect to this node
+                    notifyImpactedReplicas(nodeId, ClusterEventType.NODE_JOIN);
+                }
+            }
+        }
+
         if (nodeId.equals(currentMetadataNode)) {
-            isMetadataNodeActive = true;
+            metadataNodeActive = true;
             LOGGER.info("Metadata node is now active");
         }
         updateNodePartitions(nodeId, true);
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info(" Registering configuration parameters for node id " + nodeId);
-        }
     }
 
     private synchronized void updateNodePartitions(String nodeId, boolean added) {
@@ -163,11 +210,17 @@ public class AsterixClusterProperties {
             }
         }
         //if all storage partitions are active as well as the metadata node, then the cluster is active
-        if (isMetadataNodeActive) {
+        if (metadataNodeActive) {
             state = ClusterState.ACTIVE;
             LOGGER.info("Cluster is now ACTIVE");
             //start global recovery
             AsterixAppContextInfo.getInstance().getGlobalRecoveryManager().startGlobalRecovery();
+            if (autoFailover) {
+                //if there are any pending failback requests, process them
+                if (pendingProcessingFailbackPlans.size() > 0) {
+                    processPendingFailbackPlans();
+                }
+            }
         } else {
             requestMetadataNodeTakeover();
         }
@@ -196,7 +249,7 @@ public class AsterixClusterProperties {
      *         if it does not correspond to the set of registered Node Controllers.
      */
     public synchronized String[] getIODevices(String nodeId) {
-        Map<String, String> ncConfig = ncConfiguration.get(nodeId);
+        Map<String, String> ncConfig = activeNcConfiguration.get(nodeId);
         if (ncConfig == null) {
             if (LOGGER.isLoggable(Level.WARNING)) {
                 LOGGER.warning("Configuration parameters for nodeId " + nodeId
@@ -222,7 +275,7 @@ public class AsterixClusterProperties {
 
     public synchronized Set<String> getParticipantNodes() {
         Set<String> participantNodes = new HashSet<String>();
-        for (String pNode : ncConfiguration.keySet()) {
+        for (String pNode : activeNcConfiguration.keySet()) {
             participantNodes.add(pNode);
         }
         return participantNodes;
@@ -254,12 +307,12 @@ public class AsterixClusterProperties {
         this.globalRecoveryCompleted = globalRecoveryCompleted;
     }
 
-    public static boolean isClusterActive() {
-        if (AsterixClusterProperties.INSTANCE.getCluster() == null) {
+    public boolean isClusterActive() {
+        if (cluster == null) {
             // this is a virtual cluster
             return true;
         }
-        return AsterixClusterProperties.INSTANCE.getState() == ClusterState.ACTIVE;
+        return state == ClusterState.ACTIVE;
     }
 
     public static int getNumberOfNodes() {
@@ -279,8 +332,8 @@ public class AsterixClusterProperties {
 
     public synchronized ClusterPartition[] getClusterPartitons() {
         ArrayList<ClusterPartition> partitons = new ArrayList<>();
-        for (ClusterPartition cluster : clusterPartitions.values()) {
-            partitons.add(cluster);
+        for (ClusterPartition partition : clusterPartitions.values()) {
+            partitons.add(partition);
         }
         return partitons.toArray(new ClusterPartition[] {});
     }
@@ -301,44 +354,53 @@ public class AsterixClusterProperties {
 
         //collect the partitions of the failed NC
         List<ClusterPartition> lostPartitions = getNodeAssignedPartitions(failedNodeId);
-        for (ClusterPartition partition : lostPartitions) {
-            //find replicas for this partitions
-            Set<String> partitionReplicas = replicationProperties.getNodeReplicasIds(partition.getNodeId());
-            //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 (ncConfiguration.containsKey(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());
+        if (lostPartitions.size() > 0) {
+            for (ClusterPartition partition : lostPartitions) {
+                //find replicas for this partitions
+                Set<String> partitionReplicas = replicationProperties.getNodeReplicasIds(partition.getNodeId());
+                //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 (activeNcConfiguration.containsKey(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());
+                        }
                     }
                 }
             }
-        }
 
-        ICCMessageBroker messageBroker = (ICCMessageBroker) AsterixAppContextInfo.getInstance()
-                .getCCApplicationContext().getMessageBroker();
-        //For each replica, send a request to takeover the assigned partitions
-        for (String replica : partitionRecoveryPlan.keySet()) {
-            Integer[] partitionsToTakeover = partitionRecoveryPlan.get(replica).toArray(new Integer[] {});
-            long requestId = takeoverRequestId++;
-            TakeoverPartitionsRequestMessage takeoverRequest = new TakeoverPartitionsRequestMessage(requestId, replica,
-                    failedNodeId, 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.warning("Failed to send takeover request: " + takeoverRequest);
-                e.printStackTrace();
+            if (partitionRecoveryPlan.size() == 0) {
+                //no active replicas were found for the failed node
+                LOGGER.severe("Could not find active replicas for the partitions " + lostPartitions);
+                return;
+            } else {
+                LOGGER.info("Partitions to recover: " + lostPartitions);
+            }
+            ICCMessageBroker messageBroker = (ICCMessageBroker) AsterixAppContextInfo.getInstance()
+                    .getCCApplicationContext().getMessageBroker();
+            //For each replica, send a request to takeover the assigned partitions
+            for (String replica : partitionRecoveryPlan.keySet()) {
+                Integer[] partitionsToTakeover = partitionRecoveryPlan.get(replica).toArray(new Integer[] {});
+                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.warning("Failed to send takeover request: " + takeoverRequest);
+                    e.printStackTrace();
+                }
             }
         }
     }
@@ -368,7 +430,6 @@ public class AsterixClusterProperties {
         for (Long requestId : failedTakeoverRequests) {
             pendingTakeoverRequests.remove(requestId);
         }
-
         return nodePartitions;
     }
 
@@ -406,19 +467,223 @@ public class AsterixClusterProperties {
 
     public synchronized void processMetadataNodeTakeoverResponse(TakeoverMetadataNodeResponseMessage reponse) {
         currentMetadataNode = reponse.getNodeId();
-        isMetadataNodeActive = true;
+        metadataNodeActive = true;
         LOGGER.info("Current metadata node: " + currentMetadataNode);
         updateClusterState();
     }
 
-    public synchronized String getCurrentMetadataNode() {
-        return currentMetadataNode;
+    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
+        AsterixReplicationProperties replicationProperties = AsterixAppContextInfo.getInstance()
+                .getReplicationProperties();
+        Set<String> nodeReplicas = replicationProperties.getNodeReplicationClients(failingBackNodeId);
+        for (String replicaId : nodeReplicas) {
+            ClusterPartition[] nodePartitions = node2PartitionsMap.get(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.isLoggable(Level.INFO)) {
+            LOGGER.info("Prepared Failback plan: " + plan.toString());
+        }
+
+        processPendingFailbackPlans();
     }
 
-    public boolean isAutoFailoverEnabled() {
-        if (cluster != null && cluster.getDataReplication() != null && cluster.getDataReplication().isEnabled()) {
-            return cluster.getDataReplication().isAutoFailover();
+    private synchronized void processPendingFailbackPlans() {
+        /**
+         * 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()) {
+                        ClusterPartition clusterPartition = clusterPartitions.get(partitionId);
+                        clusterPartition.setActive(false);
+                        //partition expected to be returned to the failing back node
+                        clusterPartition.setActiveNodeId(failbackNode);
+                    }
+
+                    /**
+                     * if the returning node is the original metadata node,
+                     * then metadata node will change after the failback completes
+                     */
+                    String originalMetadataNode = AsterixAppContextInfo.getInstance().getMetadataProperties()
+                            .getMetadataNodeName();
+                    if (originalMetadataNode.equals(failbackNode)) {
+                        plan.setNodeToReleaseMetadataManager(currentMetadataNode);
+                        currentMetadataNode = "";
+                        metadataNodeActive = false;
+                    }
+
+                    //force new jobs to wait
+                    state = ClusterState.REBALANCING;
+
+                    ICCMessageBroker messageBroker = (ICCMessageBroker) AsterixAppContextInfo.getInstance()
+                            .getCCApplicationContext().getMessageBroker();
+                    //send requests to other nodes to complete on-going jobs and prepare partitions for failback
+                    Set<PreparePartitionsFailbackRequestMessage> planFailbackRequests = plan.getPlanFailbackRequests();
+                    for (PreparePartitionsFailbackRequestMessage request : planFailbackRequests) {
+                        try {
+                            messageBroker.sendApplicationMessageToNC(request, request.getNodeID());
+                            plan.addPendingRequest(request);
+                        } catch (Exception e) {
+                            LOGGER.warning("Failed to send failback request to: " + request.getNodeID());
+                            e.printStackTrace();
+                            plan.notifyNodeFailure(request.getNodeID());
+                            revertFailedFailbackPlanEffects();
+                            break;
+                        }
+                    }
+                    /**
+                     * 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());
+                }
+            }
+        }
+    }
+
+    public synchronized void processPreparePartitionsFailbackResponse(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
+            ICCMessageBroker messageBroker = (ICCMessageBroker) AsterixAppContextInfo.getInstance()
+                    .getCCApplicationContext().getMessageBroker();
+            try {
+                messageBroker.sendApplicationMessageToNC(request, request.getNodeId());
+            } catch (Exception e) {
+                LOGGER.warning("Failed to send complete failback request to: " + request.getNodeId());
+                e.printStackTrace();
+                notifyFailbackPlansNodeFailure(request.getNodeId());
+                revertFailedFailbackPlanEffects();
+            }
+        } else if (plan.getState() == FailbackPlanState.PENDING_ROLLBACK) {
+            revertFailedFailbackPlanEffects();
+        }
+    }
+
+    public synchronized void processCompleteFailbackResponse(CompleteFailbackResponseMessage reponse) {
+        /**
+         * 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(reponse.getPlanId());
+        String nodeId = plan.getNodeId();
+        failedNodes.remove(nodeId);
+        //notify impacted replicas they can reconnect to this node
+        notifyImpactedReplicas(nodeId, ClusterEventType.NODE_JOIN);
+        updateNodePartitions(nodeId, true);
+    }
+
+    private synchronized void notifyImpactedReplicas(String nodeId, ClusterEventType event) {
+        AsterixReplicationProperties replicationProperties = AsterixAppContextInfo.getInstance()
+                .getReplicationProperties();
+        Set<String> remoteReplicas = replicationProperties.getRemoteReplicasIds(nodeId);
+        String nodeIdAddress = "";
+        //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 = activeNcConfiguration.get(nodeId).get(CLUSTER_NET_IP_ADDRESS_KEY);
+        }
+
+        ReplicaEventMessage msg = new ReplicaEventMessage(nodeId, nodeIdAddress, event);
+        ICCMessageBroker messageBroker = (ICCMessageBroker) AsterixAppContextInfo.getInstance()
+                .getCCApplicationContext().getMessageBroker();
+        for (String replica : remoteReplicas) {
+            //if the remote replica is alive, send the event
+            if (activeNcConfiguration.containsKey(replica)) {
+                try {
+                    messageBroker.sendApplicationMessageToNC(msg, replica);
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+
+    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 notifyFailbackPlansNodeFailure(String nodeId) {
+        Iterator<NodeFailbackPlan> iterator = planId2FailbackPlanMap.values().iterator();
+        while (iterator.hasNext()) {
+            NodeFailbackPlan plan = iterator.next();
+            plan.notifyNodeFailure(nodeId);
+        }
+    }
+
+    public synchronized boolean isMetadataNodeActive() {
+        return metadataNodeActive;
+    }
+
+    public boolean isReplicationEnabled() {
+        if (cluster != null && cluster.getDataReplication() != null) {
+            return cluster.getDataReplication().isEnabled();
         }
         return false;
     }
+
+    public boolean isAutoFailoverEnabled() {
+        return isReplicationEnabled() && cluster.getDataReplication().isAutoFailover();
+    }
+
+    public synchronized JSONObject getClusterStateDescription() throws JSONException {
+        JSONObject stateDescription = new JSONObject();
+        stateDescription.put("State", state.name());
+        stateDescription.put("Metadata_Node", currentMetadataNode);
+        for (ClusterPartition partition : clusterPartitions.values()) {
+            stateDescription.put("partition_" + partition.getPartitionId(), partition.getActiveNodeId());
+        }
+        return stateDescription;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java
----------------------------------------------------------------------
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java
index 647a6a3..7502737 100644
--- a/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java
@@ -26,10 +26,12 @@ import java.util.HashSet;
 import java.util.Set;
 
 public class ReplicaFilesRequest {
-    Set<String> replicaIds;
+    private final Set<String> replicaIds;
+    private final Set<String> existingFiles;
 
-    public ReplicaFilesRequest(Set<String> replicaIds) {
+    public ReplicaFilesRequest(Set<String> replicaIds, Set<String> existingFiles) {
         this.replicaIds = replicaIds;
+        this.existingFiles = existingFiles;
     }
 
     public void serialize(OutputStream out) throws IOException {
@@ -38,6 +40,10 @@ public class ReplicaFilesRequest {
         for (String replicaId : replicaIds) {
             dos.writeUTF(replicaId);
         }
+        dos.writeInt(existingFiles.size());
+        for (String fileName : existingFiles) {
+            dos.writeUTF(fileName);
+        }
     }
 
     public static ReplicaFilesRequest create(DataInput input) throws IOException {
@@ -46,15 +52,19 @@ public class ReplicaFilesRequest {
         for (int i = 0; i < size; i++) {
             replicaIds.add(input.readUTF());
         }
-
-        return new ReplicaFilesRequest(replicaIds);
+        int filesCount = input.readInt();
+        Set<String> existingFiles = new HashSet<String>(filesCount);
+        for (int i = 0; i < filesCount; i++) {
+            existingFiles.add(input.readUTF());
+        }
+        return new ReplicaFilesRequest(replicaIds, existingFiles);
     }
 
     public Set<String> getReplicaIds() {
         return replicaIds;
     }
 
-    public void setReplicaIds(Set<String> replicaIds) {
-        this.replicaIds = replicaIds;
+    public Set<String> getExistingFiles() {
+        return existingFiles;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java
----------------------------------------------------------------------
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java
index 790df66..d2380c1 100644
--- a/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java
@@ -26,7 +26,6 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.SocketChannel;
 
-import org.apache.asterix.common.replication.Replica;
 import org.apache.asterix.common.replication.ReplicaEvent;
 import org.apache.asterix.common.transactions.ILogRecord;
 import org.apache.asterix.replication.management.NetworkingUtil;
@@ -52,7 +51,6 @@ public class ReplicationProtocol {
      * GET_REPLICA_LOGS: used during remote recovery to request lost txn logs
      * GET_REPLICA_MAX_LSN: used during remote recovery initialize a log manager LSN
      * GET_REPLICA_MIN_LSN: used during remote recovery to specify the low water mark per replica
-     * UPDATE_REPLICA: used to update replica info such as IP Address change.
      * 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
@@ -67,7 +65,6 @@ public class ReplicationProtocol {
         GET_REPLICA_LOGS,
         GET_REPLICA_MAX_LSN,
         GET_REPLICA_MIN_LSN,
-        UPDATE_REPLICA,
         GOODBYE,
         REPLICA_EVENT,
         LSM_COMPONENT_PROPERTIES,
@@ -115,8 +112,7 @@ public class ReplicationProtocol {
         //read replication request type
         NetworkingUtil.readBytes(socketChannel, byteBuffer, REPLICATION_REQUEST_TYPE_SIZE);
 
-        ReplicationRequestType requestType = ReplicationProtocol.ReplicationRequestType.values()[byteBuffer
-                .getInt()];
+        ReplicationRequestType requestType = ReplicationProtocol.ReplicationRequestType.values()[byteBuffer.getInt()];
         return requestType;
     }
 
@@ -215,21 +211,6 @@ public class ReplicationProtocol {
         return requestBuffer;
     }
 
-    public static ByteBuffer writeUpdateReplicaRequest(Replica replica) throws IOException {
-        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-        DataOutputStream oos = new DataOutputStream(outputStream);
-
-        oos.writeInt(ReplicationRequestType.UPDATE_REPLICA.ordinal());
-        replica.writeFields(oos);
-        oos.close();
-
-        ByteBuffer buffer = ByteBuffer.allocate(REPLICATION_REQUEST_HEADER_SIZE + oos.size());
-        buffer.putInt(ReplicationRequestType.UPDATE_REPLICA.ordinal());
-        buffer.putInt(oos.size());
-        buffer.put(outputStream.toByteArray());
-        return buffer;
-    }
-
     public static ByteBuffer writeReplicaEventRequest(ReplicaEvent event) throws IOException {
         ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
         DataOutputStream oos = new DataOutputStream(outputStream);
@@ -244,12 +225,6 @@ public class ReplicationProtocol {
         return buffer;
     }
 
-    public static Replica readReplicaUpdateRequest(ByteBuffer buffer) throws IOException {
-        ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
-        DataInputStream dis = new DataInputStream(bais);
-        return Replica.create(dis);
-    }
-
     public static ReplicaEvent readReplicaEventRequest(ByteBuffer buffer) throws IOException {
         ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
         DataInputStream dis = new DataInputStream(bais);
@@ -257,22 +232,24 @@ public class ReplicationProtocol {
         return ReplicaEvent.create(dis);
     }
 
-    public static void writeGetReplicaFilesRequest(ByteBuffer buffer, ReplicaFilesRequest request) throws IOException {
+    public static ByteBuffer writeGetReplicaFilesRequest(ByteBuffer buffer, ReplicaFilesRequest request)
+            throws IOException {
         ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-        DataOutputStream oos = new DataOutputStream(outputStream);
-        request.serialize(oos);
-        oos.close();
-
-        int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
-        if (buffer.capacity() < requestSize) {
-            buffer = ByteBuffer.allocate(requestSize);
-        } else {
-            buffer.clear();
+        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.toByteArray());
+            buffer.flip();
+            return buffer;
         }
-        buffer.putInt(ReplicationRequestType.GET_REPLICA_FILES.ordinal());
-        buffer.putInt(oos.size());
-        buffer.put(outputStream.toByteArray());
-        buffer.flip();
     }
 
     public static ByteBuffer writeGetReplicaIndexFlushRequest(ByteBuffer buffer, ReplicaIndexFlushRequest request)

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/98d38e6a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaEventNotifier.java
----------------------------------------------------------------------
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaEventNotifier.java b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaEventNotifier.java
deleted file mode 100644
index 9915c83..0000000
--- a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaEventNotifier.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.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.Set;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.common.config.AsterixReplicationProperties;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.replication.ReplicaEvent;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
-
-public class ReplicaEventNotifier implements Runnable {
-
-    private static final Logger LOGGER = Logger.getLogger(ReplicaEventNotifier.class.getName());
-
-    final int WAIT_TIME = 2000;
-    final Set<Replica> notifyReplicaNodes;
-
-    int notificationTimeOut;
-
-    final ReplicaEvent event;
-    final AsterixReplicationProperties asterixReplicationProperties;
-
-    public ReplicaEventNotifier(ReplicaEvent event, AsterixReplicationProperties asterixReplicationProperties) {
-        this.event = event;
-        this.asterixReplicationProperties = asterixReplicationProperties;
-        notificationTimeOut = asterixReplicationProperties.getReplicationTimeOut();
-        notifyReplicaNodes = asterixReplicationProperties.getRemoteReplicas(event.getReplica().getId());
-    }
-
-    @Override
-    public void run() {
-        Thread.currentThread().setName("ReplicaEventNotifier Thread");
-
-        if (notifyReplicaNodes == null) {
-            return;
-        }
-
-        ByteBuffer buffer = null;
-        try {
-            buffer = ReplicationProtocol.writeReplicaEventRequest(event);
-        } catch (IOException e) {
-            e.printStackTrace();
-        }
-
-        for (Replica replica : notifyReplicaNodes) {
-            long startTime = System.currentTimeMillis();
-            InetSocketAddress replicaAddress = replica.getAddress(asterixReplicationProperties);
-            SocketChannel connection = null;
-
-            while (true) {
-                try {
-                    connection = SocketChannel.open();
-                    connection.configureBlocking(true);
-                    connection.connect(new InetSocketAddress(replicaAddress.getHostString(), replicaAddress.getPort()));
-                    //send replica event
-                    connection.write(buffer);
-                    //send goodbye
-                    connection.write(ReplicationProtocol.getGoodbyeBuffer());
-                    break;
-                } catch (IOException | UnresolvedAddressException e) {
-                    try {
-                        Thread.sleep(WAIT_TIME);
-                    } catch (InterruptedException e1) {
-                        //ignore
-                    }
-
-                    //check if connection to replica timed out
-                    if (((System.currentTimeMillis() - startTime) / 1000) >= notificationTimeOut) {
-                        LOGGER.log(Level.WARNING, "Could not send ReplicaEvent to " + replica);
-                        break;
-                    }
-                } finally {
-                    if (connection.isOpen()) {
-                        try {
-                            connection.close();
-                        } catch (IOException e) {
-                            e.printStackTrace();
-                        }
-                    }
-                    buffer.position(0);
-                }
-            }
-        }
-    }
-}