diff --git a/fdbclient/BackupContainer.h b/fdbclient/BackupContainer.h
index 75e209216f..e4f6ebf1de 100644
--- a/fdbclient/BackupContainer.h
+++ b/fdbclient/BackupContainer.h
@@ -18,6 +18,8 @@
  * limitations under the License.
  */
 
+#ifndef FDBCLIENT_BackupContainer_H
+#define FDBCLIENT_BackupContainer_H
 #pragma once
 
 #include "flow/flow.h"
@@ -27,6 +29,8 @@
 #include "fdbclient/ReadYourWrites.h"
 #include <vector>
 
+class ReadYourWritesTransaction;
+
 Future<Optional<int64_t>> timeKeeperEpochsFromVersion(Version const &v, Reference<ReadYourWritesTransaction> const &tr);
 Future<Version> timeKeeperVersionFromDatetime(std::string const &datetime, Database const &db);
 
@@ -255,3 +259,4 @@ private:
 	std::string URL;
 };
 
+#endif
\ No newline at end of file
diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp
index 6c6ea5b071..ebf078748b 100644
--- a/fdbclient/SystemData.cpp
+++ b/fdbclient/SystemData.cpp
@@ -601,6 +601,14 @@ const KeyRangeRef restoreWorkersKeys(
 	LiteralStringRef("\xff\x02/restoreWorkers/"),
 	LiteralStringRef("\xff\x02/restoreWorkers0")
 );
+const KeyRangeRef restoreLoaderKeys(
+	LiteralStringRef("\xff\x02/restoreLoaders/"),
+	LiteralStringRef("\xff\x02/restoreLoaders0")
+);
+const KeyRangeRef restoreApplierKeys(
+	LiteralStringRef("\xff\x02/restoreAppliers/"),
+	LiteralStringRef("\xff\x02/restoreAppliers0")
+);
 const KeyRef restoreStatusKey = LiteralStringRef("\xff\x02/restoreStatus/");
 
 
@@ -611,24 +619,64 @@ const KeyRangeRef restoreRequestKeys(
 		LiteralStringRef("\xff\x02/restoreRequests0")
 );
 
-// Encode restore agent key for agentID
-const Key restoreWorkerKeyFor( UID const& agentID ) {
+// Encode restore worker key for workerID
+const Key restoreWorkerKeyFor( UID const& workerID ) {
 	BinaryWriter wr(Unversioned());
 	wr.serializeBytes( restoreWorkersKeys.begin );
-	wr << agentID;
+	wr << workerID;
+	return wr.toValue();
+}
+
+// Encode restore role (loader or applier) for roleID
+const Key restoreLoaderKeyFor( UID const& roleID ) {
+	BinaryWriter wr(Unversioned());
+	wr.serializeBytes( restoreLoaderKeys.begin );
+	wr << roleID;
+	return wr.toValue();
+}
+
+const Key restoreApplierKeyFor( UID const& roleID ) {
+	BinaryWriter wr(Unversioned());
+	wr.serializeBytes( restoreApplierKeys.begin );
+	wr << roleID;
 	return wr.toValue();
 }
 
 // Encode restore agent value
-
-const Value restoreCommandInterfaceValue( RestoreInterface const& cmdInterf ) {
+const Value restoreWorkerInterfaceValue( RestoreWorkerInterface const& cmdInterf ) {
 	BinaryWriter wr(IncludeVersion());
 	wr << cmdInterf;
 	return wr.toValue();
 }
 
-RestoreInterface decodeRestoreCommandInterfaceValue( ValueRef const& value ) {
-	RestoreInterface s;
+RestoreWorkerInterface decodeRestoreWorkerInterfaceValue( ValueRef const& value ) {
+	RestoreWorkerInterface s;
+	BinaryReader reader( value, IncludeVersion() );
+	reader >> s;
+	return s;
+}
+
+const Value restoreLoaderInterfaceValue( RestoreLoaderInterface const& cmdInterf ) {
+	BinaryWriter wr(IncludeVersion());
+	wr << cmdInterf;
+	return wr.toValue();
+}
+
+RestoreLoaderInterface decodeRestoreLoaderInterfaceValue( ValueRef const& value ) {
+	RestoreLoaderInterface s;
+	BinaryReader reader( value, IncludeVersion() );
+	reader >> s;
+	return s;
+}
+
+const Value restoreApplierInterfaceValue( RestoreApplierInterface const& cmdInterf ) {
+	BinaryWriter wr(IncludeVersion());
+	wr << cmdInterf;
+	return wr.toValue();
+}
+
+RestoreApplierInterface decodeRestoreApplierInterfaceValue( ValueRef const& value ) {
+	RestoreApplierInterface s;
 	BinaryReader reader( value, IncludeVersion() );
 	reader >> s;
 	return s;
diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h
index 70342b68ae..f3b8174fe9 100644
--- a/fdbclient/SystemData.h
+++ b/fdbclient/SystemData.h
@@ -27,6 +27,9 @@
 #include "fdbclient/FDBTypes.h"
 #include "fdbclient/StorageServerInterface.h"
 #include "fdbserver/RestoreWorkerInterface.h"
+struct RestoreLoaderInterface;
+struct RestoreApplierInterface;
+struct RestoreMasterInterface;
 
 extern const KeyRangeRef normalKeys; // '' to systemKeys.begin
 extern const KeyRangeRef systemKeys;  // [FF] to [FF][FF]
@@ -275,6 +278,9 @@ extern const KeyRangeRef monitorConfKeys;
 
 extern const KeyRef restoreLeaderKey;
 extern const KeyRangeRef restoreWorkersKeys;
+extern const KeyRangeRef restoreRolesKeys;
+extern const KeyRangeRef restoreLoaderKeys;
+extern const KeyRangeRef restoreApplierKeys;
 
 extern const KeyRef restoreStatusKey;
 
@@ -282,9 +288,16 @@ extern const KeyRef restoreRequestTriggerKey;
 extern const KeyRef restoreRequestDoneKey;
 extern const KeyRangeRef restoreRequestKeys;
 
-const Key restoreWorkerKeyFor( UID const& agentID );
-const Value restoreCommandInterfaceValue( RestoreInterface const& server );
-RestoreInterface decodeRestoreCommandInterfaceValue( ValueRef const& value );
+const Key restoreWorkerKeyFor( UID const& workerID );
+const Key restoreLoaderKeyFor( UID const& roleID );
+const Key restoreApplierKeyFor( UID const& roleID );
+
+const Value restoreWorkerInterfaceValue(RestoreWorkerInterface const& server );
+RestoreWorkerInterface decodeRestoreWorkerInterfaceValue( ValueRef const& value );
+const Value restoreLoaderInterfaceValue(RestoreLoaderInterface const& server );
+RestoreLoaderInterface decodeRestoreLoaderInterfaceValue( ValueRef const& value );
+const Value restoreApplierInterfaceValue(RestoreApplierInterface const& server );
+RestoreApplierInterface decodeRestoreApplierInterfaceValue( ValueRef const& value );
 
 // MX: parallel restore
 const Value restoreRequestTriggerValue (int const numRequests);
diff --git a/fdbserver/Restore.actor.cpp b/fdbserver/Restore.actor.cpp
index 6fa7a80efc..dd73d11e2b 100644
--- a/fdbserver/Restore.actor.cpp
+++ b/fdbserver/Restore.actor.cpp
@@ -18,7 +18,7 @@
  * limitations under the License.
  */
 
-#include "fdbserver/RestoreWorkerInterface.h"
+
 #include "fdbclient/NativeAPI.actor.h"
 #include "fdbclient/SystemData.h"
 
@@ -39,7 +39,14 @@
 #include <boost/algorithm/string/classification.hpp>
 #include <algorithm>
 
+#include "flow/ActorCollection.h"
+#include "fdbserver/RestoreUtil.h"
+#include "fdbserver/RestoreWorkerInterface.h"
 #include "fdbserver/RestoreCommon.actor.h"
+#include "fdbserver/RestoreRoleCommon.actor.h"
+#include "fdbserver/RestoreLoader.actor.h"
+#include "fdbserver/RestoreApplier.actor.h"
+#include "fdbserver/RestoreMaster.actor.h"
 
 #include "flow/actorcompiler.h"  // This must be the last #include.
 
@@ -52,95 +59,25 @@ double loadBatchSizeThresholdB = loadBatchSizeMB * 1024 * 1024;
 double mutationVectorThreshold =  100; // Bytes // correctness passed when the value is 1
 double transactionBatchSizeThreshold =  512; // Byte
 
+int restoreStatusIndex = 0;
+
 class RestoreConfig;
-struct RestoreData; // Only declare the struct exist but we cannot use its field
+struct RestoreWorkerData; // Only declare the struct exist but we cannot use its field
 
-// Forward declaration
-ACTOR Future<Void> registerMutationsToApplier(Reference<RestoreData> rd);
-ACTOR Future<Void> registerMutationsToMasterApplier(Reference<RestoreData> rd);
-ACTOR Future<Void> notifyApplierToApplyMutations(Reference<RestoreData> rd);
-ACTOR Future<Void> notifyWorkersToSetWorkersInterface(Reference<RestoreData> rd);
-ACTOR Future<Void> configureRoles(Reference<RestoreData> rd);
-ACTOR Future<Void> notifyWorkersToSetWorkersInterface(Reference<RestoreData> rd);
-ACTOR Future<Void> handleSendSampleMutationVectorRequest(RestoreSendMutationVectorRequest req, Reference<RestoreData> rd, RestoreInterface interf);
-ACTOR Future<Void> handleFinishRestoreReq(RestoreSimpleRequest req, Reference<RestoreData> rd, RestoreInterface interf, Database cx);
+// Forwaself declaration
+void initRestoreWorkerConfig();
 
-ACTOR Future<Void> workerCore( Reference<RestoreData> rd, RestoreInterface ri, Database cx );
-ACTOR Future<Void> masterCore(Reference<RestoreData> rd, RestoreInterface ri, Database cx);
-
-ACTOR static Future<Version> processRestoreRequest(RestoreInterface interf, Reference<RestoreData> rd, Database cx, RestoreRequest request);
-ACTOR static Future<Void> finishRestore(Reference<RestoreData> rd, Database cx, Standalone<VectorRef<RestoreRequest>> restoreRequests);
-ACTOR static Future<Void> _clearDB(Reference<ReadYourWritesTransaction> tr);
-
-bool concatenateBackupMutationForLogFile(Reference<RestoreData> rd, Standalone<StringRef> val_input, Standalone<StringRef> key_input);
-void concatenateBackupMutation(Standalone<StringRef> val_input, Standalone<StringRef> key_input);
-void registerBackupMutationForAll(Version empty);
-bool isKVOpsSorted(Reference<RestoreData> rd);
-bool allOpsAreKnown(Reference<RestoreData> rd);
-void sanityCheckMutationOps(Reference<RestoreData> rd);
-void parseSerializedMutation(Reference<RestoreData> rd, bool isSampling = false);
-bool collectFilesForOneVersionBatch(Reference<RestoreData> rd);
-
-// Helper class for reading restore data from a buffer and throwing the right errors.
-// This struct is mostly copied from StringRefReader. We add a sanity check in this struct.
-// TODO: Merge this struct with StringRefReader.
-struct StringRefReaderMX {
-	StringRefReaderMX(StringRef s = StringRef(), Error e = Error()) : rptr(s.begin()), end(s.end()), failure_error(e), str_size(s.size()) {}
-
-	// Return remainder of data as a StringRef
-	StringRef remainder() {
-		return StringRef(rptr, end - rptr);
-	}
-
-	// Return a pointer to len bytes at the current read position and advance read pos
-	//Consume a little-Endian data. Since we only run on little-Endian machine, the data on storage is little Endian
-	const uint8_t * consume(unsigned int len) {
-		if(rptr == end && len != 0)
-			throw end_of_stream();
-		const uint8_t *p = rptr;
-		rptr += len;
-		if(rptr > end) {
-			printf("[ERROR] StringRefReaderMX throw error! string length:%d\n", str_size);
-			printf("!!!!!!!!!!!![ERROR]!!!!!!!!!!!!!! Worker may die due to the error. Master will stuck when a worker die\n");
-			throw failure_error;
-		}
-		return p;
-	}
-
-	// Return a T from the current read position and advance read pos
-	template<typename T> const T consume() {
-		return *(const T *)consume(sizeof(T));
-	}
-
-	// Functions for consuming big endian (network byte order) integers.
-	// Consumes a big endian number, swaps it to little endian, and returns it.
-	const int32_t  consumeNetworkInt32()  { return (int32_t)bigEndian32((uint32_t)consume< int32_t>());}
-	const uint32_t consumeNetworkUInt32() { return          bigEndian32(          consume<uint32_t>());}
-
-	const int64_t  consumeNetworkInt64()  { return (int64_t)bigEndian64((uint32_t)consume< int64_t>());}
-	const uint64_t consumeNetworkUInt64() { return          bigEndian64(          consume<uint64_t>());}
-
-	bool eof() { return rptr == end; }
-
-	const uint8_t *rptr, *end;
-	const int str_size;
-	Error failure_error;
-};
+ACTOR Future<Void> handlerTerminateWorkerRequest(RestoreSimpleRequest req, Reference<RestoreWorkerData> self, RestoreWorkerInterface workerInterf, Database cx);
+ACTOR Future<Void> monitorWorkerLiveness(Reference<RestoreWorkerData> self);
+ACTOR Future<Void> commitRestoreRoleInterfaces(Reference<RestoreWorkerData> self, Database cx);
+ACTOR Future<Void> handleRecruitRoleRequest(RestoreRecruitRoleRequest req, Reference<RestoreWorkerData> self, ActorCollection *actors, Database cx);
+ACTOR Future<Void> collectRestoreWorkerInterface(Reference<RestoreWorkerData> self, Database cx, int min_num_workers);
+ACTOR Future<Void> recruitRestoreRoles(Reference<RestoreWorkerData> self);
 
 bool debug_verbose = true;
-void printGlobalNodeStatus(Reference<RestoreData>);
+void printGlobalNodeStatus(Reference<RestoreWorkerData>);
 
 
-std::vector<std::string> RestoreRoleStr = {"Invalid", "Master", "Loader", "Applier"};
-int numRoles = RestoreRoleStr.size();
-std::string getRoleStr(RestoreRole role) {
-	if ( (int) role >= numRoles || (int) role < 0) {
-		printf("[ERROR] role:%d is out of scope\n", (int) role);
-		return "[Unset]";
-	}
-	return RestoreRoleStr[(int)role];
-}
-
 const char *RestoreCommandEnumStr[] = {"Init",
 		"Set_Role", "Set_Role_Done",
 		"Sample_Range_File", "Sample_Log_File", "Sample_File_Done",
@@ -157,42 +94,6 @@ const char *RestoreCommandEnumStr[] = {"Init",
 template<> Tuple Codec<ERestoreState>::pack(ERestoreState const &val); // { return Tuple().append(val); }
 template<> ERestoreState Codec<ERestoreState>::unpack(Tuple const &val); // { return (ERestoreState)val.getInt(0); }
 
-// CMDUID implementation
-void CMDUID::initPhase(RestoreCommandEnum newPhase) {
-	printf("CMDID, current phase:%d, new phase:%d\n", phase, newPhase);
-	phase = (uint16_t) newPhase;
-	cmdID = 0;
-}
-
-void CMDUID::nextPhase() {
-	phase++;
-	cmdID = 0;
-}
-
-void CMDUID::nextCmd() {
-	cmdID++;
-}
-
-RestoreCommandEnum CMDUID::getPhase() {
-	return (RestoreCommandEnum) phase;
-}
-
-void CMDUID::setPhase(RestoreCommandEnum newPhase) {
-	phase = (uint16_t) newPhase;
-}
-
-void CMDUID::setBatch(int newBatchIndex) {
-	batch = newBatchIndex;
-}
-
-uint64_t CMDUID::getIndex() {
-	return cmdID;
-}
-
-std::string CMDUID::toString() const {
-	return format("%04ld|%04ld|%016lld", batch, phase, cmdID);
-}
-
 
 // DEBUG_FAST_RESTORE is not used right now!
 #define DEBUG_FAST_RESTORE 1
@@ -203,740 +104,42 @@ std::string CMDUID::toString() const {
 #define dbprintf_rs(fmt, args...)
 #endif
 
-// RestoreData is the context for each restore process (worker and master)
-struct RestoreData : NonCopyable, public ReferenceCounted<RestoreData>  {
-	//---- Declare status structure which records the progress and status of each worker in each role
-	std::map<UID, RestoreInterface> workers_interface; // UID is worker's node id, RestoreInterface is worker's communication interface
-	UID masterApplier; //TODO: Remove this variable. The first version uses 1 applier to apply the mutations
 
-	RestoreNodeStatus localNodeStatus; //Each worker node (process) has one such variable.
-	std::vector<RestoreNodeStatus> globalNodeStatus; // status of all notes, excluding master node, stored in master node // May change to map, like servers_info
+// Each restore worker (a process) is assigned for a role.
+// MAYBE Later: We will support multiple restore roles on a worker
+struct RestoreWorkerData :  NonCopyable, public ReferenceCounted<RestoreWorkerData> {
+	UID workerID;
+	std::map<UID, RestoreWorkerInterface> workers_workerInterface; // UID is worker's node id, RestoreWorkerInterface is worker's communication workerInterface
 
-	// range2Applier is in master and loader node. Loader node uses this to determine which applier a mutation should be sent
-	std::map<Standalone<KeyRef>, UID> range2Applier; // KeyRef is the inclusive lower bound of the key range the applier (UID) is responsible for
-	std::map<Standalone<KeyRef>, int> keyOpsCount; // The number of operations per key which is used to determine the key-range boundary for appliers
-	int numSampledMutations; // The total number of mutations received from sampled data.
+	// Restore Roles
+	Optional<RestoreLoaderInterface> loaderInterf;
+	Reference<RestoreLoaderData> loaderData;
+	Optional<RestoreApplierInterface> applierInterf;
+	Reference<RestoreApplierData> applierData;
+	Reference<RestoreMasterData> masterData;
 
-	struct ApplierStatus { // NOT USED //TODO: Remove this
-		UID id;
-		KeyRange keyRange; // the key range the applier is responsible for
-		// Applier state is changed at the following event
-		// Init: when applier's role is set
-		// Assigned: when applier is set for a key range to be respoinsible for
-		// Applying: when applier starts to apply the mutations to DB after receiving the cmd from loader
-		// Done: when applier has finished applying the mutation and notify the master. It will change to Assigned after Done
-		enum class ApplierState {Invalid = 0, Init = 1, Assigned, Applying, Done};
-		ApplierState state;
-	};
-	ApplierStatus applierStatus;
+	CMDUID cmdID;
 
-	// TODO: Record loading progress for (i) operators to check the restore status; (ii) recovering from node fault in the middle of restore
+	UID id() const { return workerID; };
 
-	// Loader's state to handle the duplicate delivery of loading commands
-	std::map<std::string, int> processedFiles; //first is filename of processed file, second is not used
-	std::map<CMDUID, int> processedCmd;
-	bool inProgressApplyToDB = false;
-	uint32_t inProgressFlag = 0;
-	CMDUID cmdID; // Command id to record the progress
-
-
-	// Temporary variables to hold files and data to restore
-	std::vector<RestoreFileFR> allFiles; // All backup files to be processed in all version batches
-	std::vector<RestoreFileFR> files; // Backup files to be parsed and applied: range and log files in 1 version batch
-	std::map<Version, Version> forbiddenVersions; // forbidden version range [first, second)
-
-	// Temporary data structure for parsing range and log files into (version, <K, V, mutationType>)
-	std::map<Version, Standalone<VectorRef<MutationRef>>> kvOps;
-	// Must use StandAlone to save mutations, otherwise, the mutationref memory will be corrupted
-	std::map<Standalone<StringRef>, Standalone<StringRef>> mutationMap; // Key is the unique identifier for a batch of mutation logs at the same version
-	std::map<Standalone<StringRef>, uint32_t> mutationPartMap; // Record the most recent
-
-	// In each version batch, we process the files in [curBackupFilesBeginIndex, curBackupFilesEndIndex] in RestoreData.allFiles.
-	long curBackupFilesBeginIndex;
-	long curBackupFilesEndIndex;
-	double totalWorkloadSize;
-	double curWorkloadSize;
-	int batchIndex;
-
-
-	Reference<IBackupContainer> bc; // Backup container is used to read backup files
-	Key bcUrl; // The url used to get the bc
-
-	// For master applier to hold the lower bound of key ranges for each appliers
-	std::vector<Standalone<KeyRef>> keyRangeLowerBounds;
-
-	// Helper functions to set/clear the flag when a worker is in the middle of processing an actor.
-	void setInProgressFlag(RestoreCommandEnum phaseEnum) {
-		int phase = (int) phaseEnum;
-		ASSERT(phase < 32);
-		inProgressFlag |= (1UL << phase);
+	RestoreWorkerData() {
+		workerID = UID();
 	}
 
-	void clearInProgressFlag(RestoreCommandEnum phaseEnum) {
-		int phase = (int) phaseEnum;
-		ASSERT(phase < 32);
-		inProgressFlag &= ~(1UL << phase);
+	~RestoreWorkerData() {
+		printf("[Exit] Worker:%s RestoreWorkerData is deleted\n", workerID.toString().c_str());
 	}
 
-	bool isInProgress(RestoreCommandEnum phaseEnum) {
-		int phase = (int) phaseEnum;
-		ASSERT(phase < 32);
-		return (inProgressFlag & (1UL << phase));
-	}
-
-	RestoreRole getRole() {
-		return localNodeStatus.role;
-	}
-
-	bool isCmdProcessed(CMDUID const &cmdID) {
-		return processedCmd.find(cmdID) != processedCmd.end();
-	}
-
-	// Describe the node information
 	std::string describeNode() {
 		std::stringstream ss;
-		ss << "[Role:" << getRoleStr(localNodeStatus.role) << "] [NodeID:" << localNodeStatus.nodeID.toString().c_str()
-			<< "] [NodeIndex:" << std::to_string(localNodeStatus.nodeIndex) << "]";
+		ss << "RestoreWorker workerID:" << workerID.toString();
 		return ss.str();
 	}
-
-	void resetPerVersionBatch() {
-		printf("[INFO]Node:%s resetPerVersionBatch\n", localNodeStatus.nodeID.toString().c_str());
-		range2Applier.clear();
-		keyOpsCount.clear();
-		numSampledMutations = 0;
-		kvOps.clear();
-		mutationMap.clear();
-		mutationPartMap.clear();
-		processedCmd.clear();
-		inProgressApplyToDB = false;
-		files.clear(); // files are backup files for a version batch
-		curWorkloadSize = 0;
-	}
-
-	vector<UID> getBusyAppliers() {
-		vector<UID> busyAppliers;
-		for (auto &app : range2Applier) {
-			busyAppliers.push_back(app.second);
-		}
-		return busyAppliers;
-	}
-
-	RestoreData() {
-		cmdID.initPhase(RestoreCommandEnum::Init);
-		localNodeStatus.role = RestoreRole::Invalid;
-		localNodeStatus.nodeIndex = 0;
-		curBackupFilesBeginIndex = 0;
-		curBackupFilesEndIndex = 0;
-	 	totalWorkloadSize = 0;
-		curWorkloadSize = 0;
-		batchIndex = 0;
-		bc = Reference<IBackupContainer>();
-		bcUrl = StringRef();
-	}
-
-	~RestoreData() {
-		printf("[Exit] NodeID:%s RestoreData is deleted\n", localNodeStatus.nodeID.toString().c_str());
-	}
 };
 
 
-void printAppliersKeyRange(Reference<RestoreData> rd) {
-	printf("[INFO] The mapping of KeyRange_start --> Applier ID\n");
-	// applier type: std::map<Standalone<KeyRef>, UID>
-	for (auto &applier : rd->range2Applier) {
-		printf("\t[INFO]%s -> %s\n", getHexString(applier.first).c_str(), applier.second.toString().c_str());
-	}
-}
-
-//Print out the works_interface info
-void printWorkersInterface(Reference<RestoreData> rd) {
-	printf("[INFO] workers_interface info: num of workers:%ld\n", rd->workers_interface.size());
-	int index = 0;
-	for (auto &interf : rd->workers_interface) {
-		printf("\t[INFO][Worker %d] NodeID:%s, Interface.id():%s\n", index,
-				interf.first.toString().c_str(), interf.second.id().toString().c_str());
-	}
-}
-
-// Return <num_of_loader, num_of_applier> in the system
-std::pair<int, int> getNumLoaderAndApplier(Reference<RestoreData> rd){
-	int numLoaders = 0;
-	int numAppliers = 0;
-	for (int i = 0; i < rd->globalNodeStatus.size(); ++i) {
-		if (rd->globalNodeStatus[i].role == RestoreRole::Loader) {
-			numLoaders++;
-		} else if (rd->globalNodeStatus[i].role == RestoreRole::Applier) {
-			numAppliers++;
-		} else {
-			printf("[ERROR] unknown role: %d\n", rd->globalNodeStatus[i].role);
-		}
-	}
-
-	if ( numLoaders + numAppliers != rd->globalNodeStatus.size() ) {
-		printf("[ERROR] Number of workers does not add up! numLoaders:%d, numApplier:%d, totalProcess:%ld\n",
-				numLoaders, numAppliers, rd->globalNodeStatus.size());
-	}
-
-	return std::make_pair(numLoaders, numAppliers);
-}
-
-std::vector<UID> getWorkingApplierIDs(Reference<RestoreData> rd) {
-	std::vector<UID> applierIDs;
-	for ( auto &applier : rd->range2Applier ) {
-		applierIDs.push_back(applier.second);
-	}
-
-	ASSERT( !applierIDs.empty() );
-	return applierIDs;
-}
-
-std::vector<UID> getApplierIDs(Reference<RestoreData> rd) {
-	std::vector<UID> applierIDs;
-	for (int i = 0; i < rd->globalNodeStatus.size(); ++i) {
-		if (rd->globalNodeStatus[i].role == RestoreRole::Applier) {
-			applierIDs.push_back(rd->globalNodeStatus[i].nodeID);
-		}
-	}
-
-	// Check if there exist duplicate applier IDs, which should never occur
-	std::sort(applierIDs.begin(), applierIDs.end());
-	bool unique = true;
-	for (int i = 1; i < applierIDs.size(); ++i) {
-		if (applierIDs[i-1] == applierIDs[i]) {
-			unique = false;
-			break;
-		}
-	}
-	if (!unique) {
-		fprintf(stderr, "[ERROR] Applier IDs are not unique! All worker IDs are as follows\n");
-		printGlobalNodeStatus(rd);
-	}
-
-	ASSERT( !applierIDs.empty() );
-	return applierIDs;
-}
-
-std::vector<UID> getLoaderIDs(Reference<RestoreData> rd) {
-	std::vector<UID> loaderIDs;
-	for (int i = 0; i < rd->globalNodeStatus.size(); ++i) {
-		if (rd->globalNodeStatus[i].role == RestoreRole::Loader) {
-			loaderIDs.push_back(rd->globalNodeStatus[i].nodeID);
-		}
-	}
-
-	// Check if there exist duplicate applier IDs, which should never occur
-	std::sort(loaderIDs.begin(), loaderIDs.end());
-	bool unique = true;
-	for (int i = 1; i < loaderIDs.size(); ++i) {
-		if (loaderIDs[i-1] == loaderIDs[i]) {
-			unique = false;
-			break;
-		}
-	}
-	if (!unique) {
-		printf("[ERROR] Applier IDs are not unique! All worker IDs are as follows\n");
-		printGlobalNodeStatus(rd);
-	}
-
-	return loaderIDs;
-}
-
-std::vector<UID> getWorkerIDs(Reference<RestoreData> rd) {
-	std::vector<UID> workerIDs;
-	for (int i = 0; i < rd->globalNodeStatus.size(); ++i) {
-		if (rd->globalNodeStatus[i].role == RestoreRole::Loader ||
-			rd->globalNodeStatus[i].role == RestoreRole::Applier) {
-			workerIDs.push_back(rd->globalNodeStatus[i].nodeID);
-		}
-	}
-
-	// Check if there exist duplicate applier IDs, which should never occur
-	std::sort(workerIDs.begin(), workerIDs.end());
-	bool unique = true;
-	for (int i = 1; i < workerIDs.size(); ++i) {
-		if (workerIDs[i-1] == workerIDs[i]) {
-			unique = false;
-			break;
-		}
-	}
-	if (!unique) {
-		printf("[ERROR] Applier IDs are not unique! All worker IDs are as follows\n");
-		printGlobalNodeStatus(rd);
-	}
-
-	return workerIDs;
-}
-
-void printGlobalNodeStatus(Reference<RestoreData> rd) {
-	printf("---Print globalNodeStatus---\n");
-	printf("Number of entries:%ld\n", rd->globalNodeStatus.size());
-	for(int i = 0; i < rd->globalNodeStatus.size(); ++i) {
-		printf("[Node:%d] %s, role:%s\n", i, rd->globalNodeStatus[i].toString().c_str(),
-				getRoleStr(rd->globalNodeStatus[i].role).c_str());
-	}
-}
-
-void printBackupFilesInfo(Reference<RestoreData> rd) {
-	printf("[INFO] The backup files for current batch to load and apply: num:%ld\n", rd->files.size());
-	for (int i = 0; i < rd->files.size(); ++i) {
-		printf("\t[INFO][File %d] %s\n", i, rd->files[i].toString().c_str());
-	}
-}
-
-
-void printAllBackupFilesInfo(Reference<RestoreData> rd) {
-	printf("[INFO] All backup files: num:%ld\n", rd->allFiles.size());
-	for (int i = 0; i < rd->allFiles.size(); ++i) {
-		printf("\t[INFO][File %d] %s\n", i, rd->allFiles[i].toString().c_str());
-	}
-}
-
-void buildForbiddenVersionRange(Reference<RestoreData> rd) {
-
-	printf("[INFO] Build forbidden version ranges for all backup files: num:%ld\n", rd->allFiles.size());
-	for (int i = 0; i < rd->allFiles.size(); ++i) {
-		if (!rd->allFiles[i].isRange) {
-			rd->forbiddenVersions.insert(std::make_pair(rd->allFiles[i].beginVersion, rd->allFiles[i].endVersion));
-		}
-	}
-}
-
-bool isForbiddenVersionRangeOverlapped(Reference<RestoreData> rd) {
-	printf("[INFO] Check if forbidden version ranges is overlapped: num of ranges:%ld\n", rd->forbiddenVersions.size());
-	if (rd->forbiddenVersions.empty()) {
-		return false;
-	}
-
-	std::map<Version, Version>::iterator prevRange = rd->forbiddenVersions.begin();
-	std::map<Version, Version>::iterator curRange = rd->forbiddenVersions.begin();
-	curRange++; // Assume rd->forbiddenVersions has at least one element!
-
-	while ( curRange != rd->forbiddenVersions.end() ) {
-		if ( curRange->first < prevRange->second ) {
-			return true; // overlapped
-		}
-		curRange++;
-	}
-
-	return false; //not overlapped
-}
-
-// endVersion is begin version for range file, because range file takes snapshot at the same version
-// endVersion is the end version (excluded) for mutations recorded in log file
-bool isVersionInForbiddenRange(Reference<RestoreData> rd, Version endVersion, bool isRange) {
-	bool isForbidden = false;
-	for (auto &range : rd->forbiddenVersions) {
-		if ( isRange ) { //the range file includes mutations at the endVersion
-			if (endVersion >= range.first && endVersion < range.second) {
-				isForbidden = true;
-				break;
-			}
-		} else { // the log file does NOT include mutations at the endVersion
-			continue; // Log file's endVersion is always a valid version batch boundary as long as the forbidden version ranges do not overlap
-		}
-	}
-
-	return isForbidden;
-}
-
-void printForbiddenVersionRange(Reference<RestoreData> rd) {
-	printf("[INFO] Number of forbidden version ranges:%ld\n", rd->forbiddenVersions.size());
-	int i = 0;
-	for (auto &range : rd->forbiddenVersions) {
-		printf("\t[INFO][Range%d] [%ld, %ld)\n", i, range.first, range.second);
-		++i;
-	}
-}
-
-void constructFilesWithVersionRange(Reference<RestoreData> rd) {
-	printf("[INFO] constructFilesWithVersionRange for num_files:%ld\n", rd->files.size());
-	rd->allFiles.clear();
-	for (int i = 0; i < rd->files.size(); i++) {
-		printf("\t[File:%d] Start %s\n", i, rd->files[i].toString().c_str());
-		Version beginVersion = 0;
-		Version endVersion = 0;
-		if (rd->files[i].isRange) {
-			// No need to parse range filename to get endVersion
-			beginVersion = rd->files[i].version;
-			endVersion = beginVersion;
-		} else { // Log file
-			//Refer to pathToLogFile() in BackupContainer.actor.cpp
-			long blockSize, len;
-			int pos = rd->files[i].fileName.find_last_of("/");
-			std::string fileName = rd->files[i].fileName.substr(pos);
-			printf("\t[File:%d] Log filename:%s, pos:%d\n", i, fileName.c_str(), pos);
-			sscanf(fileName.c_str(), "/log,%ld,%ld,%*[^,],%lu%ln", &beginVersion, &endVersion, &blockSize, &len);
-			printf("\t[File:%d] Log filename:%s produces beginVersion:%ld endVersion:%ld\n",i, fileName.c_str(), beginVersion, endVersion);
-		}
-		rd->files[i].beginVersion = beginVersion;
-		rd->files[i].endVersion = endVersion;
-		printf("\t[File:%d] End %s\n", i, rd->files[i].toString().c_str());
-		ASSERT(beginVersion <= endVersion);
-		rd->allFiles.push_back(rd->files[i]);
-		// rd->allFiles.back().beginVersion = beginVersion;
-		// rd->allFiles.back().endVersion = endVersion;
-	}
-}
-
-
-//// --- Some common functions
- ACTOR static Future<Void> _parseRangeFileToMutationsOnLoader(Reference<RestoreData> rd,
- 									Reference<IBackupContainer> bc, Version version,
- 									std::string fileName, int64_t readOffset_input, int64_t readLen_input,
- 									KeyRange restoreRange, Key addPrefix, Key removePrefix) {
-
- 	state int64_t readOffset = readOffset_input;
- 	state int64_t readLen = readLen_input;
-
-	if ( debug_verbose ) {
-		printf("[VERBOSE_DEBUG] Parse range file and get mutations 1, bc:%lx\n", bc.getPtr());
-	}
- 	// The set of key value version is rangeFile.version. the key-value set in the same range file has the same version
- 	Reference<IAsyncFile> inFile = wait(bc->readFile(fileName));
-
-	if ( debug_verbose ) {
-		printf("[VERBOSE_DEBUG] Parse range file and get mutations 2\n");
-	}
- 	state Standalone<VectorRef<KeyValueRef>> blockData = wait(parallelFileRestore::decodeRangeFileBlock(inFile, readOffset, readLen));
-
-	if ( debug_verbose ) {
-		printf("[VERBOSE_DEBUG] Parse range file and get mutations 3\n");
-		int tmpi = 0;
-		for (tmpi = 0; tmpi < blockData.size(); tmpi++) {
-			printf("\t[VERBOSE_DEBUG] mutation: key:%s value:%s\n", blockData[tmpi].key.toString().c_str(), blockData[tmpi].value.toString().c_str());
-		}
-	}
-
- 	// First and last key are the range for this file
- 	state KeyRange fileRange = KeyRangeRef(blockData.front().key, blockData.back().key);
- 	printf("[INFO] RangeFile:%s KeyRange:%s, restoreRange:%s\n",
- 			fileName.c_str(), fileRange.toString().c_str(), restoreRange.toString().c_str());
-
- 	// If fileRange doesn't intersect restore range then we're done.
- 	if(!fileRange.intersects(restoreRange)) {
- 		TraceEvent("ExtractApplyRangeFileToDB_MX").detail("NoIntersectRestoreRange", "FinishAndReturn");
- 		return Void();
- 	}
-
- 	// We know the file range intersects the restore range but there could still be keys outside the restore range.
- 	// Find the subvector of kv pairs that intersect the restore range.  Note that the first and last keys are just the range endpoints for this file
-	 // The blockData's first and last entries are metadata, not the real data
- 	int rangeStart = 1; //1
- 	int rangeEnd = blockData.size() -1; //blockData.size() - 1 // Q: the rangeStart and rangeEnd is [,)?
-	if ( debug_verbose ) {
-		printf("[VERBOSE_DEBUG] Range file decoded blockData\n");
-		for (auto& data : blockData ) {
-			printf("\t[VERBOSE_DEBUG] data key:%s val:%s\n", data.key.toString().c_str(), data.value.toString().c_str());
-		}
-	}
-
- 	// Slide start forward, stop if something in range is found
-	// Move rangeStart and rangeEnd until they is within restoreRange
- 	while(rangeStart < rangeEnd && !restoreRange.contains(blockData[rangeStart].key)) {
-		if ( debug_verbose ) {
-			printf("[VERBOSE_DEBUG] rangeStart:%d key:%s is not in the range:%s\n", rangeStart, blockData[rangeStart].key.toString().c_str(), restoreRange.toString().c_str());
-		}
-		++rangeStart;
-	 }
- 	// Side end backward, stop if something in range is found
- 	while(rangeEnd > rangeStart && !restoreRange.contains(blockData[rangeEnd - 1].key)) {
-		if ( debug_verbose ) {
-			printf("[VERBOSE_DEBUG] (rangeEnd:%d - 1) key:%s is not in the range:%s\n", rangeEnd, blockData[rangeStart].key.toString().c_str(), restoreRange.toString().c_str());
-		}
-		--rangeEnd;
-	 }
-
- 	// MX: now data only contains the kv mutation within restoreRange
- 	state VectorRef<KeyValueRef> data = blockData.slice(rangeStart, rangeEnd);
- 	printf("[INFO] RangeFile:%s blockData entry size:%d recovered data size:%d\n", fileName.c_str(), blockData.size(), data.size());
-
- 	// Shrink file range to be entirely within restoreRange and translate it to the new prefix
- 	// First, use the untranslated file range to create the shrunk original file range which must be used in the kv range version map for applying mutations
- 	state KeyRange originalFileRange = KeyRangeRef(std::max(fileRange.begin, restoreRange.begin), std::min(fileRange.end,   restoreRange.end));
-
- 	// Now shrink and translate fileRange
- 	Key fileEnd = std::min(fileRange.end,   restoreRange.end);
- 	if(fileEnd == (removePrefix == StringRef() ? normalKeys.end : strinc(removePrefix)) ) {
- 		fileEnd = addPrefix == StringRef() ? normalKeys.end : strinc(addPrefix);
- 	} else {
- 		fileEnd = fileEnd.removePrefix(removePrefix).withPrefix(addPrefix);
- 	}
- 	fileRange = KeyRangeRef(std::max(fileRange.begin, restoreRange.begin).removePrefix(removePrefix).withPrefix(addPrefix),fileEnd);
-
- 	state int start = 0;
- 	state int end = data.size();
- 	//state int dataSizeLimit = BUGGIFY ? g_random->randomInt(256 * 1024, 10e6) : CLIENT_KNOBS->RESTORE_WRITE_TX_SIZE;
-	state int dataSizeLimit = CLIENT_KNOBS->RESTORE_WRITE_TX_SIZE;
- 	state int kvCount = 0;
-
- 	//MX: This is where the key-value pair in range file is applied into DB
-	loop {
-
-		state int i = start;
-		state int txBytes = 0;
-		state int iend = start;
-
-		// find iend that results in the desired transaction size
-		for(; iend < end && txBytes < dataSizeLimit; ++iend) {
-			txBytes += data[iend].key.expectedSize();
-			txBytes += data[iend].value.expectedSize();
-		}
-
-
-		for(; i < iend; ++i) {
-			//MXX: print out the key value version, and operations.
-			if ( debug_verbose ) {
-				printf("RangeFile [key:%s, value:%s, version:%ld, op:set]\n", data[i].key.printable().c_str(), data[i].value.printable().c_str(), version);
-			}
-// 				TraceEvent("PrintRangeFile_MX").detail("Key", data[i].key.printable()).detail("Value", data[i].value.printable())
-// 					.detail("Version", rangeFile.version).detail("Op", "set");
-////				printf("PrintRangeFile_MX: mType:set param1:%s param2:%s param1_size:%d, param2_size:%d\n",
-////						getHexString(data[i].key.c_str(), getHexString(data[i].value).c_str(), data[i].key.size(), data[i].value.size());
-
-			//NOTE: Should NOT removePrefix and addPrefix for the backup data!
-			// In other words, the following operation is wrong:  data[i].key.removePrefix(removePrefix).withPrefix(addPrefix)
-			MutationRef m(MutationRef::Type::SetValue, data[i].key, data[i].value); //ASSUME: all operation in range file is set.
-			++kvCount;
-
-			// TODO: we can commit the kv operation into DB.
-			// Right now, we cache all kv operations into kvOps, and apply all kv operations later in one place
-			if ( rd->kvOps.find(version) == rd->kvOps.end() ) { // Create the map's key if mutation m is the first on to be inserted
-				//kvOps.insert(std::make_pair(rangeFile.version, Standalone<VectorRef<MutationRef>>(VectorRef<MutationRef>())));
-				rd->kvOps.insert(std::make_pair(version, VectorRef<MutationRef>()));
-			}
-
-			ASSERT(rd->kvOps.find(version) != rd->kvOps.end());
-			rd->kvOps[version].push_back_deep(rd->kvOps[version].arena(), m);
-
-		}
-
-		// Commit succeeded, so advance starting point
-		start = i;
-
-		if(start == end) {
-			//TraceEvent("ExtraApplyRangeFileToDB_MX").detail("Progress", "DoneApplyKVToDB");
-			printf("[INFO][Loader] NodeID:%s Parse RangeFile:%s: the number of kv operations = %d\n",
-					 rd->describeNode().c_str(), fileName.c_str(), kvCount);
-			return Void();
-		}
- 	}
-
- }
-
- ACTOR static Future<Void> _parseLogFileToMutationsOnLoader(Reference<RestoreData> rd,
- 									Reference<IBackupContainer> bc, Version version,
- 									std::string fileName, int64_t readOffset, int64_t readLen,
- 									KeyRange restoreRange, Key addPrefix, Key removePrefix,
- 									Key mutationLogPrefix) {
-
-	// Step: concatenate the backuped param1 and param2 (KV) at the same version.
- 	//state Key mutationLogPrefix = mutationLogPrefix;
- 	//TraceEvent("ReadLogFileStart").detail("LogFileName", fileName);
- 	state Reference<IAsyncFile> inFile = wait(bc->readFile(fileName));
- 	//TraceEvent("ReadLogFileFinish").detail("LogFileName", fileName);
-
- 	printf("Parse log file:%s readOffset:%d readLen:%ld\n", fileName.c_str(), readOffset, readLen);
- 	//TODO: NOTE: decodeLogFileBlock() should read block by block! based on my serial version. This applies to decode range file as well
- 	state Standalone<VectorRef<KeyValueRef>> data = wait(parallelFileRestore::decodeLogFileBlock(inFile, readOffset, readLen));
- 	//state Standalone<VectorRef<MutationRef>> data = wait(fileBackup::decodeLogFileBlock_MX(inFile, readOffset, readLen)); //Decode log file
- 	TraceEvent("ReadLogFileFinish").detail("LogFileName", fileName).detail("DecodedDataSize", data.contents().size());
- 	printf("ReadLogFile, raw data size:%d\n", data.size());
-
- 	state int start = 0;
- 	state int end = data.size();
- 	//state int dataSizeLimit = BUGGIFY ? g_random->randomInt(256 * 1024, 10e6) : CLIENT_KNOBS->RESTORE_WRITE_TX_SIZE;
-	state int dataSizeLimit = CLIENT_KNOBS->RESTORE_WRITE_TX_SIZE;
-	state int kvCount = 0;
-	state int numConcatenated = 0;
-	loop {
- 		try {
-// 			printf("Process start:%d where end=%d\n", start, end);
- 			if(start == end) {
- 				printf("ReadLogFile: finish reading the raw data and concatenating the mutation at the same version\n");
- 				break;
- 			}
-
- 			state int i = start;
- 			state int txBytes = 0;
- 			for(; i < end && txBytes < dataSizeLimit; ++i) {
- 				Key k = data[i].key.withPrefix(mutationLogPrefix);
- 				ValueRef v = data[i].value;
- 				txBytes += k.expectedSize();
- 				txBytes += v.expectedSize();
- 				//MXX: print out the key value version, and operations.
- 				//printf("LogFile [key:%s, value:%s, version:%ld, op:NoOp]\n", k.printable().c_str(), v.printable().c_str(), logFile.version);
- //				printf("LogFile [KEY:%s, VALUE:%s, VERSION:%ld, op:NoOp]\n", getHexString(k).c_str(), getHexString(v).c_str(), logFile.version);
- //				printBackupMutationRefValueHex(v, " |\t");
- //				printf("[DEBUG]||Concatenate backup mutation:fileInfo:%s, data:%d\n", logFile.toString().c_str(), i);
- 				bool concatenated = concatenateBackupMutationForLogFile(rd, data[i].value, data[i].key);
- 				numConcatenated += ( concatenated ? 1 : 0);
- //				//TODO: Decode the value to get the mutation type. Use NoOp to distinguish from range kv for now.
- //				MutationRef m(MutationRef::Type::NoOp, data[i].key, data[i].value); //ASSUME: all operation in log file is NoOp.
- //				if ( rd->kvOps.find(logFile.version) == rd->kvOps.end() ) {
- //					rd->kvOps.insert(std::make_pair(logFile.version, std::vector<MutationRef>()));
- //				} else {
- //					rd->kvOps[logFile.version].push_back(m);
- //				}
- 			}
-
- 			start = i;
-
- 		} catch(Error &e) {
- 			if(e.code() == error_code_transaction_too_large)
- 				dataSizeLimit /= 2;
- 		}
- 	}
-
- 	printf("[INFO] raw kv number:%d parsed from log file, concatenated:%d kv, num_log_versions:%d\n", data.size(), numConcatenated, rd->mutationMap.size());
-
-	return Void();
- }
-
- // Parse the kv pair (version, serialized_mutation), which are the results parsed from log file.
- void parseSerializedMutation(Reference<RestoreData> rd, bool isSampling) {
-	// Step: Parse the concatenated KV pairs into (version, <K, V, mutationType>) pair
- 	printf("[INFO] Parse the concatenated log data\n");
- 	std::string prefix = "||\t";
-	std::stringstream ss;
-	const int version_size = 12;
-	const int header_size = 12;
-	int kvCount = 0;
-
-	for ( auto& m : rd->mutationMap ) {
-		StringRef k = m.first.contents();
-		StringRefReaderMX readerVersion(k, restore_corrupted_data());
-		uint64_t commitVersion = readerVersion.consume<uint64_t>(); // Consume little Endian data
-
-
-		StringRef val = m.second.contents();
-		StringRefReaderMX reader(val, restore_corrupted_data());
-
-		int count_size = 0;
-		// Get the include version in the batch commit, which is not the commitVersion.
-		// commitVersion is in the key
-		uint64_t includeVersion = reader.consume<uint64_t>();
-		count_size += 8;
-		uint32_t val_length_decode = reader.consume<uint32_t>(); //Parse little endian value, confirmed it is correct!
-		count_size += 4;
-
-		if ( rd->kvOps.find(commitVersion) == rd->kvOps.end() ) {
-			rd->kvOps.insert(std::make_pair(commitVersion, VectorRef<MutationRef>()));
-		}
-
-		if ( debug_verbose ) {
-			printf("----------------------------------------------------------Register Backup Mutation into KVOPs version:%08lx\n", commitVersion);
-			printf("To decode value:%s\n", getHexString(val).c_str());
-		}
-		// In sampling, the last mutation vector may be not complete, we do not concatenate for performance benefit
-		if ( val_length_decode != (val.size() - 12) ) {
-			//IF we see val.size() == 10000, It means val should be concatenated! The concatenation may fail to copy the data
-			if (isSampling) {
-				printf("[PARSE WARNING]!!! val_length_decode:%d != val.size:%d version:%ld(0x%lx)\n",  val_length_decode, val.size(),
-					commitVersion, commitVersion);
-				printf("[PARSE WARNING] Skipped the mutation! OK for sampling workload but WRONG for restoring the workload\n");
-				continue;
-			} else {
-				printf("[PARSE ERROR]!!! val_length_decode:%d != val.size:%d version:%ld(0x%lx)\n",  val_length_decode, val.size(),
-					commitVersion, commitVersion);
-			}
-		} else {
-			if ( debug_verbose ) {
-				printf("[PARSE SUCCESS] val_length_decode:%d == (val.size:%d - 12)\n", val_length_decode, val.size());
-			}
-		}
-
-		// Get the mutation header
-		while (1) {
-			// stop when reach the end of the string
-			if(reader.eof() ) { //|| *reader.rptr == 0xFF
-				//printf("Finish decode the value\n");
-				break;
-			}
-
-
-			uint32_t type = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
-			uint32_t kLen = reader.consume<uint32_t>();//reader.consumeNetworkUInkvOps[t32();
-			uint32_t vLen = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
-			const uint8_t *k = reader.consume(kLen);
-			const uint8_t *v = reader.consume(vLen);
-			count_size += 4 * 3 + kLen + vLen;
-
-			MutationRef mutation((MutationRef::Type) type, KeyRef(k, kLen), KeyRef(v, vLen));
-			rd->kvOps[commitVersion].push_back_deep(rd->kvOps[commitVersion].arena(), mutation);
-			kvCount++;
-
-			if ( kLen < 0 || kLen > val.size() || vLen < 0 || vLen > val.size() ) {
-				printf("%s[PARSE ERROR]!!!! kLen:%d(0x%04x) vLen:%d(0x%04x)\n", prefix.c_str(), kLen, kLen, vLen, vLen);
-			}
-
-			if ( debug_verbose ) {
-				printf("%s---LogFile parsed mutations. Prefix:[%d]: Version:%016lx Type:%d K:%s V:%s k_size:%d v_size:%d\n", prefix.c_str(),
-					   kvCount,
-					   commitVersion, type,  getHexString(KeyRef(k, kLen)).c_str(), getHexString(KeyRef(v, vLen)).c_str(), kLen, vLen);
-			}
-
-		}
-		//	printf("----------------------------------------------------------\n");
-	}
-
-	printf("[INFO] Produces %d mutation operations from concatenated kv pairs that are parsed from log\n",  kvCount);
-
-}
-
-
-ACTOR Future<Void> setWorkerInterface(RestoreSimpleRequest req, Reference<RestoreData> rd, RestoreInterface interf, Database cx) {
- 	state Transaction tr(cx);
-
-	while (rd->isInProgress(RestoreCommandEnum::Set_WorkerInterface)) {
-		printf("[DEBUG] NODE:%s setWorkerInterface wait for 5s\n",  rd->describeNode().c_str());
-		wait(delay(5.0));
-	}
-	// Handle duplicate, assuming cmdUID is always unique for the same workload
-	if ( rd->isCmdProcessed(req.cmdID) ) {
-		printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", rd->describeNode().c_str(), req.cmdID.toString().c_str());
-		req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-		return Void();
-	} 
-
-	rd->setInProgressFlag(RestoreCommandEnum::Set_WorkerInterface);
-
-	state vector<RestoreInterface> agents; // agents is cmdsInterf
-	printf("[INFO][Worker] Node:%s Get the interface for all workers\n", rd->describeNode().c_str());
-	loop {
-		try {
-			rd->workers_interface.clear();
-			tr.reset();
-			tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
-			tr.setOption(FDBTransactionOptions::LOCK_AWARE);
-			Standalone<RangeResultRef> agentValues = wait(tr.getRange(restoreWorkersKeys, CLIENT_KNOBS->TOO_MANY));
-			ASSERT(!agentValues.more);
-			if(agentValues.size()) {
-				for(auto& it : agentValues) {
-					agents.push_back(BinaryReader::fromStringRef<RestoreInterface>(it.value, IncludeVersion()));
-					// Save the RestoreInterface for the later operations
-					rd->workers_interface.insert(std::make_pair(agents.back().id(), agents.back()));
-				}
-				tr.commit();
-				break;
-			}
-		} catch( Error &e ) {
-			printf("[WARNING] Node:%s setWorkerInterface() transaction error:%s\n", rd->describeNode().c_str(), e.what());
-			wait( tr.onError(e) );
-		}
-		printf("[WARNING] Node:%s setWorkerInterface should always succeed in the first loop! Something goes wrong!\n", rd->describeNode().c_str());
-		wait ( delay(1.0) );
-	};
-
-	req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-	rd->processedCmd[req.cmdID] = 1;
-	rd->clearInProgressFlag(RestoreCommandEnum::Set_WorkerInterface);
-
-	return Void();
- }
-
-
-ACTOR Future<Void> handleFinishRestoreReq(RestoreSimpleRequest req, Reference<RestoreData> rd, RestoreInterface interf, Database cx) {
+// Restore worker
+ACTOR Future<Void> handlerTerminateWorkerRequest(RestoreSimpleRequest req, Reference<RestoreWorkerData> self, RestoreWorkerInterface workerInterf, Database cx) {
  	state Transaction tr(cx);
 	
 	loop {
@@ -944,1176 +147,59 @@ ACTOR Future<Void> handleFinishRestoreReq(RestoreSimpleRequest req, Reference<Re
 			tr.reset();
 			tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
 			tr.setOption(FDBTransactionOptions::LOCK_AWARE);
-			tr.clear(restoreWorkerKeyFor(interf.id())); 
+			tr.clear(restoreWorkerKeyFor(workerInterf.id()));
+			if ( self->loaderInterf.present() ) {
+				tr.clear(restoreLoaderKeyFor(self->loaderInterf.get().id()));
+			}
+			if ( self->applierInterf.present() ) {
+				tr.clear(restoreApplierKeyFor(self->applierInterf.get().id()));
+			}
 			wait( tr.commit() ) ;
-			printf("Node:%s finish restore, clear the key for interf.id:%s and exit\n", rd->describeNode().c_str(),  interf.id().toString().c_str()); 
-			req.reply.send( RestoreCommonReply(interf.id(), req.cmdID) );
+			printf("Node:%s finish restore, clear the interface keys for all roles on the worker (id:%s) and the worker itself. Then exit\n", self->describeNode().c_str(),  workerInterf.id().toString().c_str()); 
+			req.reply.send( RestoreCommonReply(workerInterf.id(), req.cmdID) );
 			break;
 		} catch( Error &e ) {
-			printf("[WARNING] Node:%s finishRestoreHandler() transaction error:%s\n", rd->describeNode().c_str(), e.what());
+			printf("[WARNING] Node:%s finishRestoreHandler() transaction error:%s\n", self->describeNode().c_str(), e.what());
 			wait( tr.onError(e) );
 		}
 	};
 
-
 	return Void();
  }
 
-// Read restoreWorkersKeys from DB to get each restore worker's restore interface and set it to rd->workers_interface
- ACTOR Future<Void> collectWorkerInterface(Reference<RestoreData> rd, Database cx, int min_num_workers) {
-	state Transaction tr(cx);
-
-	state vector<RestoreInterface> agents; // agents is cmdsInterf
-	
-	loop {
-		try {
-			rd->workers_interface.clear();
-			agents.clear();
-			tr.reset();
-			tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
-			tr.setOption(FDBTransactionOptions::LOCK_AWARE);
-			Standalone<RangeResultRef> agentValues = wait(tr.getRange(restoreWorkersKeys, CLIENT_KNOBS->TOO_MANY));
-			ASSERT(!agentValues.more);
-			// If agentValues.size() < min_num_workers, we should wait for coming workers to register their interface before we read them once for all
-			if(agentValues.size() >= min_num_workers) {
-				for(auto& it : agentValues) {
-					agents.push_back(BinaryReader::fromStringRef<RestoreInterface>(it.value, IncludeVersion()));
-					// Save the RestoreInterface for the later operations
-					rd->workers_interface.insert(std::make_pair(agents.back().id(), agents.back()));
-					printf("collectWorkerInterface, interface id:%s\n", agents.back().id().toString().c_str());
-				}
-				break;
-			}
-			printf("%s:Wait for enough workers. Current num_workers:%d target num_workers:%d\n",
-					rd->describeNode().c_str(), agentValues.size(), min_num_workers);
-			wait( delay(5.0) );
-		} catch( Error &e ) {
-			printf("[WARNING]%s: collectWorkerInterface transaction error:%s\n", rd->describeNode().c_str(), e.what());
-			wait( tr.onError(e) );
-		}
-	}
-	ASSERT(agents.size() >= min_num_workers); // ASSUMPTION: We must have at least 1 loader and 1 applier
-
-	TraceEvent("FastRestore").detail("CollectWorkerInterfaceNumWorkers", rd->workers_interface.size());
-
-	return Void();
- }
-
-
 // Periodically send worker heartbeat to 
- ACTOR Future<Void> monitorWorkerLiveness(Reference<RestoreData> rd) {
-	ASSERT( !rd->workers_interface.empty() );
+ ACTOR Future<Void> monitorWorkerLiveness(Reference<RestoreWorkerData> self) {
+	ASSERT( !self->workers_workerInterface.empty() );
 	state int wIndex = 0;
-	for (auto &workerInterf : rd->workers_interface) {
-		printf("[Worker:%d][UID:%s][Interf.NodeInfo:%s]\n", wIndex, workerInterf.first.toString().c_str(), workerInterf.second.nodeID.toString().c_str());
+	for (auto &workerInterf : self->workers_workerInterface) {
+		printf("[Worker:%d][UID:%s][Interf.NodeInfo:%s]\n", wIndex, workerInterf.first.toString().c_str(), workerInterf.second.id().toString().c_str());
 		wIndex++;
 	}
 
 	state std::vector<Future<RestoreCommonReply>> cmdReplies;
-	state std::map<UID, RestoreInterface>::iterator workerInterf;
+	state std::map<UID, RestoreWorkerInterface>::iterator workerInterf;
 	loop {
 		wIndex = 0;
-		for ( workerInterf = rd->workers_interface.begin(); workerInterf !=  rd->workers_interface.end(); workerInterf++)  {
+		self->cmdID.initPhase(RestoreCommandEnum::Heart_Beat);
+		for ( workerInterf = self->workers_workerInterface.begin(); workerInterf !=  self->workers_workerInterface.end(); workerInterf++)  {
+			self->cmdID.nextCmd();
 			try {
 				wait( delay(1.0) );
-				cmdReplies.push_back( workerInterf->second.heartbeat.getReply(RestoreSimpleRequest(rd->cmdID)) );
+				cmdReplies.push_back( workerInterf->second.heartbeat.getReply(RestoreSimpleRequest(self->cmdID)) );
 				std::vector<RestoreCommonReply> reps = wait( timeoutError(getAll(cmdReplies), FastRestore_Failure_Timeout) );
 				cmdReplies.clear();
 				wIndex++;
 			} catch (Error &e) {
 				// Handle the command reply timeout error
-				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", rd->describeNode().c_str(),
-							rd->cmdID.toString().c_str(), e.code(), e.what());
-				printf("[Heartbeat: Node may be down][Worker:%d][UID:%s][Interf.NodeInfo:%s]\n", wIndex,  workerInterf->first.toString().c_str(), workerInterf->second.nodeID.toString().c_str());
+				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", self->describeNode().c_str(),
+							self->cmdID.toString().c_str(), e.code(), e.what());
+				printf("[Heartbeat: Node may be down][Worker:%d][UID:%s][Interf.NodeInfo:%s]\n", wIndex,  workerInterf->first.toString().c_str(), workerInterf->second.id().toString().c_str());
 			}
 		}
 		wait( delay(30.0) );
 	}
-
-	//return Void();
  }
 
-// Set roles (Loader or Applier) for workers and ask all workers to share their interface
-// The master node's localNodeStatus has been set outside of this function
-ACTOR Future<Void> configureRoles(Reference<RestoreData> rd)  {
-	printf("%s:Start configuring roles for workers\n", rd->describeNode().c_str());
-	// Set up the role, and the global status for each node
-	int numNodes = rd->workers_interface.size();
-	int numLoader = numNodes * ratio_loader_to_applier / (ratio_loader_to_applier + 1);
-	int numApplier = numNodes - numLoader;
-	if (numLoader <= 0 || numApplier <= 0) {
-		ASSERT( numLoader > 0 ); // Quick check in correctness
-		ASSERT( numApplier > 0 );
-		fprintf(stderr, "[ERROR] not enough nodes for loader and applier. numLoader:%d, numApplier:%d, ratio_loader_to_applier:%d, numAgents:%d\n", numLoader, numApplier, ratio_loader_to_applier, numNodes);
-	} else {
-		printf("Node%s: Configure roles numWorkders:%d numLoader:%d numApplier:%d\n", rd->describeNode().c_str(), numNodes, numLoader, numApplier);
-	}
-
-	rd->localNodeStatus.nodeIndex = 0; // Master has nodeIndex = 0
-
-	// The first numLoader nodes will be loader, and the rest nodes will be applier
-	int nodeIndex = 1; // worker's nodeIndex starts from 1
-	for (auto &workerInterf : rd->workers_interface) {
-		// globalNodeStatus does not include the master's info because master holds globalNodeStatus
-		rd->globalNodeStatus.push_back(RestoreNodeStatus());
-		rd->globalNodeStatus.back().nodeID = workerInterf.second.id();
-		rd->globalNodeStatus.back().nodeIndex = nodeIndex;
-		if ( nodeIndex < numLoader + 1) {
-			rd->globalNodeStatus.back().init(RestoreRole::Loader);
-		} else {
-			rd->globalNodeStatus.back().init(RestoreRole::Applier);
-		}
-		nodeIndex++;
-	}
-
-	// Set the last Applier as the master applier
-	rd->masterApplier = rd->globalNodeStatus.back().nodeID;
-	printf("masterApplier ID:%s\n", rd->masterApplier.toString().c_str());
-
-	// Notify each worker about the worker's role
-	state int index = 0;
-	state RestoreRole role;
-	state UID nodeID;
-	printf("Node:%s Start configuring roles for workers\n", rd->describeNode().c_str());
-	rd->cmdID.initPhase(RestoreCommandEnum::Set_Role);
-	loop {
-		try {
-			wait(delay(1.0));
-			std::vector<Future<RestoreCommonReply>> cmdReplies;
-			index = 0;
-			for (auto &workerInterf : rd->workers_interface)  {
-				role = rd->globalNodeStatus[index].role;
-				nodeID = rd->globalNodeStatus[index].nodeID;
-				rd->cmdID.nextCmd();
-				printf("[CMD:%s] Node:%s Set role (%s) to node (index=%d uid=%s)\n", rd->cmdID.toString().c_str(), rd->describeNode().c_str(),
-						getRoleStr(role).c_str(), index, nodeID.toString().c_str());
-				cmdReplies.push_back( workerInterf.second.setRole.getReply(RestoreSetRoleRequest(rd->cmdID, role, index,  rd->masterApplier)) );
-				index++;
-			}
-			std::vector<RestoreCommonReply> reps = wait( timeoutError(getAll(cmdReplies), FastRestore_Failure_Timeout) );
-			printf("[SetRole] Finished\n");
-			break;
-		} catch (Error &e) {
-			// Handle the command reply timeout error
-			fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", rd->describeNode().c_str(),
-						rd->cmdID.toString().c_str(), e.code(), e.what());
-			printf("Node:%s waits on replies time out. Current phase: Set_Role, Retry all commands.\n", rd->describeNode().c_str());
-		}
-	}
-
-	// Sanity check roles configuration
-	std::pair<int, int> numWorkers = getNumLoaderAndApplier(rd);
-	int numLoaders = numWorkers.first;
-	int numAppliers = numWorkers.second;
-	ASSERT( rd->globalNodeStatus.size() > 0 );
-	ASSERT( numLoaders > 0 );
-	ASSERT( numAppliers > 0 );
-
-	printf("Node:%s finish configure roles\n", rd->describeNode().c_str());
-
-	return Void();
-}
-
-// Ask each restore worker to share its restore interface
-ACTOR Future<Void> notifyWorkersToSetWorkersInterface(Reference<RestoreData> rd) {
-	state int index = 0;
-	loop {
-		try {
-			wait(delay(1.0));
-			index = 0;
-			std::vector<Future<RestoreCommonReply>> cmdReplies;
-			for(auto& workersInterface : rd->workers_interface) {
-				rd->cmdID.nextCmd();
-				printf("[CMD:%s] Node:%s setWorkerInterface for node (index=%d uid=%s)\n", 
-						rd->cmdID.toString().c_str(), rd->describeNode().c_str(),
-						index,  rd->globalNodeStatus[index].nodeID.toString().c_str());
-				cmdReplies.push_back( workersInterface.second.setWorkerInterface.getReply(RestoreSimpleRequest(rd->cmdID)) );
-				index++;
-			}
-			std::vector<RestoreCommonReply> reps = wait( timeoutError(getAll(cmdReplies), FastRestore_Failure_Timeout) );
-			printf("[setWorkerInterface] Finished\n");
-			break;
-		} catch (Error &e) {
-			fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", rd->describeNode().c_str(),
-					rd->cmdID.toString().c_str(), e.code(), e.what());
-			printf("Node:%s waits on replies time out. Current phase: setWorkerInterface, Retry all commands.\n", rd->describeNode().c_str());
-		}
-	}
-
-	return Void();
-}
-
-void printApplierKeyRangeInfo(std::map<UID, Standalone<KeyRangeRef>>  appliers) {
-	printf("[INFO] appliers num:%ld\n", appliers.size());
-	int index = 0;
-	for(auto &applier : appliers) {
-		printf("\t[INFO][Applier:%d] ID:%s --> KeyRange:%s\n", index, applier.first.toString().c_str(), applier.second.toString().c_str());
-	}
-}
-
-ACTOR Future<Void> assignKeyRangeToAppliers(Reference<RestoreData> rd, Database cx)  { //, VectorRef<RestoreInterface> ret_agents
-	//construct the key range for each applier
-	std::vector<KeyRef> lowerBounds;
-	std::vector<Standalone<KeyRangeRef>> keyRanges;
-	std::vector<UID> applierIDs;
-
-	// printf("[INFO] Node:%s, Assign key range to appliers. num_appliers:%ld\n", rd->describeNode().c_str(), rd->range2Applier.size());
-	for (auto& applier : rd->range2Applier) {
-		lowerBounds.push_back(applier.first);
-		applierIDs.push_back(applier.second);
-		// printf("\t[INFO] ApplierID:%s lowerBound:%s\n",
-		// 		applierIDs.back().toString().c_str(),
-		// 		lowerBounds.back().toString().c_str());
-	}
-	for (int i  = 0; i < lowerBounds.size(); ++i) {
-		KeyRef startKey = lowerBounds[i];
-		KeyRef endKey;
-		if ( i < lowerBounds.size() - 1) {
-			endKey = lowerBounds[i+1];
-		} else {
-			endKey = normalKeys.end;
-		}
-
-		if (startKey > endKey) {
-			fprintf(stderr, "ERROR at assignKeyRangeToAppliers, startKey:%s > endKey:%s\n", startKey.toString().c_str(), endKey.toString().c_str());
-		}
-
-		keyRanges.push_back(KeyRangeRef(startKey, endKey));
-	}
-
-	ASSERT( applierIDs.size() == keyRanges.size() );
-	state std::map<UID, Standalone<KeyRangeRef>> appliers;
-	appliers.clear(); // If this function is called more than once in multiple version batches, appliers may carry over the data from earlier version batch
-	for (int i = 0; i < applierIDs.size(); ++i) {
-		if (appliers.find(applierIDs[i]) != appliers.end()) {
-			printf("[ERROR] ApplierID appear more than once. appliers size:%ld applierID: %s\n",
-					appliers.size(), applierIDs[i].toString().c_str());
-			printApplierKeyRangeInfo(appliers);
-		}
-		ASSERT( appliers.find(applierIDs[i]) == appliers.end() ); // we should not have a duplicate applierID respoinsbile for multiple key ranges
-		appliers.insert(std::make_pair(applierIDs[i], keyRanges[i]));
-	}
-
-	state std::vector<Future<RestoreCommonReply>> cmdReplies;
-	loop {
-		try {
-			cmdReplies.clear();
-			rd->cmdID.initPhase(RestoreCommandEnum::Assign_Applier_KeyRange);
-			for (auto& applier : appliers) {
-				KeyRangeRef keyRange = applier.second;
-				UID nodeID = applier.first;
-				ASSERT(rd->workers_interface.find(nodeID) != rd->workers_interface.end());
-				RestoreInterface& cmdInterf = rd->workers_interface[nodeID];
-				printf("[CMD] Node:%s, Assign KeyRange:%s [begin:%s end:%s] to applier ID:%s\n", rd->describeNode().c_str(),
-						keyRange.toString().c_str(),
-						getHexString(keyRange.begin).c_str(), getHexString(keyRange.end).c_str(),
-						nodeID.toString().c_str());
-				rd->cmdID.nextCmd();
-				cmdReplies.push_back( cmdInterf.setApplierKeyRangeRequest.getReply(RestoreSetApplierKeyRangeRequest(rd->cmdID, nodeID, keyRange)) );
-
-			}
-			printf("[INFO] Wait for %ld applier to accept the cmd Assign_Applier_KeyRange\n", appliers.size());
-			std::vector<RestoreCommonReply> reps = wait( timeoutError(getAll(cmdReplies), FastRestore_Failure_Timeout) );
-			for (int i = 0; i < reps.size(); ++i) {
-				printf("[INFO] Get reply:%s for Assign_Applier_KeyRange\n",
-						reps[i].toString().c_str());
-			}
-
-			break;
-		} catch (Error &e) {
-			if (e.code() != error_code_io_timeout) {
-				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s timeout\n", rd->describeNode().c_str(), rd->cmdID.toString().c_str());
-			} else {
-				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", rd->describeNode().c_str(),
-						rd->cmdID.toString().c_str(), e.code(), e.what());
-			}
-		}
-	}
-
-	return Void();
-}
-
-// Notify loader about appliers' responsible key range
-ACTOR Future<Void> notifyAppliersKeyRangeToLoader(Reference<RestoreData> rd, Database cx)  {
-	state std::vector<UID> loaders = getLoaderIDs(rd);
-	state std::vector<Future<RestoreCommonReply>> cmdReplies;
-	state Standalone<VectorRef<UID>> appliers;
-	state Standalone<VectorRef<KeyRange>> ranges;
-
-	state std::map<Standalone<KeyRef>, UID>::iterator applierRange;
-	for (applierRange = rd->range2Applier.begin(); applierRange != rd->range2Applier.end(); applierRange++) {
-		KeyRef beginRange = applierRange->first;
-		KeyRange range(KeyRangeRef(beginRange, beginRange)); // TODO: Use the end of key range
-		appliers.push_back(appliers.arena(), applierRange->second);
-		ranges.push_back(ranges.arena(), range);
-	}
-
-	printf("Notify_Loader_ApplierKeyRange: number of appliers:%d\n", appliers.size());
-	ASSERT( appliers.size() == ranges.size() && appliers.size() != 0 );
-
-	rd->cmdID.initPhase( RestoreCommandEnum::Notify_Loader_ApplierKeyRange );
-	state UID nodeID;
-	state int i = 0;
-	for (i = 0; i < loaders.size(); ++i) {
-		nodeID = loaders[i];
-		rd->cmdID.nextCmd();
-		ASSERT(rd->workers_interface.find(nodeID) != rd->workers_interface.end());
-		loop {
-			try {
-				cmdReplies.clear();
-				RestoreInterface& cmdInterf = rd->workers_interface[nodeID];
-				printf("[CMD] Node:%s Notify node:%s about appliers key range\n", rd->describeNode().c_str(), nodeID.toString().c_str());
-				//cmdReplies.push_back( cmdInterf.setApplierKeyRangeRequest.getReply(RestoreSetApplierKeyRangeRequest(rd->cmdID, applierRange->second, range)) );
-				cmdReplies.push_back( cmdInterf.setApplierKeyRangeVectorRequest.getReply(RestoreSetApplierKeyRangeVectorRequest(rd->cmdID, appliers, ranges)) );
-				printf("[INFO] Wait for node:%s to accept the cmd Notify_Loader_ApplierKeyRange\n", nodeID.toString().c_str());
-				std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) );
-				for (int i = 0; i < reps.size(); ++i) {
-					printf("[INFO] Get reply:%s from Notify_Loader_ApplierKeyRange cmd for node.\n",
-							reps[i].toString().c_str());
-				}
-				cmdReplies.clear();
-				break;
-			} catch (Error &e) {
-				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s timeout\n", rd->describeNode().c_str(), rd->cmdID.toString().c_str());
-			}
-		}
-	}
-
-	return Void();
-}
-
-
-void printLowerBounds(std::vector<Standalone<KeyRef>> lowerBounds) {
-	if ( debug_verbose == false )
-		return;
-
-	printf("[INFO] Print out %ld keys in the lowerbounds\n", lowerBounds.size());
-	for (int i = 0; i < lowerBounds.size(); i++) {
-		printf("\t[INFO][%d] %s\n", i, getHexString(lowerBounds[i]).c_str());
-	}
-}
-
-std::vector<Standalone<KeyRef>> _calculateAppliersKeyRanges(Reference<RestoreData> rd, int numAppliers) {
-	ASSERT(numAppliers > 0);
-	std::vector<Standalone<KeyRef>> lowerBounds;
-	int numSampledMutations = 0;
-	for (auto &count : rd->keyOpsCount) {
-		numSampledMutations += count.second;
-	}
-
-	//intervalLength = (numSampledMutations - remainder) / (numApplier - 1)
-	int intervalLength = std::max(numSampledMutations / numAppliers, 1); // minimal length is 1
-	int curCount = 0;
-	int curInterval = 0;
-
-	printf("[INFO] Node:%s calculateAppliersKeyRanges(): numSampledMutations:%d numAppliers:%d intervalLength:%d\n",
-			rd->describeNode().c_str(),
-			rd->numSampledMutations, numAppliers, intervalLength);
-	for (auto &count : rd->keyOpsCount) {
-		if (curCount >= curInterval * intervalLength) {
-			printf("[INFO] Node:%s calculateAppliersKeyRanges(): Add a new key range  [%d]:%s: curCount:%d\n",
-					rd->describeNode().c_str(), curInterval, count.first.toString().c_str(), curCount);
-			lowerBounds.push_back(count.first); // The lower bound of the current key range
-			curInterval++;
-		}
-		curCount += count.second;
-	}
-
-	if ( lowerBounds.size() != numAppliers ) {
-		printf("[WARNING] calculateAppliersKeyRanges() WE MAY NOT USE ALL APPLIERS efficiently! num_keyRanges:%ld numAppliers:%d\n",
-				lowerBounds.size(), numAppliers);
-		printLowerBounds(lowerBounds);
-	}
-
-	//ASSERT(lowerBounds.size() <= numAppliers + 1); // We may have at most numAppliers + 1 key ranges
-	if ( lowerBounds.size() >= numAppliers ) {
-		printf("[WARNING] Key ranges number:%ld > numAppliers:%d. Merge the last ones\n", lowerBounds.size(), numAppliers);
-	}
-
-	while ( lowerBounds.size() >= numAppliers ) {
-		printf("[WARNING] Key ranges number:%ld > numAppliers:%d. Merge the last ones\n", lowerBounds.size(), numAppliers);
-		lowerBounds.pop_back();
-	}
-
-	return lowerBounds;
-}
-
-ACTOR Future<Standalone<VectorRef<RestoreRequest>>> collectRestoreRequests(Database cx) {
-	state int restoreId = 0;
-	state int checkNum = 0;
-	state Standalone<VectorRef<RestoreRequest>> restoreRequests;
-	state Future<Void> watch4RestoreRequest;
-
-	//wait for the restoreRequestTriggerKey to be set by the client/test workload
-	state ReadYourWritesTransaction tr2(cx);
-
-	loop {
-		try {
-			tr2.reset(); // The transaction may fail! Must full reset the transaction
-			tr2.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
-			tr2.setOption(FDBTransactionOptions::LOCK_AWARE);
-			// Assumption: restoreRequestTriggerKey has not been set
-			// Question: What if  restoreRequestTriggerKey has been set? we will stuck here?
-			// Question: Can the following code handle the situation?
-			// Note: restoreRequestTriggerKey may be set before the watch is set or may have a conflict when the client sets the same key
-			// when it happens, will we  stuck at wait on the watch?
-
-			watch4RestoreRequest = tr2.watch(restoreRequestTriggerKey);
-			wait(tr2.commit());
-			printf("[INFO][Master] Finish setting up watch for restoreRequestTriggerKey\n");
-			break;
-		} catch(Error &e) {
-			printf("[WARNING] Transaction for restore request in watch restoreRequestTriggerKey. Error:%s\n", e.name());
-			wait(tr2.onError(e));
-		}
-	};
-
-
-	loop {
-		try {
-			tr2.reset(); // The transaction may fail! Must full reset the transaction
-			tr2.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
-			tr2.setOption(FDBTransactionOptions::LOCK_AWARE);
-			// Assumption: restoreRequestTriggerKey has not been set
-			// Before we wait on the watch, we must make sure the key is not there yet!
-			//printf("[INFO][Master] Make sure restoreRequestTriggerKey does not exist before we wait on the key\n");
-			Optional<Value> triggerKey = wait( tr2.get(restoreRequestTriggerKey) );
-			if ( triggerKey.present() ) {
-				printf("!!! restoreRequestTriggerKey (and restore requests) is set before restore agent waits on the request. Restore agent can immediately proceed\n");
-				break;
-			}
-			wait(watch4RestoreRequest);
-			printf("[INFO][Master] restoreRequestTriggerKey watch is triggered\n");
-			break;
-		} catch(Error &e) {
-			printf("[WARNING] Transaction for restore request at wait on watch restoreRequestTriggerKey. Error:%s\n", e.name());
-			wait(tr2.onError(e));
-		}
-	};
-
-	loop {
-		try {
-			tr2.reset();
-			tr2.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
-			tr2.setOption(FDBTransactionOptions::LOCK_AWARE);
-
-			state Optional<Value> numRequests = wait(tr2.get(restoreRequestTriggerKey));
-			int num = decodeRestoreRequestTriggerValue(numRequests.get());
-			//TraceEvent("RestoreRequestKey").detail("NumRequests", num);
-			printf("[INFO] RestoreRequestNum:%d\n", num);
-
-			state Standalone<RangeResultRef> restoreRequestValues = wait(tr2.getRange(restoreRequestKeys, CLIENT_KNOBS->TOO_MANY));
-			printf("Restore worker get restoreRequest: %s\n", restoreRequestValues.toString().c_str());
-
-			ASSERT(!restoreRequestValues.more);
-
-			if(restoreRequestValues.size()) {
-				for ( auto &it : restoreRequestValues ) {
-					printf("Now decode restore request value...\n");
-					restoreRequests.push_back(restoreRequests.arena(), decodeRestoreRequestValue(it.value));
-				}
-			}
-			break;
-		} catch(Error &e) {
-			printf("[WARNING] Transaction error: collect restore requests. Error:%s\n", e.name());
-			wait(tr2.onError(e));
-		}
-	};
-
-	return restoreRequests;
-}
-
-void initBackupContainer(Reference<RestoreData> rd, Key url) {
-	if ( rd->bcUrl == url && rd->bc.isValid() ) {
-		return;
-	}
-	printf("initBackupContainer, url:%s\n", url.toString().c_str());
-	rd->bcUrl = url;
-	rd->bc = IBackupContainer::openContainer(url.toString());
-	//state BackupDescription desc = wait(rd->bc->describeBackup());
-	//return Void();
-}
-
-// NOTE: This function can now get the backup file descriptors
-ACTOR static Future<Void> collectBackupFiles(Reference<RestoreData> rd, Database cx, RestoreRequest request) {
-	state Key tagName = request.tagName;
-	state Key url = request.url;
-	state bool waitForComplete = request.waitForComplete;
-	state Version targetVersion = request.targetVersion;
-	state bool verbose = request.verbose;
-	state KeyRange range = request.range;
-	state Key addPrefix = request.addPrefix;
-	state Key removePrefix = request.removePrefix;
-	state bool lockDB = request.lockDB;
-	state UID randomUid = request.randomUid;
-
-	ASSERT( lockDB == true );
-
-	initBackupContainer(rd, url);
-
-	state Reference<IBackupContainer> bc = rd->bc;
-	state BackupDescription desc = wait(bc->describeBackup());
-
-	wait(desc.resolveVersionTimes(cx));
-
-	printf("[INFO] Backup Description\n%s", desc.toString().c_str());
-	printf("[INFO] Restore for url:%s, lockDB:%d\n", url.toString().c_str(), lockDB);
-	if(targetVersion == invalidVersion && desc.maxRestorableVersion.present())
-		targetVersion = desc.maxRestorableVersion.get();
-
-	printf("[INFO] collectBackupFiles: now getting backup files for restore request: %s\n", request.toString().c_str());
-	Optional<RestorableFileSet> restorable = wait(bc->getRestoreSet(targetVersion));
-
-	if(!restorable.present()) {
-		printf("[WARNING] restoreVersion:%ld (%lx) is not restorable!\n", targetVersion, targetVersion);
-		throw restore_missing_data();
-	}
-
-	if (!rd->files.empty()) {
-		printf("[WARNING] global files are not empty! files.size() is %ld. We forcely clear files\n", rd->files.size());
-		rd->files.clear();
-	}
-
-	printf("[INFO] Found backup files: num of files:%ld\n", rd->files.size());
- 	for(const RangeFile &f : restorable.get().ranges) {
- 		TraceEvent("FoundRangeFileMX").detail("FileInfo", f.toString());
- 		printf("[INFO] FoundRangeFile, fileInfo:%s\n", f.toString().c_str());
-		RestoreFileFR file(f.version, f.fileName, true, f.blockSize, f.fileSize, f.version, f.version);
- 		rd->files.push_back(file);
- 	}
- 	for(const LogFile &f : restorable.get().logs) {
- 		TraceEvent("FoundLogFileMX").detail("FileInfo", f.toString());
-		printf("[INFO] FoundLogFile, fileInfo:%s\n", f.toString().c_str());
-		RestoreFileFR file(f.beginVersion, f.fileName, false, f.blockSize, f.fileSize, f.endVersion, f.beginVersion);
-		rd->files.push_back(file);
- 	}
-
-	printf("[INFO] Restoring backup to version: %lld\n", (long long) targetVersion);
-
-	return Void();
-}
-
-// The manager that manage the control of sampling workload
-ACTOR static Future<Void> sampleWorkload(Reference<RestoreData> rd, RestoreRequest request, Reference<RestoreConfig> restoreConfig, int64_t sampleMB_input) {
-	state Key tagName = request.tagName;
-	state Key url = request.url;
-	state bool waitForComplete = request.waitForComplete;
-	state Version targetVersion = request.targetVersion;
-	state bool verbose = request.verbose;
-	state KeyRange restoreRange = request.range;
-	state Key addPrefix = request.addPrefix;
-	state Key removePrefix = request.removePrefix;
-	state bool lockDB = request.lockDB;
-	state UID randomUid = request.randomUid;
-	state Key mutationLogPrefix = restoreConfig->mutationLogPrefix();
-
-	state bool allLoadReqsSent = false;
-	state std::vector<UID> loaderIDs = getLoaderIDs(rd);
-	state std::vector<UID> applierIDs = getApplierIDs(rd);
-	state std::vector<UID> finishedLoaderIDs;
-	state int64_t sampleMB = sampleMB_input; //100;
-	state int64_t sampleB = sampleMB * 1024 * 1024; // Sample a block for every sampleB bytes. // Should adjust this value differently for simulation mode and real mode
-	state int64_t curFileIndex = 0;
-	state int64_t curFileOffset = 0;
-	state int64_t loadSizeB = 0;
-	state int64_t loadingCmdIndex = 0;
-	state int64_t sampleIndex = 0;
-	state double totalBackupSizeB = 0;
-	state double samplePercent = 0.05; // sample 1 data block per samplePercent (0.01) of data. num_sample = 1 / samplePercent
-
-	// We should sample 1% data
-	for (int i = 0; i < rd->files.size(); i++) {
-		totalBackupSizeB += rd->files[i].fileSize;
-	}
-	sampleB = std::max((int) (samplePercent * totalBackupSizeB), 10 * 1024 * 1024); // The minimal sample size is 10MB
-	printf("Node:%s totalBackupSizeB:%.1fB (%.1fMB) samplePercent:%.2f, sampleB:%ld\n", rd->describeNode().c_str(),
-			totalBackupSizeB,  totalBackupSizeB / 1024 / 1024, samplePercent, sampleB);
-
-	// Step: Distribute sampled file blocks to loaders to sample the mutations
-	rd->cmdID.initPhase(RestoreCommandEnum::Sample_Range_File);
-	curFileIndex = 0;
-	state CMDUID checkpointCMDUID = rd->cmdID;
-	state int checkpointCurFileIndex = curFileIndex;
-	state int64_t checkpointCurFileOffset = 0;
-	state std::vector<Future<RestoreCommonReply>> cmdReplies;
-	state RestoreCommandEnum cmdType;
-	loop { // For retry on timeout
-		try {
-			if ( allLoadReqsSent ) {
-				break; // All load requests have been handled
-			}
-			wait(delay(1.0));
-
-			cmdReplies.clear();
-
-			printf("[Sampling] Node:%s We will sample the workload among %ld backup files.\n", rd->describeNode().c_str(), rd->files.size());
-			printf("[Sampling] Node:%s totalBackupSizeB:%.1fB (%.1fMB) samplePercent:%.2f, sampleB:%ld, loadSize:%dB sampleIndex:%ld\n", rd->describeNode().c_str(),
-				totalBackupSizeB,  totalBackupSizeB / 1024 / 1024, samplePercent, sampleB, loadSizeB, sampleIndex);
-			for (auto &loaderID : loaderIDs) {
-				// Find the sample file
-				while ( curFileIndex < rd->files.size() && rd->files[curFileIndex].fileSize == 0 ) {
-					// NOTE: && rd->files[curFileIndex].cursor >= rd->files[curFileIndex].fileSize
-					printf("[Sampling] File %ld:%s filesize:%ld skip the file\n", curFileIndex,
-							rd->files[curFileIndex].fileName.c_str(), rd->files[curFileIndex].fileSize);
-					curFileOffset = 0;
-					curFileIndex++;
-				}
-				// Find the next sample point
-				while ( loadSizeB / sampleB < sampleIndex && curFileIndex < rd->files.size() ) {
-					if (rd->files[curFileIndex].fileSize == 0) {
-						// NOTE: && rd->files[curFileIndex].cursor >= rd->files[curFileIndex].fileSize
-						printf("[Sampling] File %ld:%s filesize:%ld skip the file\n", curFileIndex,
-								rd->files[curFileIndex].fileName.c_str(), rd->files[curFileIndex].fileSize);
-						curFileIndex++;
-						curFileOffset = 0;
-						continue;
-					}
-					if ( loadSizeB / sampleB >= sampleIndex ) {
-						break;
-					}
-					if (curFileIndex >= rd->files.size()) {
-						break;
-					}
-					loadSizeB += std::min( rd->files[curFileIndex].blockSize, std::max(rd->files[curFileIndex].fileSize - curFileOffset * rd->files[curFileIndex].blockSize, (int64_t) 0) );
-					curFileOffset++;
-					if ( rd->files[curFileIndex].blockSize == 0 || curFileOffset >= rd->files[curFileIndex].fileSize / rd->files[curFileIndex].blockSize ) {
-						curFileOffset = 0;
-						curFileIndex++;
-					}
-				}
-				if ( curFileIndex >= rd->files.size() ) {
-					allLoadReqsSent = true;
-					break;
-				}
-
-				//sampleIndex++;
-
-				// Notify loader to sample the file
-				LoadingParam param;
-				param.url = request.url;
-				param.version = rd->files[curFileIndex].version;
-				param.filename = rd->files[curFileIndex].fileName;
-				param.offset = curFileOffset * rd->files[curFileIndex].blockSize; // The file offset in bytes
-				//param.length = std::min(rd->files[curFileIndex].fileSize - rd->files[curFileIndex].cursor, loadSizeB);
-				param.length = std::min(rd->files[curFileIndex].blockSize, std::max((int64_t)0, rd->files[curFileIndex].fileSize - param.offset));
-				loadSizeB += param.length;
-				sampleIndex = std::ceil(loadSizeB / sampleB);
-				curFileOffset++;
-
-				//loadSizeB = param.length;
-				param.blockSize = rd->files[curFileIndex].blockSize;
-				param.restoreRange = restoreRange;
-				param.addPrefix = addPrefix;
-				param.removePrefix = removePrefix;
-				param.mutationLogPrefix = mutationLogPrefix;
-				if ( !(param.length > 0  &&  param.offset >= 0 && param.offset < rd->files[curFileIndex].fileSize) ) {
-					printf("[ERROR] param: length:%ld offset:%ld fileSize:%ld for %ldth file:%s\n",
-							param.length, param.offset, rd->files[curFileIndex].fileSize, curFileIndex,
-							rd->files[curFileIndex].toString().c_str());
-				}
-
-
-				printf("[Sampling][File:%ld] filename:%s offset:%ld blockSize:%ld filesize:%ld loadSize:%ldB sampleIndex:%ld\n",
-						curFileIndex, rd->files[curFileIndex].fileName.c_str(), curFileOffset,
-						rd->files[curFileIndex].blockSize, rd->files[curFileIndex].fileSize,
-						loadSizeB, sampleIndex);
-
-
-				ASSERT( param.length > 0 );
-				ASSERT( param.offset >= 0 );
-				ASSERT( param.offset <= rd->files[curFileIndex].fileSize );
-				UID nodeID = loaderID;
-
-				ASSERT(rd->workers_interface.find(nodeID) != rd->workers_interface.end());
-				RestoreInterface& cmdInterf = rd->workers_interface[nodeID];
-				printf("[Sampling][CMD] Node:%s Loading %s on node %s\n", 
-						rd->describeNode().c_str(), param.toString().c_str(), nodeID.toString().c_str());
-
-				rd->cmdID.nextCmd(); // The cmd index is the i^th file (range or log file) to be processed
-				if (!rd->files[curFileIndex].isRange) {
-					cmdType = RestoreCommandEnum::Sample_Log_File;
-					rd->cmdID.setPhase(RestoreCommandEnum::Sample_Log_File);
-					cmdReplies.push_back( cmdInterf.sampleLogFile.getReply(RestoreLoadFileRequest(rd->cmdID, param)) );
-				} else {
-					cmdType = RestoreCommandEnum::Sample_Range_File;
-					rd->cmdID.setPhase(RestoreCommandEnum::Sample_Range_File);
-					cmdReplies.push_back( cmdInterf.sampleRangeFile.getReply(RestoreLoadFileRequest(rd->cmdID, param)) );
-				}
-				
-				printf("[Sampling] Master cmdType:%d cmdUID:%s isRange:%d destinationNode:%s\n", 
-						(int) cmdType, rd->cmdID.toString().c_str(), (int) rd->files[curFileIndex].isRange,
-						nodeID.toString().c_str());
-				
-				if (param.offset + param.length >= rd->files[curFileIndex].fileSize) { // Reach the end of the file
-					curFileIndex++;
-					curFileOffset = 0;
-				}
-				if ( curFileIndex >= rd->files.size() ) {
-					allLoadReqsSent = true;
-					break;
-				}
-				++loadingCmdIndex;
-			}
-
-			printf("[Sampling] Wait for %ld loaders to accept the cmd Sample_Range_File or Sample_Log_File\n", cmdReplies.size());
-
-			if ( !cmdReplies.empty() ) {
-				//TODO: change to getAny. NOTE: need to keep the still-waiting replies
-				std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) ); 
-				//std::vector<RestoreCommonReply> reps = wait( getAll(cmdReplies) ); 
-
-				finishedLoaderIDs.clear();
-				for (int i = 0; i < reps.size(); ++i) {
-					printf("[Sampling][%d out of %d] Get reply:%s for  Sample_Range_File or Sample_Log_File\n",
-							i, reps.size(), reps[i].toString().c_str());
-					finishedLoaderIDs.push_back(reps[i].id);
-					//int64_t repLoadingCmdIndex = reps[i].cmdIndex;
-				}
-				loaderIDs = finishedLoaderIDs;
-				checkpointCMDUID = rd->cmdID;
-				checkpointCurFileIndex = curFileIndex;
-				checkpointCurFileOffset = curFileOffset;
-			}
-
-			if (allLoadReqsSent) {
-				printf("[Sampling] allLoadReqsSent, sampling finished\n");
-				break; // NOTE: need to change when change to wait on any cmdReplies
-			}
-
-		} catch (Error &e) {
-			// Handle the command reply timeout error
-			fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", rd->describeNode().c_str(),
-					rd->cmdID.toString().c_str(), e.code(), e.what());
-			rd->cmdID = checkpointCMDUID;
-			curFileIndex = checkpointCurFileIndex;
-			curFileOffset = checkpointCurFileOffset;
-			allLoadReqsSent = false;
-			printf("[Sampling][Waring] Retry at CMDID:%s curFileIndex:%ld\n", rd->cmdID.toString().c_str(), curFileIndex);
-		}
-	}
-
-	wait(delay(1.0));
-
-	// Ask master applier to calculate the key ranges for appliers
-	state int numKeyRanges = 0;
-	loop {
-		try {
-			printf("[Sampling][CMD] Ask master applier %s for the key ranges for appliers\n", rd->masterApplier.toString().c_str());
-			RestoreInterface& cmdInterf = rd->workers_interface[rd->masterApplier];
-			ASSERT(applierIDs.size() > 0);
-			rd->cmdID.initPhase(RestoreCommandEnum::Calculate_Applier_KeyRange);
-			rd->cmdID.nextCmd();
-			GetKeyRangeNumberReply rep = wait( timeoutError( 
-				cmdInterf.calculateApplierKeyRange.getReply(RestoreCalculateApplierKeyRangeRequest(rd->cmdID, applierIDs.size())),  FastRestore_Failure_Timeout) );
-			printf("[Sampling][CMDRep] number of key ranges calculated by master applier:%d\n", rep.keyRangeNum);
-			numKeyRanges = rep.keyRangeNum;
-
-			if (numKeyRanges <= 0 || numKeyRanges >= applierIDs.size() ) {
-				printf("[WARNING] Calculate_Applier_KeyRange receives wrong reply (numKeyRanges:%ld) from other phases. applierIDs.size:%d Retry Calculate_Applier_KeyRange\n", numKeyRanges, applierIDs.size());
-				continue;
-			}
-
-			if ( numKeyRanges < applierIDs.size() ) {
-				printf("[WARNING][Sampling] numKeyRanges:%d < appliers number:%ld. %ld appliers will not be used!\n",
-						numKeyRanges, applierIDs.size(), applierIDs.size() - numKeyRanges);
-			}
-
-			break;
-		} catch (Error &e) {
-			// Handle the command reply timeout error
-			fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", rd->describeNode().c_str(),
-					rd->cmdID.toString().c_str(), e.code(), e.what());
-			printf("[Sampling] [Warning] Retry on Calculate_Applier_KeyRange\n");
-		}
-	}
-
-	wait(delay(1.0));
-
-	// Ask master applier to return the key range for appliers
-	state std::vector<Future<GetKeyRangeReply>> keyRangeReplies;
-	loop {
-		try {
-			rd->range2Applier.clear();
-			keyRangeReplies.clear(); // In case error happens in try loop
-			rd->cmdID.initPhase(RestoreCommandEnum::Get_Applier_KeyRange);
-			//rd->cmdID.nextCmd();
-			for (int i = 0; i < applierIDs.size() && i < numKeyRanges; ++i) {
-				UID applierID = applierIDs[i];
-				rd->cmdID.nextCmd();
-				printf("[Sampling][Master] Node:%s, CMDID:%s Ask masterApplier:%s for the lower boundary of the key range for applier:%s\n",
-						rd->describeNode().c_str(), rd->cmdID.toString().c_str(),
-						rd->masterApplier.toString().c_str(), applierID.toString().c_str());
-				ASSERT(rd->workers_interface.find(rd->masterApplier) != rd->workers_interface.end());
-				RestoreInterface& masterApplierCmdInterf = rd->workers_interface[rd->masterApplier];
-				keyRangeReplies.push_back( masterApplierCmdInterf.getApplierKeyRangeRequest.getReply(
-					RestoreGetApplierKeyRangeRequest(rd->cmdID, i)) );
-			}
-			std::vector<GetKeyRangeReply> reps = wait( timeoutError( getAll(keyRangeReplies), FastRestore_Failure_Timeout) );
-
-			ASSERT( reps.size() <= applierIDs.size() );
-
-			// TODO: Directly use the replied lowerBound and upperBound
-			for (int i = 0; i < reps.size() && i < numKeyRanges; ++i) {
-				UID applierID = applierIDs[i];
-				Standalone<KeyRef> lowerBound = reps[i].lowerBound;
-				// if (i < numKeyRanges) {
-				// 	lowerBound = reps[i].lowerBound;
-				// } else {
-				// 	lowerBound = normalKeys.end;
-				// }
-
-				if (i == 0) {
-					lowerBound = LiteralStringRef("\x00"); // The first interval must starts with the smallest possible key
-				}
-				printf("[INFO] Node:%s Assign key-to-applier map: Key:%s -> applierID:%s\n", rd->describeNode().c_str(),
-						getHexString(lowerBound).c_str(), applierID.toString().c_str());
-				rd->range2Applier.insert(std::make_pair(lowerBound, applierID));
-			}
-
-			break;
-		} catch (Error &e) {
-			// TODO: Handle the command reply timeout error
-			fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", rd->describeNode().c_str(),
-					rd->cmdID.toString().c_str(), e.code(), e.what());
-			printf("[Sampling] [Warning] Retry on Get_Applier_KeyRange\n");
-		}
-	}
-	printf("[Sampling] rd->range2Applier has been set. Its size is:%d\n", rd->range2Applier.size());
-	printAppliersKeyRange(rd);
-
-	wait(delay(1.0));
-
-	return Void();
-
-}
-
-bool isBackupEmpty(Reference<RestoreData> rd) {
-	for (int i = 0; i < rd->files.size(); ++i) {
-		if (rd->files[i].fileSize > 0) {
-			return false;
-		}
-	}
-	return true;
-}
-
-// Distribution workload per version batch
-ACTOR static Future<Void> distributeWorkloadPerVersionBatch(RestoreInterface interf, Reference<RestoreData> rd, Database cx, RestoreRequest request, Reference<RestoreConfig> restoreConfig) {
-	state Key tagName = request.tagName;
-	state Key url = request.url;
-	state bool waitForComplete = request.waitForComplete;
-	state Version targetVersion = request.targetVersion;
-	state bool verbose = request.verbose;
-	state KeyRange restoreRange = request.range;
-	state Key addPrefix = request.addPrefix;
-	state Key removePrefix = request.removePrefix;
-	state bool lockDB = request.lockDB;
-	state UID randomUid = request.randomUid;
-	state Key mutationLogPrefix = restoreConfig->mutationLogPrefix();
-
-	if ( isBackupEmpty(rd) ) {
-		printf("[WARNING] Node:%s distributeWorkloadPerVersionBatch() load an empty batch of backup. Print out the empty backup files info.\n", rd->describeNode().c_str());
-		printBackupFilesInfo(rd);
-		return Void();
-	}
-
-	printf("[INFO] Node:%s mutationLogPrefix:%s (hex value:%s)\n", rd->describeNode().c_str(), mutationLogPrefix.toString().c_str(), getHexString(mutationLogPrefix).c_str());
-
-	// Determine the key range each applier is responsible for
-	std::pair<int, int> numWorkers = getNumLoaderAndApplier(rd);
-	int numLoaders = numWorkers.first;
-	int numAppliers = numWorkers.second;
-	ASSERT( rd->globalNodeStatus.size() > 0 );
-	ASSERT( numLoaders > 0 );
-	ASSERT( numAppliers > 0 );
-
-	state int loadingSizeMB = 0; //numLoaders * 1000; //NOTE: We want to load the entire file in the first version, so we want to make this as large as possible
-	int64_t sampleSizeMB = 0; //loadingSizeMB / 100; // Will be overwritten. The sampleSizeMB will be calculated based on the batch size
-
-	state double startTime = now();
-	state double startTimeBeforeSampling = now();
-	// TODO: WiP Sample backup files to determine the key range for appliers
-	wait( sampleWorkload(rd, request, restoreConfig, sampleSizeMB) );
-	wait( delay(1.0) );
-
-	printf("[Progress] distributeWorkloadPerVersionBatch sampling time:%.2f seconds\n", now() - startTime);
-	state double startTimeAfterSampling = now();
-
-	// Notify each applier about the key range it is responsible for, and notify appliers to be ready to receive data
-	startTime = now();
-	wait( assignKeyRangeToAppliers(rd, cx) );
-	wait( delay(1.0) );
-	printf("[Progress] distributeWorkloadPerVersionBatch assignKeyRangeToAppliers time:%.2f seconds\n", now() - startTime);
-
-	startTime = now();
-	wait( notifyAppliersKeyRangeToLoader(rd, cx) );
-	wait( delay(1.0) );
-	printf("[Progress] distributeWorkloadPerVersionBatch notifyAppliersKeyRangeToLoader time:%.2f seconds\n", now() - startTime);
-
-	// Determine which backup data block (filename, offset, and length) each loader is responsible for and
-	// Notify the loader about the data block and send the cmd to the loader to start loading the data
-	// Wait for the ack from loader and repeats
-
-	// Prepare the file's loading status
-	for (int i = 0; i < rd->files.size(); ++i) {
-		rd->files[i].cursor = 0;
-	}
-
-	// Send loading cmd to available loaders whenever loaders become available
-	// NOTE: We must split the workload in the correct boundary:
-	// For range file, it's the block boundary;
-	// For log file, it is the version boundary.
-	// This is because
-	// (1) The set of mutations at a version may be encoded in multiple KV pairs in log files.
-	// We need to concatenate the related KVs to a big KV before we can parse the value into a vector of mutations at that version
-	// (2) The backuped KV are arranged in blocks in range file.
-	// For simplicity, we distribute at the granularity of files for now.
-
-	state int loadSizeB = loadingSizeMB * 1024 * 1024;
-	state int loadingCmdIndex = 0;
-	state std::vector<UID> loaderIDs = getLoaderIDs(rd);
-	state std::vector<UID> applierIDs;
-	state std::vector<UID> finishedLoaderIDs = loaderIDs;
-
-
-	state int checkpointCurFileIndex = 0;
-	state long checkpointCurOffset = 0; 
-
-	startTime = now();
-	// We should load log file before we do range file
-	state RestoreCommandEnum phaseType = RestoreCommandEnum::Assign_Loader_Log_File;
-	state std::vector<Future<RestoreCommonReply>> cmdReplies;
-	loop {
-		state int curFileIndex = 0; // The smallest index of the files that has not been FULLY loaded
-		state long curOffset = 0;
-		state bool allLoadReqsSent = false;
-		loop {
-			try {
-				if ( allLoadReqsSent ) {
-					break; // All load requests have been handled
-				}
-				wait(delay(1.0));
-
-				cmdReplies.clear();
-				printf("[INFO] Number of backup files:%ld\n", rd->files.size());
-				rd->cmdID.initPhase(phaseType);
-				for (auto &loaderID : loaderIDs) {
-					while (  curFileIndex < rd->files.size() && rd->files[curFileIndex].fileSize == 0 ) {
-						// NOTE: && rd->files[curFileIndex].cursor >= rd->files[curFileIndex].fileSize
-						printf("[INFO] File %ld:%s filesize:%ld skip the file\n", curFileIndex,
-								rd->files[curFileIndex].fileName.c_str(), rd->files[curFileIndex].fileSize);
-						curFileIndex++;
-						curOffset = 0;
-					}
-					if ( curFileIndex >= rd->files.size() ) {
-						allLoadReqsSent = true;
-						break;
-					}
-					LoadingParam param;
-					//rd->files[curFileIndex].cursor = 0; // This is a hacky way to make sure cursor is correct in current version when we load 1 file at a time
-					param.url = request.url;
-					param.version = rd->files[curFileIndex].version;
-					param.filename = rd->files[curFileIndex].fileName;
-					param.offset = curOffset; //rd->files[curFileIndex].cursor;
-					param.length = std::min(rd->files[curFileIndex].fileSize - curOffset, rd->files[curFileIndex].blockSize);
-					//param.length = rd->files[curFileIndex].fileSize;
-					loadSizeB = param.length;
-					param.blockSize = rd->files[curFileIndex].blockSize;
-					param.restoreRange = restoreRange;
-					param.addPrefix = addPrefix;
-					param.removePrefix = removePrefix;
-					param.mutationLogPrefix = mutationLogPrefix;
-					if ( !(param.length > 0  &&  param.offset >= 0 && param.offset < rd->files[curFileIndex].fileSize) ) {
-						printf("[ERROR] param: length:%ld offset:%ld fileSize:%ld for %ldth filename:%s\n",
-								param.length, param.offset, rd->files[curFileIndex].fileSize, curFileIndex,
-								rd->files[curFileIndex].fileName.c_str());
-					}
-					ASSERT( param.length > 0 );
-					ASSERT( param.offset >= 0 );
-					ASSERT( param.offset < rd->files[curFileIndex].fileSize );
-					rd->files[curFileIndex].cursor = rd->files[curFileIndex].cursor +  param.length;
-					UID nodeID = loaderID;
-					// TODO: record the loading status
-
-					ASSERT(rd->workers_interface.find(nodeID) != rd->workers_interface.end());
-					RestoreInterface& cmdInterf = rd->workers_interface[nodeID];
-
-					RestoreCommandEnum cmdType = RestoreCommandEnum::Assign_Loader_Range_File;
-					if (rd->files[curFileIndex].isRange) {
-						cmdType = RestoreCommandEnum::Assign_Loader_Range_File;
-						rd->cmdID.setPhase(RestoreCommandEnum::Assign_Loader_Range_File);
-					} else {
-						cmdType = RestoreCommandEnum::Assign_Loader_Log_File;
-						rd->cmdID.setPhase(RestoreCommandEnum::Assign_Loader_Log_File);
-					}
-
-					if ( (phaseType == RestoreCommandEnum::Assign_Loader_Log_File && rd->files[curFileIndex].isRange) 
-						|| (phaseType == RestoreCommandEnum::Assign_Loader_Range_File && !rd->files[curFileIndex].isRange) ) {
-						rd->files[curFileIndex].cursor = 0;
-						curFileIndex++;
-						curOffset = 0;
-					} else { // load the type of file in the phaseType
-						rd->cmdID.nextCmd();
-						printf("[CMD] Loading fileIndex:%ld fileInfo:%s loadingParam:%s on node %s\n",
-							curFileIndex, rd->files[curFileIndex].toString().c_str(), 
-							param.toString().c_str(), nodeID.toString().c_str()); // VERY USEFUL INFO
-						printf("[INFO] Node:%s CMDUID:%s cmdType:%d isRange:%d loaderNode:%s\n", rd->describeNode().c_str(), rd->cmdID.toString().c_str(),
-								(int) cmdType, (int) rd->files[curFileIndex].isRange, nodeID.toString().c_str());
-						if (rd->files[curFileIndex].isRange) {
-							cmdReplies.push_back( cmdInterf.loadRangeFile.getReply(RestoreLoadFileRequest(rd->cmdID, param)) );
-						} else {
-							cmdReplies.push_back( cmdInterf.loadLogFile.getReply(RestoreLoadFileRequest(rd->cmdID, param)) );
-						}
-						curOffset += param.length;
-
-						// Reach the end of the file
-						if ( param.length + param.offset >= rd->files[curFileIndex].fileSize ) {
-							curFileIndex++;
-							curOffset = 0;
-						}
-						
-						// if (param.length <= loadSizeB) { // Reach the end of the file
-						// 	ASSERT( rd->files[curFileIndex].cursor == rd->files[curFileIndex].fileSize );
-						// 	curFileIndex++;
-						// }
-					}
-					
-					if ( curFileIndex >= rd->files.size() ) {
-						allLoadReqsSent = true;
-						break;
-					}
-					//++loadingCmdIndex; // Replaced by cmdUID
-				}
-
-				printf("[INFO] Wait for %ld loaders to accept the cmd Assign_Loader_File\n", cmdReplies.size());
-
-				// Question: How to set reps to different value based on cmdReplies.empty()?
-				if ( !cmdReplies.empty() ) {
-					std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) ); //TODO: change to getAny. NOTE: need to keep the still-waiting replies
-					//std::vector<RestoreCommonReply> reps = wait( getAll(cmdReplies) ); 
-
-					finishedLoaderIDs.clear();
-					cmdReplies.clear();
-					for (int i = 0; i < reps.size(); ++i) {
-						printf("[INFO] Get Ack reply:%s for Assign_Loader_File\n",
-								reps[i].toString().c_str());
-						finishedLoaderIDs.push_back(reps[i].id);
-						//int64_t repLoadingCmdIndex = reps[i].cmdIndex;
-					}
-					//loaderIDs = finishedLoaderIDs; // loaderIDs are also used in enumerating all loaders. The finishedLoaderIDs can be different based on the getRply results
-					checkpointCurFileIndex = curFileIndex; // Save the previous success point
-					checkpointCurOffset = curOffset;
-				}
-
-				// TODO: Let master print all nodes status. Note: We need a function to print out all nodes status
-
-				if (allLoadReqsSent) {
-					printf("[INFO] allLoadReqsSent has finished.\n");
-					break; // NOTE: need to change when change to wait on any cmdReplies
-				}
-
-			} catch (Error &e) {
-				// TODO: Handle the command reply timeout error
-				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", rd->describeNode().c_str(),
-						rd->cmdID.toString().c_str(), e.code(), e.what());
-				curFileIndex = checkpointCurFileIndex;
-				curOffset = checkpointCurOffset;
-			}
-		}
-
-		if (phaseType == RestoreCommandEnum::Assign_Loader_Log_File) {
-			phaseType = RestoreCommandEnum::Assign_Loader_Range_File;
-		} else if (phaseType == RestoreCommandEnum::Assign_Loader_Range_File) {
-			break;
-		}
-	}
-
-	wait( delay(1.0) );
-	printf("[Progress] distributeWorkloadPerVersionBatch loadFiles time:%.2f seconds\n", now() - startTime);
-
-	ASSERT( cmdReplies.empty() );
-	
-	wait( delay(5.0) );
-	// Notify the applier to applly mutation to DB
-
-	startTime = now();
-	wait( notifyApplierToApplyMutations(rd) );
-	printf("[Progress] distributeWorkloadPerVersionBatch applyToDB time:%.2f seconds\n", now() - startTime);
-
-	state double endTime = now();
-
-	double runningTime = endTime - startTimeBeforeSampling;
-	printf("[Progress] Node:%s distributeWorkloadPerVersionBatch runningTime without sampling time:%.2f seconds, with sampling time:%.2f seconds\n",
-			rd->describeNode().c_str(),
-			runningTime, endTime - startTimeAfterSampling);
-
-	return Void();
-
-}
-
-ACTOR Future<Void> notifyApplierToApplyMutations(Reference<RestoreData> rd) {
-	state std::vector<UID> appliers = getApplierIDs(rd);
-	state std::vector<Future<RestoreCommonReply>> cmdReplies;
-	loop {
-		try {
-			rd->cmdID.initPhase( RestoreCommandEnum::Apply_Mutation_To_DB );
-			for (auto& nodeID : appliers) {
-				ASSERT(rd->workers_interface.find(nodeID) != rd->workers_interface.end());
-				RestoreInterface& cmdInterf = rd->workers_interface[nodeID];
-				printf("[CMD] Node:%s Notify node:%s to apply mutations to DB\n", rd->describeNode().c_str(), nodeID.toString().c_str());
-				cmdReplies.push_back( cmdInterf.applyToDB.getReply(RestoreSimpleRequest(rd->cmdID)) );
-			}
-			printf("[INFO] Wait for %ld appliers to apply mutations to DB\n", appliers.size());
-			std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) );
-			//std::vector<RestoreCommonReply> reps = wait( getAll(cmdReplies) );
-			printf("[INFO] %ld appliers finished applying mutations to DB\n", appliers.size());
-
-			cmdReplies.clear();
-
-			wait(delay(5.0));
-
-			break;
-		} catch (Error &e) {
-			fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", rd->describeNode().c_str(),
-					rd->cmdID.toString().c_str(), e.code(), e.what());
-		}
-	}
-
-	return Void();
-}
-
-
-void sanityCheckMutationOps(Reference<RestoreData> rd) {
-	if (rd->kvOps.empty())
-		return;
-
-	if ( isKVOpsSorted(rd) ) {
- 		printf("[CORRECT] KVOps is sorted by version\n");
- 	} else {
- 		printf("[ERROR]!!! KVOps is NOT sorted by version\n");
- 	}
-
- 	if ( allOpsAreKnown(rd) ) {
- 		printf("[CORRECT] KVOps all operations are known.\n");
- 	} else {
- 		printf("[ERROR]!!! KVOps has unknown mutation op. Exit...\n");
- 	}
-}
-
-ACTOR Future<Void> sanityCheckRestoreOps(Reference<RestoreData> rd, Database cx, UID uid) {
-	sanityCheckMutationOps(rd);
-
-	state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
-	tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
-	tr->setOption(FDBTransactionOptions::LOCK_AWARE);
-
- 	printf("Now apply KVOps to DB. start...\n");
- 	tr->reset();
- 	wait(checkDatabaseLock(tr, uid));
-	wait(tr->commit());
-
-	return Void();
-
-}
 
 void initRestoreWorkerConfig() {
 	MIN_NUM_WORKERS = g_network->isSimulated() ? 3 : 120; //10; // TODO: This can become a configuration param later
@@ -2132,1958 +218,193 @@ void initRestoreWorkerConfig() {
 			MIN_NUM_WORKERS, ratio_loader_to_applier, loadBatchSizeMB, loadBatchSizeThresholdB, transactionBatchSizeThreshold);
 }
 
-ACTOR Future<Void> _restoreWorker(Database cx_input, LocalityData locality) {
-	state Database cx = cx_input;
-	state RestoreInterface interf;
-	interf.initEndpoints();
-	state Optional<RestoreInterface> leaderInterf;
-	//Global data for the worker
-	state Reference<RestoreData> rd = Reference<RestoreData>(new RestoreData());
-	rd->localNodeStatus.nodeID = interf.id();
 
-	initRestoreWorkerConfig();
+// Restore Worker
+ACTOR Future<Void> commitRestoreRoleInterfaces(Reference<RestoreWorkerData> self, Database cx) {
+	state ReadYourWritesTransaction tr(cx);
+	// For now, we assume only one role per restore worker
+	ASSERT( !(self->loaderInterf.present() && self->applierInterf.present()) );
 
-	// Compete in registering its restoreInterface as the leader.
-	state Transaction tr(cx);
 	loop {
 		try {
 			tr.reset();
 			tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
 			tr.setOption(FDBTransactionOptions::LOCK_AWARE);
-			Optional<Value> leader = wait(tr.get(restoreLeaderKey));
-			if(leader.present()) {
-				leaderInterf = BinaryReader::fromStringRef<RestoreInterface>(leader.get(), IncludeVersion());
-				// NOTE: Handle the situation that the leader's commit of its key causes error(commit_unknown_result)
-				// In this situation, the leader will try to register its key again, which will never succeed.
-				// We should let leader escape from the infinite loop
-				if ( leaderInterf.get().id() == interf.id() ) {
-					printf("[Worker] NodeID:%s is the leader and has registered its key in commit_unknown_result error. Let it set the key again\n",
-							leaderInterf.get().id().toString().c_str());
-					tr.set(restoreLeaderKey, BinaryWriter::toValue(interf, IncludeVersion()));
-					wait(tr.commit());
-					 // reset leaderInterf to invalid for the leader process
-					 // because a process will not execute leader's logic unless leaderInterf is invalid
-					leaderInterf = Optional<RestoreInterface>();
-					break;
-				}
-				printf("[Worker] Leader key exists:%s. Worker registers its restore interface id:%s\n",
-						leaderInterf.get().id().toString().c_str(), interf.id().toString().c_str());
-				tr.set(restoreWorkerKeyFor(interf.id()), restoreCommandInterfaceValue(interf));
-				wait(tr.commit());
-				break;
+			if ( self->loaderInterf.present() ) {
+				tr.set( restoreLoaderKeyFor(self->loaderInterf.get().id()), restoreLoaderInterfaceValue(self->loaderInterf.get()) );
 			}
-			printf("[Worker] NodeID:%s tries to register its interface as leader\n", interf.id().toString().c_str());
-			tr.set(restoreLeaderKey, BinaryWriter::toValue(interf, IncludeVersion()));
-			wait(tr.commit());
+			if ( self->applierInterf.present() ) {
+				tr.set( restoreApplierKeyFor(self->applierInterf.get().id()), restoreApplierInterfaceValue(self->applierInterf.get()) );
+			}
+			wait (tr.commit() );
 			break;
 		} catch( Error &e ) {
-			// ATTENTION: We may have error commit_unknown_result, the commit may or may not succeed!
-			// We must handle this error, otherwise, if the leader does not know its key has been registered, the leader will stuck here!
-			printf("[INFO] NodeID:%s restoreWorker select leader error, error code:%d error info:%s\n",
-					interf.id().toString().c_str(), e.code(), e.what());
+			printf("[WARNING]%s: commitRestoreRoleInterfaces transaction error:%s\n", self->describeNode().c_str(), e.what());
 			wait( tr.onError(e) );
 		}
 	}
 
-	//we are not the leader, so put our interface in the agent list
-	if(leaderInterf.present()) {
-		// Initialize the node's UID
-		//rd->localNodeStatus.nodeID = interf.id();
-		wait( workerCore(rd, interf, cx) );
+	return Void();
+} 
+
+// Restore Worker
+ACTOR Future<Void> handleRecruitRoleRequest(RestoreRecruitRoleRequest req, Reference<RestoreWorkerData> self, ActorCollection *actors, Database cx) {
+	printf("[INFO][Worker] Node:%s get role %s\n", self->describeNode().c_str(),
+			getRoleStr(req.role).c_str());
+
+	if (req.role == RestoreRole::Loader) {
+		ASSERT( !self->loaderInterf.present() );
+		self->loaderData = Reference<RestoreLoaderData>(new RestoreLoaderData());
+		self->loaderInterf = RestoreLoaderInterface();
+		actors->add( restoreLoaderCore(self->loaderData, self->loaderInterf.get(), cx) );
+	} else if (req.role == RestoreRole::Applier) {
+		ASSERT( !self->applierInterf.present() );
+		self->applierData = Reference<RestoreApplierData>( new RestoreApplierData() );
+		self->applierInterf = RestoreApplierInterface();
+		actors->add( restoreApplierCore(self->applierData, self->applierInterf.get(), cx) );
 	} else {
-		wait( masterCore(rd, interf, cx)  );
+		TraceEvent(SevError, "FastRestore").detail("HandleRecruitRoleRequest", "UnknownRole"); //.detail("Request", req.printable());
 	}
 
+	wait( commitRestoreRoleInterfaces(self, cx) ); // Commit the interface after the interface is ready to accept requests
+	req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+
 	return Void();
 }
 
-ACTOR Future<Void> restoreWorker(Reference<ClusterConnectionFile> ccf, LocalityData locality) {
-	Database cx = Database::createDatabase(ccf->getFilename(), Database::API_VERSION_LATEST,locality);
-	wait(_restoreWorker(cx, locality));
-	return Void();
-}
 
-// ToDelete: If we can pass the correctness test
-ACTOR static Future<Void> finishRestore(Reference<RestoreData> rd, Database cx, Standalone<VectorRef<RestoreRequest>> restoreRequests) {
-	// Make restore workers quit
-	state std::vector<UID> workersIDs = getWorkerIDs(rd); // All workers ID
-	state std::vector<Future<RestoreCommonReply>> cmdReplies;
-	state std::map<UID, RestoreInterface>::iterator workerInterf;
-	printGlobalNodeStatus(rd);
+// Read restoreWorkersKeys from DB to get each restore worker's restore workerInterface and set it to self->workers_workerInterface
+// This is done before we assign restore roles for restore workers
+ ACTOR Future<Void> collectRestoreWorkerInterface(Reference<RestoreWorkerData> self, Database cx, int min_num_workers) {
+	state Transaction tr(cx);
+
+	state vector<RestoreWorkerInterface> agents; // agents is cmdsInterf
+	
 	loop {
 		try {
-			cmdReplies.clear();
-			rd->cmdID.initPhase(RestoreCommandEnum::Finish_Restore);
-			
-			for ( workerInterf = rd->workers_interface.begin(); workerInterf != rd->workers_interface.end(); workerInterf++ ) {
-				if ( std::find(workersIDs.begin(), workersIDs.end(), workerInterf->first) == workersIDs.end() ) {
-					continue; // The workerInterf is not discovered at configureRoles and therefore not involve in restore
+			self->workers_workerInterface.clear();
+			agents.clear();
+			tr.reset();
+			tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
+			tr.setOption(FDBTransactionOptions::LOCK_AWARE);
+			Standalone<RangeResultRef> agentValues = wait(tr.getRange(restoreWorkersKeys, CLIENT_KNOBS->TOO_MANY));
+			ASSERT(!agentValues.more);
+			// If agentValues.size() < min_num_workers, we should wait for coming workers to register their workerInterface before we read them once for all
+			if(agentValues.size() >= min_num_workers) {
+				for(auto& it : agentValues) {
+					agents.push_back(BinaryReader::fromStringRef<RestoreWorkerInterface>(it.value, IncludeVersion()));
+					// Save the RestoreWorkerInterface for the later operations
+					self->workers_workerInterface.insert(std::make_pair(agents.back().id(), agents.back()));
+					printf("collectWorkerInterface, workerInterface id:%s\n", agents.back().id().toString().c_str());
 				}
-				rd->cmdID.nextCmd();
-				RestoreInterface &interf = workerInterf->second;
-				cmdReplies.push_back(interf.finishRestore.getReply(RestoreSimpleRequest(rd->cmdID)));
-			}
-
-			if (!cmdReplies.empty()) {
-				std::vector<RestoreCommonReply> reps =  wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout / 100 ) );
-				//std::vector<RestoreCommonReply> reps =  wait( getAll(cmdReplies) );
-				cmdReplies.clear();
-			}
-			printf("All restore workers have quited\n");
-
-			break;
-		} catch(Error &e) {
-			printf("[ERROR] At sending finishRestore request. error code:%d message:%s. Retry...\n", e.code(), e.what());
-			rd->workers_interface.clear();
-			cmdReplies.clear();
-			wait( collectWorkerInterface(rd, cx, 0) );
-		}
-	}
-
-	// Notify tester that the restore has finished
-	state ReadYourWritesTransaction tr3(cx);
-	loop {
-		try {
-			tr3.reset();
-			tr3.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
-			tr3.setOption(FDBTransactionOptions::LOCK_AWARE);
-			tr3.clear(restoreRequestTriggerKey);
-			tr3.clear(restoreRequestKeys);
-			tr3.set(restoreRequestDoneKey, restoreRequestDoneValue(restoreRequests.size()));
-			wait(tr3.commit());
-			TraceEvent("LeaderFinishRestoreRequest");
-			printf("[INFO] RestoreLeader write restoreRequestDoneKey\n");
-
-			break;
-		}  catch( Error &e ) {
-			TraceEvent("RestoreAgentLeaderErrorTr3").detail("ErrorCode", e.code()).detail("ErrorName", e.name());
-			printf("[Error] RestoreLead operation on restoreRequestDoneKey, error:%s\n", e.what());
-			wait( tr3.onError(e) );
-		}
-	};
-
-
- 	// TODO:  Validate that the range version map has exactly the restored ranges in it.  This means that for any restore operation
- 	// the ranges to restore must be within the backed up ranges, otherwise from the restore perspective it will appear that some
- 	// key ranges were missing and so the backup set is incomplete and the restore has failed.
- 	// This validation cannot be done currently because Restore only supports a single restore range but backups can have many ranges.
-
- 	// Clear the applyMutations stuff, including any unapplied mutations from versions beyond the restored version.
- 	//	restore.clearApplyMutationsKeys(tr);
-
-	printf("[INFO] Notify the end of the restore\n");
-	TraceEvent("NotifyRestoreFinished");
-
-	return Void();
-}
-
-////--- Restore functions
-ACTOR static Future<Void> unlockDB(Database cx, UID uid) {
-	state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
-	 loop {
-		try {
-			tr->reset();
-			tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
-			tr->setOption(FDBTransactionOptions::LOCK_AWARE);
-			printf("CheckDBlock:%s START\n", uid.toString().c_str());
-			wait(checkDatabaseLock(tr, uid));
-			printf("CheckDBlock:%s DONE\n", uid.toString().c_str());
-
-			printf("UnlockDB now. Start.\n");
-			wait(unlockDatabase(tr, uid)); //NOTE: unlockDatabase didn't commit inside the function!
-
-			printf("CheckDBlock:%s START\n", uid.toString().c_str());
-			wait(checkDatabaseLock(tr, uid));
-			printf("CheckDBlock:%s DONE\n", uid.toString().c_str());
-
-			printf("UnlockDB now. Commit.\n");
-			wait( tr->commit() );
-
-			printf("UnlockDB now. Done.\n");
-			break;
-		} catch( Error &e ) {
-			printf("Error when we unlockDB. Error:%s\n", e.what());
-			wait(tr->onError(e));
-		}
-	 };
-
- 	return Void();
- }
-
- struct FastRestoreStatus {
-	double curWorkloadSize;
-	double curRunningTime;
-	double curSpeed;
-
-	double totalWorkloadSize;
-	double totalRunningTime;
-	double totalSpeed;
-};
-
-int restoreStatusIndex = 0;
-ACTOR static Future<Void> registerStatus(Database cx, struct FastRestoreStatus status) {
- 	state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
-	loop {
-		try {
-			printf("[Restore_Status][%d] curWorkload:%.2f curRunningtime:%.2f curSpeed:%.2f totalWorkload:%.2f totalRunningTime:%.2f totalSpeed:%.2f\n",
-					restoreStatusIndex, status.curWorkloadSize, status.curRunningTime, status.curSpeed, status.totalWorkloadSize, status.totalRunningTime, status.totalSpeed);
-
-			tr->reset();
-			tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
-			tr->setOption(FDBTransactionOptions::LOCK_AWARE);
-
-			tr->set(restoreStatusKeyFor(StringRef(std::string("curWorkload") + std::to_string(restoreStatusIndex))), restoreStatusValue(status.curWorkloadSize));
-			tr->set(restoreStatusKeyFor(StringRef(std::string("curRunningTime") + std::to_string(restoreStatusIndex))), restoreStatusValue(status.curRunningTime));
-			tr->set(restoreStatusKeyFor(StringRef(std::string("curSpeed") + std::to_string(restoreStatusIndex))), restoreStatusValue(status.curSpeed));
-
-			tr->set(restoreStatusKeyFor(StringRef(std::string("totalWorkload"))), restoreStatusValue(status.totalWorkloadSize));
-			tr->set(restoreStatusKeyFor(StringRef(std::string("totalRunningTime"))), restoreStatusValue(status.totalRunningTime));
-			tr->set(restoreStatusKeyFor(StringRef(std::string("totalSpeed"))), restoreStatusValue(status.totalSpeed));
-
-			wait( tr->commit() );
-			restoreStatusIndex++;
-
-			break;
-		} catch( Error &e ) {
-			printf("Transaction Error when we registerStatus. Error:%s\n", e.what());
-			wait(tr->onError(e));
-		}
-	 };
-
-	return Void();
-}
-
-
-ACTOR static Future<Void> _lockDB(Database cx, UID uid, bool lockDB) {
-	printf("[Lock] DB will be locked, uid:%s, lockDB:%d\n", uid.toString().c_str(), lockDB);
-	
-	ASSERT( lockDB );
-
-	loop {
-		try {
-			wait(lockDatabase(cx, uid));
-			break;
-		} catch( Error &e ) {
-			printf("Transaction Error when we lockDB. Error:%s\n", e.what());
-			wait(tr->onError(e));
-		}
-	}
-
-	state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
-	loop {
-		try {
-			tr->reset();
-			tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
-			tr->setOption(FDBTransactionOptions::LOCK_AWARE);
-
-			wait(checkDatabaseLock(tr, uid));
-
-			tr->commit();
-			break;
-		} catch( Error &e ) {
-			printf("Transaction Error when we lockDB. Error:%s\n", e.what());
-			wait(tr->onError(e));
-		}
-	}
-
-
-	return Void();
-}
-
-ACTOR static Future<Void> _clearDB(Reference<ReadYourWritesTransaction> tr) {
-	loop {
-		try {
-			tr->reset();
-			tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
-			tr->setOption(FDBTransactionOptions::LOCK_AWARE);
-			tr->clear(normalKeys);
-			tr->commit();
-			break;
-		} catch(Error &e) {
-			printf("Retry at clean up DB before restore. error code:%d message:%s. Retry...\n", e.code(), e.what());
-			if(e.code() != error_code_restore_duplicate_tag) {
-				wait(tr->onError(e));
-			}
-		}
-	}
-
-	return Void();
-}
-
-ACTOR Future<Void> initializeVersionBatch(Reference<RestoreData> rd, int batchIndex) {
-	rd->batchIndex = batchIndex;
-	state std::vector<UID> workerIDs = getWorkerIDs(rd);
-	state int index = 0;
-	loop {
-		try {
-			wait(delay(1.0));
-			std::vector<Future<RestoreCommonReply>> cmdReplies;
-			rd->cmdID.initPhase(RestoreCommandEnum::RESET_VersionBatch);
-			for(auto& workerID : workerIDs) {
-				ASSERT( rd->workers_interface.find(workerID) != rd->workers_interface.end() );
-				auto& cmdInterf = rd->workers_interface[workerID];
-				RestoreRole role = rd->globalNodeStatus[index].role;
-				UID nodeID = rd->globalNodeStatus[index].nodeID;
-				rd->cmdID.nextCmd();
-				printf("[CMD:%s] Node:%s Initialize version batch %d\n", rd->cmdID.toString().c_str(), rd->describeNode().c_str(),
-						batchIndex);
-				cmdReplies.push_back( cmdInterf.initVersionBatch.getReply(RestoreVersionBatchRequest(rd->cmdID, batchIndex)) );
-				index++;
-			}
-			std::vector<RestoreCommonReply> reps = wait( timeoutError(getAll(cmdReplies), FastRestore_Failure_Timeout) );
-			printf("Initilaize Version Batch done\n");
-
-			break;
-		} catch (Error &e) {
-			// TODO: Handle the command reply timeout error
-			if (e.code() != error_code_io_timeout) {
-				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s timeout\n", rd->describeNode().c_str(), rd->cmdID.toString().c_str());
-			} else {
-				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", rd->describeNode().c_str(),
-						rd->cmdID.toString().c_str(), e.code(), e.what());
-			}
-
-			printf("Node:%s waits on replies time out. Current phase: Set_Role, Retry all commands.\n", rd->describeNode().c_str());
-		}
-	}
-
-	return Void();
-}
-
-// Collect the set of backup files to be used for a version batch
-// Return true if there is still files to be restored; false otherwise.
-// This function will change the process' RestoreData
-bool collectFilesForOneVersionBatch(Reference<RestoreData> rd) {
-	rd->files.clear();
-	rd->curWorkloadSize = 0;
- 	Version endVersion = -1;
-	bool isRange = false;
-	bool validVersion = false;
-	// Step: Find backup files in each version batch and restore them.
-	while ( rd->curBackupFilesBeginIndex < rd->allFiles.size() ) {
-		// Find the curBackupFilesEndIndex, such that the to-be-loaded files size (curWorkloadSize) is as close to loadBatchSizeThresholdB as possible,
-		// and curBackupFilesEndIndex must not belong to the forbidden version range!
-		if ( rd->curBackupFilesEndIndex < rd->allFiles.size() ) {
-			endVersion =  rd->allFiles[rd->curBackupFilesEndIndex].endVersion;
-			isRange = rd->allFiles[rd->curBackupFilesEndIndex].isRange;
-			validVersion = !isVersionInForbiddenRange(rd, endVersion, isRange);
-			rd->curWorkloadSize  += rd->allFiles[rd->curBackupFilesEndIndex].fileSize;
-			printf("[DEBUG][Batch:%d] Calculate backup files for a version batch: endVersion:%lld isRange:%d validVersion:%d curWorkloadSize:%.2fB curBackupFilesBeginIndex:%ld curBackupFilesEndIndex:%ld, files.size:%ld\n",
-				rd->batchIndex, (long long) endVersion, isRange, validVersion, rd->curWorkloadSize , rd->curBackupFilesBeginIndex, rd->curBackupFilesEndIndex, rd->allFiles.size());
-		}
-		if ( (validVersion && rd->curWorkloadSize  >= loadBatchSizeThresholdB) || rd->curBackupFilesEndIndex >= rd->allFiles.size() )  {
-			if ( rd->curBackupFilesEndIndex >= rd->allFiles.size() && rd->curWorkloadSize <= 0 ) {
-				printf("Restore finishes: curBackupFilesEndIndex:%ld, allFiles.size:%ld, curWorkloadSize:%.2f\n",
-						rd->curBackupFilesEndIndex, rd->allFiles.size(), rd->curWorkloadSize );
-				//break; // return result
-			}
-			// Construct the files [curBackupFilesBeginIndex, curBackupFilesEndIndex]
-			//rd->resetPerVersionBatch();
-			//rd->cmdID.setBatch(rd->batchIndex);
-			if ( rd->curBackupFilesBeginIndex < rd->allFiles.size()) {
-				for (int fileIndex = rd->curBackupFilesBeginIndex; fileIndex <= rd->curBackupFilesEndIndex && fileIndex < rd->allFiles.size(); fileIndex++) {
-					rd->files.push_back(rd->allFiles[fileIndex]);
-				}
-			}
-			printBackupFilesInfo(rd);
-			rd->totalWorkloadSize += rd->curWorkloadSize;
-			break;
-		} else if (validVersion && rd->curWorkloadSize < loadBatchSizeThresholdB) {
-			rd->curBackupFilesEndIndex++;
-		} else if (!validVersion && rd->curWorkloadSize < loadBatchSizeThresholdB) {
-			rd->curBackupFilesEndIndex++;
-		} else if (!validVersion && rd->curWorkloadSize >= loadBatchSizeThresholdB) {
-			// Now: just move to the next file. We will eventually find a valid version but load more than loadBatchSizeThresholdB
-			printf("[WARNING] The loading batch size will be larger than expected! curBatchSize:%.2fB, expectedBatchSize:%2.fB, endVersion:%ld\n",
-					rd->curWorkloadSize, loadBatchSizeThresholdB, endVersion);
-			rd->curBackupFilesEndIndex++;
-			// TODO: Roll back to find a valid version
-		}
-	}
-
-	return (rd->files.size() > 0);
-}
-
-ACTOR static Future<Version> processRestoreRequest(RestoreInterface interf, Reference<RestoreData> rd, Database cx, RestoreRequest request) {
-	state Key tagName = request.tagName;
-	state Key url = request.url;
-	state bool waitForComplete = request.waitForComplete;
-	state Version targetVersion = request.targetVersion;
-	state bool verbose = request.verbose;
-	state KeyRange range = request.range;
-	state Key addPrefix = request.addPrefix;
-	state Key removePrefix = request.removePrefix;
-	state bool lockDB = request.lockDB;
-	state UID randomUid = request.randomUid;
-
-	//MX: Lock DB if it is not locked
-	printf("RestoreRequest lockDB:%d\n", lockDB);
-	if ( lockDB == false ) {
-		printf("[WARNING] RestoreRequest lockDB:%d; we will overwrite request.lockDB to true and forcely lock db\n", lockDB);
-		lockDB = true;
-		request.lockDB = true;
-	}
-
-	state long curBackupFilesBeginIndex = 0;
-	state long curBackupFilesEndIndex = 0;
-
-	state double totalWorkloadSize = 0;
-	state double totalRunningTime = 0; // seconds
-	state double curRunningTime = 0; // seconds
-	state double curStartTime = 0;
-	state double curEndTime = 0;
-	state double curWorkloadSize = 0; //Bytes
-
-	
-	state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
-	state Reference<RestoreConfig> restoreConfig(new RestoreConfig(randomUid));
-
-	// lock DB for restore
-	wait( _lockDB(cx, randomUid, lockDB) );
-	wait( _clearDB(tr) );
-
-	// Step: Collect all backup files
-	printf("===========Restore request start!===========\n");
-	state double startTime = now();
-	wait( collectBackupFiles(rd, cx, request) );
-	printf("[Perf] Node:%s collectBackupFiles takes %.2f seconds\n", rd->describeNode().c_str(), now() - startTime);
-	constructFilesWithVersionRange(rd);
-	rd->files.clear(); // Ensure no mistakely use rd->files
-	
-	// Sort the backup files based on end version.
-	sort(rd->allFiles.begin(), rd->allFiles.end());
-	printAllBackupFilesInfo(rd);
-
-	buildForbiddenVersionRange(rd);
-	printForbiddenVersionRange(rd);
-	if ( isForbiddenVersionRangeOverlapped(rd) ) {
-		fprintf(stderr, "[ERROR] forbidden version ranges are overlapped! Check out the forbidden version range above\n");
-	}
-
-	rd->batchIndex = 0;
-	state int prevBatchIndex = 0;
-	state long prevCurBackupFilesBeginIndex = 0;
-	state long prevCurBackupFilesEndIndex = 0;
-	state double prevCurWorkloadSize = 0;
-	state double prevtotalWorkloadSize = 0;
-
-	loop {
-		try {
-			curStartTime = now();
-			rd->files.clear();
-			rd->resetPerVersionBatch();
-			rd->cmdID.setBatch(rd->batchIndex);
-			// Checkpoint the progress of the previous version batch
-			prevBatchIndex = rd->batchIndex;
-			prevCurBackupFilesBeginIndex = rd->curBackupFilesBeginIndex;
-			prevCurBackupFilesEndIndex = rd->curBackupFilesEndIndex;
-			prevCurWorkloadSize = rd->curWorkloadSize;
-			prevtotalWorkloadSize = rd->totalWorkloadSize;
-			
-			bool hasBackupFilesToProcess = collectFilesForOneVersionBatch(rd);
-			if ( !hasBackupFilesToProcess ) { // No more backup files to restore
-				printf("No backup files to process any more\n");
 				break;
 			}
-
-			printf("[Progress][Start version batch] Node:%s, restoreBatchIndex:%d, curWorkloadSize:%.2f------\n", rd->describeNode().c_str(), rd->batchIndex, rd->curWorkloadSize);
-			wait( initializeVersionBatch(rd, rd->batchIndex) );
-
-			wait( delay(1.0) );
-
-			wait( distributeWorkloadPerVersionBatch(interf, rd, cx, request, restoreConfig) );
-
-			curEndTime = now();
-			curRunningTime = curEndTime - curStartTime;
-			ASSERT(curRunningTime >= 0);
-			totalRunningTime += curRunningTime;
-
-			struct FastRestoreStatus status;
-			status.curRunningTime = curRunningTime;
-			status.curWorkloadSize = rd->curWorkloadSize;
-			status.curSpeed = rd->curWorkloadSize /  curRunningTime;
-			status.totalRunningTime = totalRunningTime;
-			status.totalWorkloadSize = rd->totalWorkloadSize;
-			status.totalSpeed = rd->totalWorkloadSize / totalRunningTime;
-
-			printf("[Progress][Finish version batch] restoreBatchIndex:%d, curWorkloadSize:%.2f B, curWorkload:%.2f B curRunningtime:%.2f s curSpeed:%.2f B/s  totalWorkload:%.2f B totalRunningTime:%.2f s totalSpeed:%.2f B/s\n",
-					rd->batchIndex, rd->curWorkloadSize,
-					status.curWorkloadSize, status.curRunningTime, status.curSpeed, status.totalWorkloadSize, status.totalRunningTime, status.totalSpeed);
-
-			wait( registerStatus(cx, status) );
-			printf("[Progress] Finish 1 version batch. curBackupFilesBeginIndex:%ld curBackupFilesEndIndex:%ld allFiles.size():%ld",
-				rd->curBackupFilesBeginIndex, rd->curBackupFilesEndIndex, rd->allFiles.size());
-
-			rd->curBackupFilesBeginIndex = rd->curBackupFilesEndIndex + 1;
-			rd->curBackupFilesEndIndex++;
-			rd->curWorkloadSize = 0;
-			rd->batchIndex++;
-
-		} catch(Error &e) {
-			fprintf(stdout, "!!![MAY HAVE BUG] Reset the version batch state to the start of the current version batch, due to error:%s\n", e.what());
-			if(e.code() != error_code_restore_duplicate_tag) {
-				wait(tr->onError(e));
-			}
-			rd->batchIndex = prevBatchIndex;
-			rd->curBackupFilesBeginIndex = prevCurBackupFilesBeginIndex;
-			rd->curBackupFilesEndIndex = prevCurBackupFilesEndIndex;
-			rd->curWorkloadSize = prevCurWorkloadSize;
-			rd->totalWorkloadSize = prevtotalWorkloadSize;
+			printf("%s:Wait for enough workers. Current num_workers:%d target num_workers:%d\n",
+					self->describeNode().c_str(), agentValues.size(), min_num_workers);
+			wait( delay(5.0) );
+		} catch( Error &e ) {
+			printf("[WARNING]%s: collectWorkerInterface transaction error:%s\n", self->describeNode().c_str(), e.what());
+			wait( tr.onError(e) );
 		}
 	}
+	ASSERT(agents.size() >= min_num_workers); // ASSUMPTION: We must have at least 1 loader and 1 applier
 
-	// Unlock DB  at the end of handling the restore request
-	
-	wait( unlockDB(cx, randomUid) );
-	printf("Finish restore uid:%s \n", randomUid.toString().c_str());
+	TraceEvent("FastRestore").detail("CollectWorkerInterfaceNumWorkers", self->workers_workerInterface.size());
 
-	return targetVersion;
-}
-
-//-------Helper functions
-std::string getHexString(StringRef input) {
-	std::stringstream ss;
-	for (int i = 0; i<input.size(); i++) {
-		if ( i % 4 == 0 )
-			ss << " ";
-		if ( i == 12 ) { //The end of 12bytes, which is the version size for value
-			ss << "|";
-		}
-		if ( i == (12 + 12) ) { //The end of version + header
-			ss << "@";
-		}
-		ss << std::setfill('0') << std::setw(2) << std::hex << (int) input[i]; // [] operator moves the pointer in step of unit8
-	}
-	return ss.str();
-}
-
-std::string getHexKey(StringRef input, int skip) {
-	std::stringstream ss;
-	for (int i = 0; i<skip; i++) {
-		if ( i % 4 == 0 )
-			ss << " ";
-		ss << std::setfill('0') << std::setw(2) << std::hex << (int) input[i]; // [] operator moves the pointer in step of unit8
-	}
-	ss << "||";
-
-	//hashvalue
-	ss << std::setfill('0') << std::setw(2) << std::hex << (int) input[skip]; // [] operator moves the pointer in step of unit8
-	ss << "|";
-
-	// commitversion in 64bit
-	int count = 0;
-	for (int i = skip+1; i<input.size() && i < skip+1+8; i++) {
-		if ( count++ % 4 == 0 )
-			ss << " ";
-		ss << std::setfill('0') << std::setw(2) << std::hex << (int) input[i]; // [] operator moves the pointer in step of unit8
-	}
-	// part value
-	count = 0;
-	for (int i = skip+1+8; i<input.size(); i++) {
-		if ( count++ % 4 == 0 )
-			ss << " ";
-		ss << std::setfill('0') << std::setw(2) << std::hex << (int) input[i]; // [] operator moves the pointer in step of unit8
-	}
-	return ss.str();
-}
+	return Void();
+ }
 
 
-void printMutationListRefHex(MutationListRef m, std::string prefix) {
-	MutationListRef::Iterator iter = m.begin();
-	for ( ;iter != m.end(); ++iter) {
-		printf("%s mType:%04x param1:%s param2:%s param1_size:%d, param2_size:%d\n", prefix.c_str(), iter->type,
-			   getHexString(iter->param1).c_str(), getHexString(iter->param2).c_str(), iter->param1.size(), iter->param2.size());
-	}
-}
+// RestoreWorker that has restore master role: Recruite a role for each worker
+ACTOR Future<Void> recruitRestoreRoles(Reference<RestoreWorkerData> self)  {
+	printf("%s:Start configuring roles for workers\n", self->describeNode().c_str());
+	ASSERT( self->masterData.isValid() );
 
-//TODO: Print out the backup mutation log value. The backup log value (i.e., the value in the kv pair) has the following format
-//version(12B)|mutationRef|MutationRef|....
-//A mutationRef has the format: |type_4B|param1_size_4B|param2_size_4B|param1|param2.
-//Note: The data is stored in little endian! You need to convert it to BigEndian so that you know how long the param1 and param2 is and how to format them!
-void printBackupMutationRefValueHex(Standalone<StringRef> val_input, std::string prefix) {
-	std::stringstream ss;
-	const int version_size = 12;
-	const int header_size = 12;
-	StringRef val = val_input.contents();
-	StringRefReaderMX reader(val, restore_corrupted_data());
-
-	int count_size = 0;
-	// Get the version
-	uint64_t version = reader.consume<uint64_t>();
-	count_size += 8;
-	uint32_t val_length_decode = reader.consume<uint32_t>();
-	count_size += 4;
-
-	printf("----------------------------------------------------------\n");
-	printf("To decode value:%s\n", getHexString(val).c_str());
-	if ( val_length_decode != (val.size() - 12) ) {
-		fprintf(stderr, "%s[PARSE ERROR]!!! val_length_decode:%d != val.size:%d\n", prefix.c_str(), val_length_decode, val.size());
+	// Set up the role, and the global status for each node
+	int numNodes = self->workers_workerInterface.size();
+	state int numLoader = numNodes * ratio_loader_to_applier / (ratio_loader_to_applier + 1);
+	int numApplier = numNodes - numLoader;
+	if (numLoader <= 0 || numApplier <= 0) {
+		ASSERT( numLoader > 0 ); // Quick check in correctness
+		ASSERT( numApplier > 0 );
+		fprintf(stderr, "[ERROR] not enough nodes for loader and applier. numLoader:%d, numApplier:%d, ratio_loader_to_applier:%d, numAgents:%d\n", numLoader, numApplier, ratio_loader_to_applier, numNodes);
 	} else {
-		if ( debug_verbose ) {
-			printf("%s[PARSE SUCCESS] val_length_decode:%d == (val.size:%d - 12)\n", prefix.c_str(), val_length_decode, val.size());
-		}
+		printf("Node%s: Configure roles numWorkders:%d numLoader:%d numApplier:%d\n", self->describeNode().c_str(), numNodes, numLoader, numApplier);
 	}
 
-	// Get the mutation header
-	while (1) {
-		// stop when reach the end of the string
-		if(reader.eof() ) { //|| *reader.rptr == 0xFFCheckRestoreRequestDoneErrorMX
-			//printf("Finish decode the value\n");
-			break;
-		}
-
-
-		uint32_t type = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
-		uint32_t kLen = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
-		uint32_t vLen = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
-		const uint8_t *k = reader.consume(kLen);
-		const uint8_t *v = reader.consume(vLen);
-		count_size += 4 * 3 + kLen + vLen;
-
-		if ( kLen < 0 || kLen > val.size() || vLen < 0 || vLen > val.size() ) {
-			fprintf(stderr, "%s[PARSE ERROR]!!!! kLen:%d(0x%04x) vLen:%d(0x%04x)\n", prefix.c_str(), kLen, kLen, vLen, vLen);
-		}
-
-		if ( debug_verbose ) {
-			printf("%s---DedodeBackupMutation: Type:%d K:%s V:%s k_size:%d v_size:%d\n", prefix.c_str(),
-				   type,  getHexString(KeyRef(k, kLen)).c_str(), getHexString(KeyRef(v, vLen)).c_str(), kLen, vLen);
-		}
-
-	}
-	if ( debug_verbose ) {
-		printf("----------------------------------------------------------\n");
-	}
-}
-
-void printBackupLogKeyHex(Standalone<StringRef> key_input, std::string prefix) {
-	std::stringstream ss;
-	const int version_size = 12;
-	const int header_size = 12;
-	StringRef val = key_input.contents();
-	StringRefReaderMX reader(val, restore_corrupted_data());
-
-	int count_size = 0;
-	// Get the version
-	uint64_t version = reader.consume<uint64_t>();
-	count_size += 8;
-	uint32_t val_length_decode = reader.consume<uint32_t>();
-	count_size += 4;
-
-	printf("----------------------------------------------------------\n");
-	printf("To decode value:%s\n", getHexString(val).c_str());
-	if ( val_length_decode != (val.size() - 12) ) {
-		fprintf(stderr, "%s[PARSE ERROR]!!! val_length_decode:%d != val.size:%d\n", prefix.c_str(), val_length_decode, val.size());
-	} else {
-		printf("%s[PARSE SUCCESS] val_length_decode:%d == (val.size:%d - 12)\n", prefix.c_str(), val_length_decode, val.size());
-	}
-
-	// Get the mutation header
-	while (1) {
-		// stop when reach the end of the string
-		if(reader.eof() ) { //|| *reader.rptr == 0xFF
-			//printf("Finish decode the value\n");
-			break;
-		}
-
-
-		uint32_t type = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
-		uint32_t kLen = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
-		uint32_t vLen = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
-		const uint8_t *k = reader.consume(kLen);
-		const uint8_t *v = reader.consume(vLen);
-		count_size += 4 * 3 + kLen + vLen;
-
-		if ( kLen < 0 || kLen > val.size() || vLen < 0 || vLen > val.size() ) {
-			printf("%s[PARSE ERROR]!!!! kLen:%d(0x%04x) vLen:%d(0x%04x)\n", prefix.c_str(), kLen, kLen, vLen, vLen);
-		}
-
-		printf("%s---DedoceBackupMutation: Type:%d K:%s V:%s k_size:%d v_size:%d\n", prefix.c_str(),
-			   type,  getHexString(KeyRef(k, kLen)).c_str(), getHexString(KeyRef(v, vLen)).c_str(), kLen, vLen);
-
-	}
-	printf("----------------------------------------------------------\n");
-}
-
-void printKVOps(Reference<RestoreData> rd) {
-	std::string typeStr = "MSet";
-	TraceEvent("PrintKVOPs").detail("MapSize", rd->kvOps.size());
-	printf("PrintKVOPs num_of_version:%ld\n", rd->kvOps.size());
-	for ( auto it = rd->kvOps.begin(); it != rd->kvOps.end(); ++it ) {
-		TraceEvent("PrintKVOPs\t").detail("Version", it->first).detail("OpNum", it->second.size());
-		printf("PrintKVOPs Version:%08lx num_of_ops:%d\n",  it->first, it->second.size());
-		for ( auto m = it->second.begin(); m != it->second.end(); ++m ) {
-			if (  m->type >= MutationRef::Type::SetValue && m->type <= MutationRef::Type::MAX_ATOMIC_OP )
-				typeStr = typeString[m->type];
-			else {
-				printf("PrintKVOPs MutationType:%d is out of range\n", m->type);
-			}
-
-			printf("\tPrintKVOPs Version:%016lx MType:%s K:%s, V:%s K_size:%d V_size:%d\n", it->first, typeStr.c_str(),
-				   getHexString(m->param1).c_str(), getHexString(m->param2).c_str(), m->param1.size(), m->param2.size());
-
-			TraceEvent("PrintKVOPs\t\t").detail("Version", it->first)
-					.detail("MType", m->type).detail("MTypeStr", typeStr)
-					.detail("MKey", getHexString(m->param1))
-					.detail("MValueSize", m->param2.size())
-					.detail("MValue", getHexString(m->param2));
-		}
-	}
-}
-
-// Sanity check if KVOps is sorted
-bool isKVOpsSorted(Reference<RestoreData> rd) {
-	bool ret = true;
-	auto prev = rd->kvOps.begin();
-	for ( auto it = rd->kvOps.begin(); it != rd->kvOps.end(); ++it ) {
-		if ( prev->first > it->first ) {
-			ret = false;
-			break;
-		}
-		prev = it;
-	}
-	return ret;
-}
-
-bool allOpsAreKnown(Reference<RestoreData> rd) {
-	bool ret = true;
-	for ( auto it = rd->kvOps.begin(); it != rd->kvOps.end(); ++it ) {
-		for ( auto m = it->second.begin(); m != it->second.end(); ++m ) {
-			if ( m->type == MutationRef::SetValue || m->type == MutationRef::ClearRange
-			    || isAtomicOp((MutationRef::Type) m->type) )
-				continue;
-			else {
-				printf("[ERROR] Unknown mutation type:%d\n", m->type);
-				ret = false;
-			}
-		}
-
-	}
-
-	return ret;
-}
-
-//key_input format: [logRangeMutation.first][hash_value_of_commit_version:1B][bigEndian64(commitVersion)][bigEndian32(part)]
-bool concatenateBackupMutationForLogFile(Reference<RestoreData> rd, Standalone<StringRef> val_input, Standalone<StringRef> key_input) {
-	std::string prefix = "||\t";
-	std::stringstream ss;
-	const int version_size = 12;
-	const int header_size = 12;
-	StringRef val = val_input.contents();
-	StringRefReaderMX reader(val, restore_corrupted_data());
-	StringRefReaderMX readerKey(key_input, restore_corrupted_data()); //read key_input!
-	int logRangeMutationFirstLength = key_input.size() - 1 - 8 - 4;
-	bool concatenated = false;
-
-	if ( logRangeMutationFirstLength < 0 ) {
-		printf("[ERROR]!!! logRangeMutationFirstLength:%ld < 0, key_input.size:%ld\n", logRangeMutationFirstLength, key_input.size());
-	}
-
-	if ( debug_verbose ) {
-		printf("[DEBUG] Process key_input:%s\n", getHexKey(key_input, logRangeMutationFirstLength).c_str());
-	}
-
-	//PARSE key
-	Standalone<StringRef> id_old = key_input.substr(0, key_input.size() - 4); //Used to sanity check the decoding of key is correct
-	Standalone<StringRef> partStr = key_input.substr(key_input.size() - 4, 4); //part
-	StringRefReaderMX readerPart(partStr, restore_corrupted_data());
-	uint32_t part_direct = readerPart.consumeNetworkUInt32(); //Consume a bigEndian value
-	if ( debug_verbose  ) {
-		printf("[DEBUG] Process prefix:%s and partStr:%s part_direct:%08x fromm key_input:%s, size:%ld\n",
-			   getHexKey(id_old, logRangeMutationFirstLength).c_str(),
-			   getHexString(partStr).c_str(),
-			   part_direct,
-			   getHexKey(key_input, logRangeMutationFirstLength).c_str(),
-			   key_input.size());
-	}
-
-	StringRef longRangeMutationFirst;
-
-	if ( logRangeMutationFirstLength > 0 ) {
-		printf("readerKey consumes %dB\n", logRangeMutationFirstLength);
-		longRangeMutationFirst = StringRef(readerKey.consume(logRangeMutationFirstLength), logRangeMutationFirstLength);
-	}
-
-	uint8_t hashValue = readerKey.consume<uint8_t>();
-	uint64_t commitVersion = readerKey.consumeNetworkUInt64(); // Consume big Endian value encoded in log file, commitVersion is in littleEndian
-	uint64_t commitVersionBE = bigEndian64(commitVersion);
-	uint32_t part = readerKey.consumeNetworkUInt32(); //Consume big Endian value encoded in log file
-	uint32_t partBE = bigEndian32(part);
-	Standalone<StringRef> id2 = longRangeMutationFirst.withSuffix(StringRef(&hashValue,1)).withSuffix(StringRef((uint8_t*) &commitVersion, 8));
-
-	//Use commitVersion as id
-	Standalone<StringRef> id = StringRef((uint8_t*) &commitVersion, 8);
-
-	if ( debug_verbose ) {
-		printf("[DEBUG] key_input_size:%d longRangeMutationFirst:%s hashValue:%02x commitVersion:%016lx (BigEndian:%016lx) part:%08x (BigEndian:%08x), part_direct:%08x mutationMap.size:%ld\n",
-			   key_input.size(), longRangeMutationFirst.printable().c_str(), hashValue,
-			   commitVersion, commitVersionBE,
-			   part, partBE,
-			   part_direct, rd->mutationMap.size());
-	}
-
-	if ( rd->mutationMap.find(id) == rd->mutationMap.end() ) {
-		rd->mutationMap.insert(std::make_pair(id, val_input));
-		if ( part_direct != 0 ) {
-			printf("[ERROR]!!! part:%d != 0 for key_input:%s\n", part_direct, getHexString(key_input).c_str());
-		}
-		rd->mutationPartMap.insert(std::make_pair(id, part_direct));
-	} else { // concatenate the val string
-//		printf("[INFO] Concatenate the log's val string at version:%ld\n", id.toString().c_str());
-		rd->mutationMap[id] = rd->mutationMap[id].contents().withSuffix(val_input.contents()); //Assign the new Areana to the map's value
-		if ( part_direct != (rd->mutationPartMap[id] + 1) ) {
-			printf("[ERROR]!!! current part id:%d new part_direct:%d is not the next integer of key_input:%s\n", rd->mutationPartMap[id], part_direct, getHexString(key_input).c_str());
-			printf("[HINT] Check if the same range or log file has been processed more than once!\n");
-		}
-		if ( part_direct != part ) {
-			printf("part_direct:%08x != part:%08x\n", part_direct, part);
-		}
-		rd->mutationPartMap[id] = part_direct;
-		concatenated = true;
-	}
-
-	return concatenated;
-}
-
-bool isRangeMutation(MutationRef m) {
-	if (m.type == MutationRef::Type::ClearRange) {
-		if (m.type == MutationRef::Type::DebugKeyRange) {
-			printf("[ERROR] DebugKeyRange mutation is in backup data unexpectedly. We still handle it as a range mutation; the suspicious mutation:%s\n", m.toString().c_str());
-		}
-		return true;
-	} else {
-		if ( !(m.type == MutationRef::Type::SetValue ||
-				isAtomicOp((MutationRef::Type) m.type)) ) {
-			printf("[ERROR] %s mutation is in backup data unexpectedly. We still handle it as a key mutation; the suspicious mutation:%s\n", typeString[m.type], m.toString().c_str());
-
-		}
-		return false;
-	}
-}
-
-void splitMutation(Reference<RestoreData> rd,  MutationRef m, Arena& mvector_arena, VectorRef<MutationRef> mvector, Arena& nodeIDs_arena, VectorRef<UID> nodeIDs) {
-	// mvector[i] should be mapped to nodeID[i]
-	ASSERT(mvector.empty());
-	ASSERT(nodeIDs.empty());
-	// key range [m->param1, m->param2)
-	//std::map<Standalone<KeyRef>, UID>;
-	std::map<Standalone<KeyRef>, UID>::iterator itlow, itup; //we will return [itlow, itup)
-	itlow = rd->range2Applier.lower_bound(m.param1); // lower_bound returns the iterator that is >= m.param1
-	if ( itlow != rd->range2Applier.begin()) { // m.param1 is not the smallest key \00
-		// (itlow-1) is the node whose key range includes m.param1
-		--itlow;
-	} else {
-		if (m.param1 != LiteralStringRef("\00")) {
-			printf("[ERROR] splitMutation has bug on range mutation:%s\n", m.toString().c_str());
-		}
-	}
-
-	itup = rd->range2Applier.upper_bound(m.param2); // upper_bound returns the iterator that is > m.param2; return rmap::end if no keys are considered to go after m.param2.
-	ASSERT( itup == rd->range2Applier.end() || itup->first >= m.param2 );
-	// Now adjust for the case: example: mutation range is [a, d); we have applier's ranges' inclusive lower bound values are: a, b, c, d, e; upper_bound(d) returns itup to e, but we want itup to d.
-	--itup;
-	ASSERT( itup->first <= m.param2 );
-	if ( itup->first < m.param2 ) {
-		++itup; //make sure itup is >= m.param2, that is, itup is the next key range >= m.param2
-	}
-
-	while (itlow->first < itup->first) {
-		MutationRef curm; //current mutation
-		curm.type = m.type;
-		curm.param1 = itlow->first;
-		itlow++;
-		if (itlow == rd->range2Applier.end()) {
-			curm.param2 = normalKeys.end;
-		} else {
-			curm.param2 = itlow->first;
-		}
-		mvector.push_back(mvector_arena, curm);
-
-		nodeIDs.push_back(nodeIDs_arena, itlow->second);
-	}
-
-	return;
-}
-
-ACTOR Future<Void> registerMutationsToApplier(Reference<RestoreData> rd) {
-	printf("[INFO][Loader] Node:%s rd->masterApplier:%s, hasApplierInterface:%d registerMutationsToApplier\n",
-			rd->describeNode().c_str(), rd->masterApplier.toString().c_str(),
-			rd->workers_interface.find(rd->masterApplier) != rd->workers_interface.end());
-
-	state RestoreInterface applierCmdInterf; // = rd->workers_interface[rd->masterApplier];
-	state int packMutationNum = 0;
-	state int packMutationThreshold = 10;
-	state int kvCount = 0;
-	state std::vector<Future<RestoreCommonReply>> cmdReplies;
-
-	state int splitMutationIndex = 0;
-
-	printAppliersKeyRange(rd);
-
-	//state double mutationVectorThreshold = 1;//1024 * 10; // Bytes.
-	state std::map<UID, Standalone<VectorRef<MutationRef>>> applierMutationsBuffer; // The mutation vector to be sent to each applier
-	state std::map<UID, double> applierMutationsSize; // buffered mutation vector size for each applier
-	// Initialize the above two maps
-	state std::vector<UID> applierIDs = getWorkingApplierIDs(rd);
+	// Assign a role to each worker
+	state int nodeIndex = 0;
+	state RestoreRole role;
+	state UID nodeID;
+	printf("Node:%s Start configuring roles for workers\n", self->describeNode().c_str());
+	self->cmdID.initPhase(RestoreCommandEnum::Set_Role);
 	loop {
 		try {
-			packMutationNum = 0;
-			splitMutationIndex = 0;
-			kvCount = 0;
-			state std::map<Version, Standalone<VectorRef<MutationRef>>>::iterator kvOp;
-			rd->cmdID.initPhase(RestoreCommandEnum::Loader_Send_Mutations_To_Applier);
-			// In case try-catch has error and loop back
-			applierMutationsBuffer.clear();
-			applierMutationsSize.clear();
-			for (auto &applierID : applierIDs) {
-				applierMutationsBuffer[applierID] = Standalone<VectorRef<MutationRef>>(VectorRef<MutationRef>());
-				applierMutationsSize[applierID] = 0.0;
-			}
-			for ( kvOp = rd->kvOps.begin(); kvOp != rd->kvOps.end(); kvOp++) {
-				state uint64_t commitVersion = kvOp->first;
-				state int mIndex;
-				state MutationRef kvm;
-				for (mIndex = 0; mIndex < kvOp->second.size(); mIndex++) {
-					kvm = kvOp->second[mIndex];
-					if ( debug_verbose ) {
-						printf("[VERBOSE_DEBUG] mutation to sent to applier, mutation:%s\n", kvm.toString().c_str());
-					}
-					// Send the mutation to applier
-					if (isRangeMutation(kvm)) {
-						// Because using a vector of mutations causes overhead, and the range mutation should happen rarely;
-						// We handle the range mutation and key mutation differently for the benefit of avoiding memory copy
-						state Standalone<VectorRef<MutationRef>> mvector;
-						state Standalone<VectorRef<UID>> nodeIDs;
-						// '' Bug may be here! The splitMutation() may be wrong!
-						splitMutation(rd, kvm, mvector.arena(), mvector.contents(), nodeIDs.arena(), nodeIDs.contents());
-						ASSERT(mvector.size() == nodeIDs.size());
-
-						for (splitMutationIndex = 0; splitMutationIndex < mvector.size(); splitMutationIndex++ ) {
-							MutationRef mutation = mvector[splitMutationIndex];
-							UID applierID = nodeIDs[splitMutationIndex];
-							applierCmdInterf = rd->workers_interface[applierID];
-							applierMutationsBuffer[applierID].push_back(applierMutationsBuffer[applierID].arena(), mutation); // Q: Maybe push_back_deep()?
-							applierMutationsSize[applierID] += mutation.expectedSize();
-
-							kvCount++;
-						}
-
-						for (auto &applierID : applierIDs) {
-							if ( applierMutationsSize[applierID] >= mutationVectorThreshold ) {
-								rd->cmdID.nextCmd();
-								cmdReplies.push_back(applierCmdInterf.sendMutationVector.getReply(
-									RestoreSendMutationVectorRequest(rd->cmdID, commitVersion, applierMutationsBuffer[applierID])));
-								applierMutationsBuffer[applierID].pop_front(applierMutationsBuffer[applierID].size());
-								applierMutationsSize[applierID] = 0;
-
-								printf("[INFO][Loader] Waits for applier to receive %ld range mutations\n", cmdReplies.size());
-								std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) );
-								cmdReplies.clear();
-							}
-						}
-					} else { // mutation operates on a particular key
-						std::map<Standalone<KeyRef>, UID>::iterator itlow = rd->range2Applier.lower_bound(kvm.param1); // lower_bound returns the iterator that is >= m.param1
-						// make sure itlow->first <= m.param1
-						if ( itlow == rd->range2Applier.end() || itlow->first > kvm.param1 ) {
-							--itlow;
-						}
-						ASSERT( itlow->first <= kvm.param1 );
-						MutationRef mutation = kvm;
-						UID applierID = itlow->second;
-						applierCmdInterf = rd->workers_interface[applierID];
-						kvCount++;
-
-						applierMutationsBuffer[applierID].push_back(applierMutationsBuffer[applierID].arena(), mutation); // Q: Maybe push_back_deep()?
-						applierMutationsSize[applierID] += mutation.expectedSize();
-						if ( applierMutationsSize[applierID] >= mutationVectorThreshold ) {
-							rd->cmdID.nextCmd();
-							cmdReplies.push_back(applierCmdInterf.sendMutationVector.getReply(
-												RestoreSendMutationVectorRequest(rd->cmdID, commitVersion, applierMutationsBuffer[applierID])));
-							applierMutationsBuffer[applierID].pop_front(applierMutationsBuffer[applierID].size());
-							applierMutationsSize[applierID] = 0;
-
-							printf("[INFO][Loader] Waits for applier to receive %ld range mutations\n", cmdReplies.size());
-							std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) );
-							cmdReplies.clear();
-						}
-					}
+			std::vector<Future<RestoreCommonReply>> cmdReplies;
+			for (auto &workerInterf : self->workers_workerInterface)  {
+				if ( nodeIndex < numLoader ) {
+					role = RestoreRole::Loader;
+				} else {
+					role = RestoreRole::Applier;
 				}
-
+				nodeID = workerInterf.first;
+				self->cmdID.nextCmd();
+				printf("[CMD:%s] Node:%s Set role (%s) to node (index=%d uid=%s)\n", self->cmdID.toString().c_str(), self->describeNode().c_str(),
+						getRoleStr(role).c_str(), nodeIndex, nodeID.toString().c_str());
+				cmdReplies.push_back( workerInterf.second.recruitRole.getReply(RestoreRecruitRoleRequest(self->cmdID, role, nodeIndex)) );
+				nodeIndex++;
 			}
-
-			// In case the mutation vector is not larger than mutationVectorThreshold
-			printf("[DEBUG][Loader] sendMutationVector sends the remaining applierMutationsBuffer, applierIDs.size:%d\n", applierIDs.size());
-			for (auto &applierID : applierIDs) {
-				if (applierMutationsBuffer[applierID].empty()) { //&& applierMutationsSize[applierID] >= 1
-					continue;
-				}
-				printf("[DEBUG][Loader] sendMutationVector for applierID:%s\n", applierID.toString().c_str());
-				rd->cmdID.nextCmd();
-				cmdReplies.push_back(applierCmdInterf.sendMutationVector.getReply(
-									RestoreSendMutationVectorRequest(rd->cmdID, commitVersion, applierMutationsBuffer[applierID])));
-				applierMutationsBuffer[applierID].pop_front(applierMutationsBuffer[applierID].size());
-				applierMutationsSize[applierID] = 0;
-				printf("[INFO][Loader] Waits for applier to receive %ld range mutations\n", cmdReplies.size());
-				std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) ); // Q: We need to wait for each reply, otherwise, correctness has error. Why?
-				cmdReplies.clear();
-			}
-
-			if (!cmdReplies.empty()) {
-				printf("[INFO][Loader] Last Waits for applier to receive %ld range mutations\n", cmdReplies.size());
-				std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) );
-				//std::vector<RestoreCommonReply> reps =  wait( getAll(cmdReplies) );
-				cmdReplies.clear();
-			}
-			printf("[Summary][Loader] Node:%s Last CMDUID:%s produces %d mutation operations\n",
-					rd->describeNode().c_str(), rd->cmdID.toString().c_str(), kvCount);
-
+			std::vector<RestoreCommonReply> reps = wait( timeoutError(getAll(cmdReplies), FastRestore_Failure_Timeout) );
+			printf("[RecruitRestoreRoles] Finished\n");
 			break;
-
 		} catch (Error &e) {
 			// Handle the command reply timeout error
-			fprintf(stdout, "[ERROR] registerMutationsToApplier Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", rd->describeNode().c_str(),
-					rd->cmdID.toString().c_str(), e.code(), e.what());
-		}
-	};
-
-	return Void();
-}
-
-// Loader: Register sampled mutations
-ACTOR Future<Void> registerMutationsToMasterApplier(Reference<RestoreData> rd) {
-	printf("[Sampling] Node:%s registerMutationsToMaster() rd->masterApplier:%s, hasApplierInterface:%d\n",
-			rd->describeNode().c_str(), rd->masterApplier.toString().c_str(),
-			rd->workers_interface.find(rd->masterApplier) != rd->workers_interface.end());
-
-	ASSERT(rd->workers_interface.find(rd->masterApplier) != rd->workers_interface.end());
-
-	state RestoreInterface applierCmdInterf = rd->workers_interface[rd->masterApplier];
-	state UID applierID = rd->masterApplier;
-	state int packMutationNum = 0;
-	state int packMutationThreshold = 1;
-	state int kvCount = 0;
-	state std::vector<Future<RestoreCommonReply>> cmdReplies;
-
-	state int splitMutationIndex = 0;
-	state std::map<Version, Standalone<VectorRef<MutationRef>>>::iterator kvOp;
-	state int mIndex;
-	state uint64_t commitVersion;
-	state MutationRef kvm;
-
-	state Standalone<VectorRef<MutationRef>> mutationsBuffer; // The mutation vector to be sent to master applier
-	state double mutationsSize = 0;
-	//state double mutationVectorThreshold = 1; //1024 * 10; // Bytes
-	loop {
-		try {
-			cmdReplies.clear();
-			mutationsBuffer.pop_front(mutationsBuffer.size());
-			mutationsSize = 0;
-			packMutationNum = 0;
-			rd->cmdID.initPhase(RestoreCommandEnum::Loader_Send_Sample_Mutation_To_Applier); 
-			// TODO: Consider using a different EndPoint for loader and applier communication.
-			// Otherwise, applier may receive loader's message while applier is waiting for master to assign key-range
-			for ( kvOp = rd->kvOps.begin(); kvOp != rd->kvOps.end(); kvOp++) {
-				commitVersion = kvOp->first;
-				
-				for (mIndex = 0; mIndex < kvOp->second.size(); mIndex++) {
-					kvm = kvOp->second[mIndex];
-					rd->cmdID.nextCmd();
-					if ( debug_verbose || true ) { // Debug deterministic bug
-						printf("[VERBOSE_DEBUG] send mutation to applier, mIndex:%d mutation:%s\n", mIndex, kvm.toString().c_str());
-					}
-					mutationsBuffer.push_back(mutationsBuffer.arena(), kvm);
-					mutationsSize += kvm.expectedSize();
-					if ( mutationsSize >= mutationVectorThreshold ) {
-						rd->cmdID.nextCmd();
-						cmdReplies.push_back(applierCmdInterf.sendSampleMutationVector.getReply(
-							RestoreSendMutationVectorRequest(rd->cmdID, commitVersion, mutationsBuffer)));
-							mutationsBuffer.pop_front(mutationsBuffer.size());
-							mutationsSize = 0;
-						if ( debug_verbose ) {
-							printf("[INFO][Loader] Waits for master applier to receive %ld mutations\n", mutationsBuffer.size());
-						}
-						std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) );
-						//std::vector<RestoreCommonReply> reps = wait( getAll(cmdReplies) );
-						cmdReplies.clear();
-					}
-
-					kvCount++;
-				}
-			}
-
-			// The leftover mutationVector whose size is < mutationVectorThreshold
-			if ( mutationsSize > 0 ) {
-				rd->cmdID.nextCmd();
-				cmdReplies.push_back(applierCmdInterf.sendSampleMutationVector.getReply(
-					RestoreSendMutationVectorRequest(rd->cmdID, commitVersion, mutationsBuffer)));
-					mutationsBuffer.pop_front(mutationsBuffer.size());
-					mutationsSize = 0;
-			}
-
-
-			if (!cmdReplies.empty()) {
-				printf("[INFO][Loader] Last waits for master applier to receive %ld mutations\n", mutationsBuffer.size());
-				//std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout) );
-				std::vector<RestoreCommonReply> reps = wait( getAll(cmdReplies) );
-				cmdReplies.clear();
-			}
-
-			printf("[Sample Summary][Loader] Node:%s produces %d mutation operations\n", rd->describeNode().c_str(), kvCount);
-			break;
-		} catch (Error &e) {
-			// TODO: Handle the command reply timeout error
-			if (e.code() != error_code_io_timeout) {
-				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s timeout\n", rd->describeNode().c_str(), rd->cmdID.toString().c_str());
-			} else {
-				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", rd->describeNode().c_str(),
-						rd->cmdID.toString().c_str(), e.code(), e.what());
-			}
-			printf("[WARNING] Node:%s timeout at waiting on replies of Loader_Send_Sample_Mutation_To_Applier. Retry...\n", rd->describeNode().c_str());
+			fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", self->describeNode().c_str(),
+						self->cmdID.toString().c_str(), e.code(), e.what());
+			printf("Node:%s waits on replies time out. Current phase: Set_Role, Retry all commands.\n", self->describeNode().c_str());
 		}
 	}
 
 	return Void();
 }
 
-ACTOR Future<Void> handleHeartbeat(RestoreSimpleRequest req, Reference<RestoreData> rd, RestoreInterface interf) {
-	// wait( delay(1.0) );
-	req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
 
-	return Void();
-}
-
-ACTOR Future<Void> handleVersionBatchRequest(RestoreVersionBatchRequest req, Reference<RestoreData> rd, RestoreInterface interf) {
-	// wait( delay(1.0) );
-	printf("[Batch:%d] Node:%s Start...\n", req.batchID, rd->describeNode().c_str());
-	while (rd->isInProgress(RestoreCommandEnum::RESET_VersionBatch)) {
-		printf("[DEBUG] NODE:%s handleVersionBatchRequest wait for 5s\n",  rd->describeNode().c_str());
-		wait(delay(5.0));
-	}
-
-	// Handle duplicate, assuming cmdUID is always unique for the same workload
-	if ( rd->isCmdProcessed(req.cmdID) ) {
-		printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", rd->describeNode().c_str(), req.cmdID.toString().c_str());
-		req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-		return Void();
-	} 
-
-	rd->setInProgressFlag(RestoreCommandEnum::RESET_VersionBatch);
-
-	rd->resetPerVersionBatch();
-	rd->processedFiles.clear();
-	req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-
-	rd->processedCmd[req.cmdID] = 1;
-	rd->clearInProgressFlag(RestoreCommandEnum::RESET_VersionBatch);
-
-	// This actor never returns. You may cancel it in master
-	return Void();
-}
-
-ACTOR Future<Void> handleSetRoleRequest(RestoreSetRoleRequest req, Reference<RestoreData> rd, RestoreInterface interf) {
-	// wait( delay(1.0) );
-	rd->localNodeStatus.init(req.role);
-	rd->localNodeStatus.nodeID = interf.id();
-	rd->localNodeStatus.nodeIndex = req.nodeIndex;
-	rd->masterApplier = req.masterApplierID;
-	printf("[INFO][Worker] Node:%s get role %s\n", rd->describeNode().c_str(),
-			getRoleStr(rd->localNodeStatus.role).c_str());
-	req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-
-	// This actor never returns. You may cancel it in master
-	return Void();
-}
-
-
-ACTOR Future<Void> handleSampleRangeFileRequest(RestoreLoadFileRequest req, Reference<RestoreData> rd, RestoreInterface interf) {
-	//printf("[INFO] Node:%s Got Restore Command: cmdID:%s.\n", rd->describeNode().c_str(), req.cmdID.toString().c_str());
-
-	state LoadingParam param = req.param;
-	state int beginBlock = 0;
-	state int j = 0;
-	state int readLen = 0;
-	state int64_t readOffset = param.offset;
-
-	while (rd->isInProgress(RestoreCommandEnum::Sample_Range_File)) {
-		printf("[DEBUG] NODE:%s sampleRangeFile wait for 5s\n",  rd->describeNode().c_str());
-		wait(delay(5.0));
-	}
-
-	// Handle duplicate, assuming cmdUID is always unique for the same workload
-	if ( rd->isCmdProcessed(req.cmdID) ) {
-		printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", rd->describeNode().c_str(), req.cmdID.toString().c_str());
-		req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-		return Void();
-	} 
-
-	rd->setInProgressFlag(RestoreCommandEnum::Sample_Range_File);
-	printf("[Sample_Range_File][Loader] Node: %s, loading param:%s\n",
-			rd->describeNode().c_str(), param.toString().c_str());
-
-	// TODO: This can be expensive
-	state Reference<IBackupContainer> bc =  rd->bc;
-	printf("[INFO] node:%s open backup container for url:%s\n",
-			rd->describeNode().c_str(),
-			param.url.toString().c_str());
-
-
-	rd->kvOps.clear(); //Clear kvOps so that kvOps only hold mutations for the current data block. We will send all mutations in kvOps to applier
-	rd->mutationMap.clear();
-	rd->mutationPartMap.clear();
-
-	ASSERT( param.blockSize > 0 );
-	//state std::vector<Future<Void>> fileParserFutures;
-	if (param.offset % param.blockSize != 0) {
-		printf("[WARNING] Parse file not at block boundary! param.offset:%ld param.blocksize:%ld, remainder:%ld\n",
-				param.offset, param.blockSize, param.offset % param.blockSize);
-	}
-
-	ASSERT( param.offset + param.blockSize >= param.length ); // We only sample one data block or less (at the end of the file) of a file.
-	for (j = param.offset; j < param.length; j += param.blockSize) {
-		readOffset = j;
-		readLen = std::min<int64_t>(param.blockSize, param.length - j);
-		wait( _parseRangeFileToMutationsOnLoader(rd, bc, param.version, param.filename, readOffset, readLen, param.restoreRange, param.addPrefix, param.removePrefix) );
-		++beginBlock;
-	}
-
-	printf("[Sampling][Loader] Node:%s finishes sample Range file:%s\n", rd->describeNode().c_str(), param.filename.c_str());
-	// TODO: Send to applier to apply the mutations
-	printf("[Sampling][Loader] Node:%s will send sampled mutations to applier\n", rd->describeNode().c_str());
-	wait( registerMutationsToMasterApplier(rd) ); // Send the parsed mutation to applier who will apply the mutation to DB
-
-	//rd->processedFiles.insert(std::make_pair(param.filename, 1));
-
-	//TODO: Send ack to master that loader has finished loading the data
-	req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-	rd->processedCmd[req.cmdID] = 1; // Record the processed comand to handle duplicate command
-	//rd->kvOps.clear(); 
-
-	rd->clearInProgressFlag(RestoreCommandEnum::Sample_Range_File);
-
-	return Void();
-}
-
-ACTOR Future<Void> handleSampleLogFileRequest(RestoreLoadFileRequest req, Reference<RestoreData> rd, RestoreInterface interf) {
-	state LoadingParam param = req.param;
-	state int beginBlock = 0;
-	state int j = 0;
-	state int readLen = 0;
-	state int64_t readOffset = param.offset;
-
-	while (rd->isInProgress(RestoreCommandEnum::Sample_Log_File)) {
-		printf("[DEBUG] NODE:%s sampleLogFile wait for 5s\n",  rd->describeNode().c_str());
-		wait(delay(5.0));
-	}
-
-	// Handle duplicate message
-	if ( rd->isCmdProcessed(req.cmdID) ) {
-		printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", rd->describeNode().c_str(), req.cmdID.toString().c_str());
-		req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-		return Void();
-	}
-
-	rd->setInProgressFlag(RestoreCommandEnum::Sample_Log_File);
-	printf("[Sample_Log_File][Loader]  Node: %s, loading param:%s\n", rd->describeNode().c_str(), param.toString().c_str());
-
-	// TODO: Expensive operation
-	state Reference<IBackupContainer> bc =  rd->bc;
-	printf("[Sampling][Loader] Node:%s open backup container for url:%s\n",
-			rd->describeNode().c_str(),
-			param.url.toString().c_str());
-	printf("[Sampling][Loader] Node:%s filename:%s blockSize:%ld\n",
-			rd->describeNode().c_str(),
-			param.filename.c_str(), param.blockSize);
-
-	rd->kvOps.clear(); //Clear kvOps so that kvOps only hold mutations for the current data block. We will send all mutations in kvOps to applier
-	rd->mutationMap.clear();
-	rd->mutationPartMap.clear();
-
-	ASSERT( param.blockSize > 0 );
-	//state std::vector<Future<Void>> fileParserFutures;
-	if (param.offset % param.blockSize != 0) {
-		printf("[WARNING] Parse file not at block boundary! param.offset:%ld param.blocksize:%ld, remainder:%ld\n",
-			param.offset, param.blockSize, param.offset % param.blockSize);
-	}
-	ASSERT( param.offset + param.blockSize >= param.length ); // Assumption: Only sample one data block or less
-	for (j = param.offset; j < param.length; j += param.blockSize) {
-		readOffset = j;
-		readLen = std::min<int64_t>(param.blockSize, param.length - j);
-		// NOTE: Log file holds set of blocks of data. We need to parse the data block by block and get the kv pair(version, serialized_mutations)
-		// The set of mutations at the same version may be splitted into multiple kv pairs ACROSS multiple data blocks when the size of serialized_mutations is larger than 20000.
-		wait( _parseLogFileToMutationsOnLoader(rd, bc, param.version, param.filename, readOffset, readLen, param.restoreRange, param.addPrefix, param.removePrefix, param.mutationLogPrefix) );
-		++beginBlock;
-	}
-	printf("[Sampling][Loader] Node:%s finishes parsing the data block into kv pairs (version, serialized_mutations) for file:%s\n", rd->describeNode().c_str(), param.filename.c_str());
-	parseSerializedMutation(rd, true);
-
-	printf("[Sampling][Loader] Node:%s finishes process Log file:%s\n", rd->describeNode().c_str(), param.filename.c_str());
-	printf("[Sampling][Loader] Node:%s will send log mutations to applier\n", rd->describeNode().c_str());
-	wait( registerMutationsToMasterApplier(rd) ); // Send the parsed mutation to applier who will apply the mutation to DB
-
-	req.reply.send(RestoreCommonReply(interf.id(), req.cmdID)); // master node is waiting
-	rd->processedFiles.insert(std::make_pair(param.filename, 1));
-	rd->processedCmd[req.cmdID] = 1;
-
-	rd->clearInProgressFlag(RestoreCommandEnum::Sample_Log_File);
-
-	return Void();
-}
-
-ACTOR Future<Void> handleCalculateApplierKeyRangeRequest(RestoreCalculateApplierKeyRangeRequest req, Reference<RestoreData> rd, RestoreInterface interf) {
-	state int numMutations = 0;
-	state std::vector<Standalone<KeyRef>> keyRangeLowerBounds;
-
-	while (rd->isInProgress(RestoreCommandEnum::Calculate_Applier_KeyRange)) {
-		printf("[DEBUG] NODE:%s Calculate_Applier_KeyRange wait for 5s\n",  rd->describeNode().c_str());
-		wait(delay(5.0));
-	}
-
-	wait( delay(1.0) );
-	// Handle duplicate message
-	// We need to recalculate the value for duplicate message! Because the reply to duplicate message may arrive earlier!
-	if (rd->isCmdProcessed(req.cmdID) && !keyRangeLowerBounds.empty() ) {
-		printf("[DEBUG] Node:%s skip duplicate cmd:%s\n", rd->describeNode().c_str(), req.cmdID.toString().c_str());
-		req.reply.send(GetKeyRangeNumberReply(keyRangeLowerBounds.size()));
-		return Void();
-	}
-	rd->setInProgressFlag(RestoreCommandEnum::Calculate_Applier_KeyRange);
-
-	// Applier will calculate applier key range
-	printf("[INFO][Applier] CMD:%s, Node:%s Calculate key ranges for %d appliers\n",
-			req.cmdID.toString().c_str(), rd->describeNode().c_str(), req.numAppliers);
-
-	//ASSERT(req.cmd == (RestoreCommandEnum) req.cmdID.phase);
-	if ( keyRangeLowerBounds.empty() ) {
-		keyRangeLowerBounds = _calculateAppliersKeyRanges(rd, req.numAppliers); // keyRangeIndex is the number of key ranges requested
-		rd->keyRangeLowerBounds = keyRangeLowerBounds;
-	}
-	
-	printf("[INFO][Applier] CMD:%s, NodeID:%s: num of key ranges:%ld\n",
-			rd->cmdID.toString().c_str(), rd->describeNode().c_str(), keyRangeLowerBounds.size());
-	req.reply.send(GetKeyRangeNumberReply(keyRangeLowerBounds.size()));
-	rd->processedCmd[req.cmdID] = 1; // We should not skip this command in the following phase. Otherwise, the handler in other phases may return a wrong number of appliers
-	rd->clearInProgressFlag(RestoreCommandEnum::Calculate_Applier_KeyRange);
-
-	return Void();
-}
-
-ACTOR Future<Void> handleGetApplierKeyRangeRequest(RestoreGetApplierKeyRangeRequest req, Reference<RestoreData> rd, RestoreInterface interf) {
-	state int numMutations = 0;
-	//state std::vector<Standalone<KeyRef>> keyRangeLowerBounds = rd->keyRangeLowerBounds;
-
-	while (rd->isInProgress(RestoreCommandEnum::Get_Applier_KeyRange)) {
-		printf("[DEBUG] NODE:%s Calculate_Applier_KeyRange wait for 5s\n",  rd->describeNode().c_str());
-		wait(delay(5.0));
-	}
-
-	wait( delay(1.0) );
-	//NOTE: Must reply a valid lowerBound and upperBound! Otherwise, the master will receive an invalid value!
-	// if (rd->isCmdProcessed(req.cmdID) ) {
-	// 	printf("[DEBUG] Node:%s skip duplicate cmd:%s\n", rd->describeNode().c_str(), req.cmdID.toString().c_str());
-	// 	req.reply.send(GetKeyRangeReply(interf.id(), req.cmdID)); // Must wait until the previous command returns
-	// 	return Void();
-	// }
-	rd->setInProgressFlag(RestoreCommandEnum::Get_Applier_KeyRange);
-	
-	if ( req.applierIndex < 0 || req.applierIndex >= rd->keyRangeLowerBounds.size() ) {
-		printf("[INFO][Applier] NodeID:%s Get_Applier_KeyRange keyRangeIndex is out of range. keyIndex:%d keyRagneSize:%ld\n",
-				rd->describeNode().c_str(), req.applierIndex,  rd->keyRangeLowerBounds.size());
-	}
-	//ASSERT(req.cmd == (RestoreCommandEnum) req.cmdID.phase);
-
-	printf("[INFO][Applier] NodeID:%s replies Get_Applier_KeyRange. keyRangeIndex:%d lower_bound_of_keyRange:%s\n",
-			rd->describeNode().c_str(), req.applierIndex, getHexString(rd->keyRangeLowerBounds[req.applierIndex]).c_str());
-
-	KeyRef lowerBound = rd->keyRangeLowerBounds[req.applierIndex];
-	KeyRef upperBound = (req.applierIndex + 1) < rd->keyRangeLowerBounds.size() ? rd->keyRangeLowerBounds[req.applierIndex+1] : normalKeys.end;
-
-	req.reply.send(GetKeyRangeReply(interf.id(), req.cmdID, req.applierIndex, lowerBound, upperBound));
-	rd->clearInProgressFlag(RestoreCommandEnum::Get_Applier_KeyRange);
-
-	return Void();
-
-}
-
-// Assign key range to applier
-ACTOR Future<Void> handleSetApplierKeyRangeRequest(RestoreSetApplierKeyRangeRequest req, Reference<RestoreData> rd, RestoreInterface interf) {
-	// Idempodent operation. OK to re-execute the duplicate cmd
-	// The applier should remember the key range it is responsible for
-	//ASSERT(req.cmd == (RestoreCommandEnum) req.cmdID.phase);
-	//rd->applierStatus.keyRange = req.range;
-	while (rd->isInProgress(RestoreCommandEnum::Assign_Applier_KeyRange)) {
-		printf("[DEBUG] NODE:%s handleSetApplierKeyRangeRequest wait for 1s\n",  rd->describeNode().c_str());
-		wait(delay(1.0));
-	}
-	if ( rd->isCmdProcessed(req.cmdID) ) {
-		req.reply.send(RestoreCommonReply(interf.id(),req.cmdID));
-		return Void();
-	}
-	rd->setInProgressFlag(RestoreCommandEnum::Assign_Applier_KeyRange);
-
-	rd->range2Applier[req.range.begin] = req.applierID;
-
-	rd->processedCmd[req.cmdID] = 1;
-	rd->clearInProgressFlag(RestoreCommandEnum::Assign_Applier_KeyRange);
-
-	req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-
-	return Void();
-}
-
-ACTOR Future<Void> handleSetApplierKeyRangeVectorRequest(RestoreSetApplierKeyRangeVectorRequest req, Reference<RestoreData> rd, RestoreInterface interf) {
-	// Idempodent operation. OK to re-execute the duplicate cmd
-	// The applier should remember the key range it is responsible for
-	//ASSERT(req.cmd == (RestoreCommandEnum) req.cmdID.phase);
-	//rd->applierStatus.keyRange = req.range;
-	while (rd->isInProgress(RestoreCommandEnum::Notify_Loader_ApplierKeyRange)) {
-		printf("[DEBUG] NODE:%s handleSetApplierKeyRangeVectorRequest wait for 1s\n",  rd->describeNode().c_str());
-		wait(delay(1.0));
-	}
-	if ( rd->isCmdProcessed(req.cmdID) ) {
-		req.reply.send(RestoreCommonReply(interf.id(),req.cmdID));
-		return Void();
-	}
-	rd->setInProgressFlag(RestoreCommandEnum::Notify_Loader_ApplierKeyRange);
-
-	VectorRef<UID> appliers = req.applierIDs;
-	VectorRef<KeyRange> ranges = req.ranges;
-	for ( int i = 0; i < appliers.size(); i++ ) {
-		rd->range2Applier[ranges[i].begin] = appliers[i];
-	}
-	
-	rd->processedCmd[req.cmdID] = 1;
-	rd->clearInProgressFlag(RestoreCommandEnum::Notify_Loader_ApplierKeyRange);
-	req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-
-	return Void();
-}
-
-ACTOR Future<Void> handleLoadRangeFileRequest(RestoreLoadFileRequest req, Reference<RestoreData> rd, RestoreInterface interf) {
-	//printf("[INFO] Worker Node:%s starts handleLoadRangeFileRequest\n", rd->describeNode().c_str());
-
-	state LoadingParam param;
-	state int64_t beginBlock = 0;
-	state int64_t j = 0;
-	state int64_t readLen = 0;
-	state int64_t readOffset = 0;
-	state Reference<IBackupContainer> bc;
-
-	param = req.param;
-	beginBlock = 0;
-	j = 0;
-	readLen = 0;
-	readOffset = 0;
-	readOffset = param.offset;
-
-	while (rd->isInProgress(RestoreCommandEnum::Assign_Loader_Range_File)) {
-		printf("[DEBUG] NODE:%s loadRangeFile wait for 5s\n",  rd->describeNode().c_str());
-		wait(delay(5.0));
-	}
-
-	//Note: handle duplicate message delivery
-	if (rd->processedFiles.find(param.filename) != rd->processedFiles.end() ||
-		rd->isCmdProcessed(req.cmdID)) {
-		// printf("[WARNING]Node:%s, CMDUID:%s file:%s is delivered more than once! Reply directly without loading the file\n",
-		// 		rd->describeNode().c_str(), req.cmdID.toString().c_str(),
-		// 		param.filename.c_str());
-		req.reply.send(RestoreCommonReply(interf.id(),req.cmdID));
-		return Void();
-	}
-
-	rd->setInProgressFlag(RestoreCommandEnum::Assign_Loader_Range_File);
-
-	printf("[INFO][Loader] Node:%s, CMDUID:%s Execute: Assign_Loader_Range_File, role: %s, loading param:%s\n",
-			rd->describeNode().c_str(), req.cmdID.toString().c_str(),
-			getRoleStr(rd->localNodeStatus.role).c_str(),
-			param.toString().c_str());
-
-	bc = rd->bc;
-	// printf("[INFO] Node:%s CMDUID:%s open backup container for url:%s\n",
-	// 		rd->describeNode().c_str(), req.cmdID.toString().c_str(),
-	// 		param.url.toString().c_str());
-
-
-	rd->kvOps.clear(); //Clear kvOps so that kvOps only hold mutations for the current data block. We will send all mutations in kvOps to applier
-	rd->mutationMap.clear();
-	rd->mutationPartMap.clear();
-
-	ASSERT( param.blockSize > 0 );
-	//state std::vector<Future<Void>> fileParserFutures;
-	if (param.offset % param.blockSize != 0) {
-		printf("[WARNING] Parse file not at block boundary! param.offset:%ld param.blocksize:%ld, remainder:%ld\n",
-				param.offset, param.blockSize, param.offset % param.blockSize);
-	}
-	for (j = param.offset; j < param.length; j += param.blockSize) {
-		readOffset = j;
-		readLen = std::min<int64_t>(param.blockSize, param.length - j);
-		printf("[DEBUG_TMP] _parseRangeFileToMutationsOnLoader starts\n");
-		wait( _parseRangeFileToMutationsOnLoader(rd, bc, param.version, param.filename, readOffset, readLen, param.restoreRange, param.addPrefix, param.removePrefix) );
-		printf("[DEBUG_TMP] _parseRangeFileToMutationsOnLoader ends\n");
-		++beginBlock;
-	}
-
-	printf("[INFO][Loader] Node:%s CMDUID:%s finishes process Range file:%s\n",
-			rd->describeNode().c_str(), req.cmdID.toString().c_str(),
-			param.filename.c_str());
-	// TODO: Send to applier to apply the mutations
-	// printf("[INFO][Loader] Node:%s CMDUID:%s will send range mutations to applier\n",
-	// 		rd->describeNode().c_str(), rd->cmdID.toString().c_str());
-	wait( registerMutationsToApplier(rd) ); // Send the parsed mutation to applier who will apply the mutation to DB
-	wait ( delay(1.0) );
-	
-	rd->processedFiles[param.filename] =  1;
-	rd->processedCmd[req.cmdID] = 1;
-
-	rd->clearInProgressFlag(RestoreCommandEnum::Assign_Loader_Range_File);
-	printf("[INFO][Loader] Node:%s CMDUID:%s clear inProgressFlag :%lx for Assign_Loader_Range_File.\n",
-			rd->describeNode().c_str(), req.cmdID.toString().c_str(), rd->inProgressFlag);
-
-	//Send ack to master that loader has finished loading the data
-	printf("[INFO][Loader] Node:%s CMDUID:%s send ack.\n",
-			rd->describeNode().c_str(), rd->cmdID.toString().c_str());
-	req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-
-	return Void();
-
-}
-
-
-ACTOR Future<Void> handleLoadLogFileRequest(RestoreLoadFileRequest req, Reference<RestoreData> rd, RestoreInterface interf) {
-	printf("[INFO] Worker Node:%s starts handleLoadLogFileRequest\n", rd->describeNode().c_str());
-
-	state LoadingParam param;
-	state int64_t beginBlock = 0;
-	state int64_t j = 0;
-	state int64_t readLen = 0;
-	state int64_t readOffset = 0;
-	state Reference<IBackupContainer> bc;
-
-	param = req.param;
-	beginBlock = 0;
-	j = 0;
-	readLen = 0;
-	readOffset = 0;
-	readOffset = param.offset;
-
-	while (rd->isInProgress(RestoreCommandEnum::Assign_Loader_Log_File)) {
-		printf("[DEBUG] NODE:%s loadLogFile wait for 5s\n",  rd->describeNode().c_str());
-		wait(delay(5.0));
-	}
-	
-	//Note: handle duplicate message delivery
-	if (rd->processedFiles.find(param.filename) != rd->processedFiles.end()
-	   || rd->isCmdProcessed(req.cmdID)) {
-		printf("[WARNING] Node:%s CMDUID:%s file:%s is delivered more than once! Reply directly without loading the file\n",
-				rd->describeNode().c_str(), req.cmdID.toString().c_str(),
-				param.filename.c_str());
-		req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-		return Void();
-	}
-
-	rd->setInProgressFlag(RestoreCommandEnum::Assign_Loader_Log_File);
-
-	printf("[INFO][Loader] Node:%s CMDUID:%s Assign_Loader_Log_File role: %s, loading param:%s\n",
-								rd->describeNode().c_str(), req.cmdID.toString().c_str(),
-								getRoleStr(rd->localNodeStatus.role).c_str(),
-								param.toString().c_str());
-
-	bc = rd->bc;
-	printf("[INFO][Loader] Node:%s CMDUID:%s open backup container for url:%s\n",
-			rd->describeNode().c_str(), req.cmdID.toString().c_str(),
-			param.url.toString().c_str());
-	printf("[INFO][Loader] Node:%s CMDUID:%s filename:%s blockSize:%ld\n",
-			rd->describeNode().c_str(), req.cmdID.toString().c_str(),
-			param.filename.c_str(), param.blockSize);
-
-	rd->kvOps.clear(); //Clear kvOps so that kvOps only hold mutations for the current data block. We will send all mutations in kvOps to applier
-	rd->mutationMap.clear();
-	rd->mutationPartMap.clear();
-
-	ASSERT( param.blockSize > 0 );
-	//state std::vector<Future<Void>> fileParserFutures;
-	if (param.offset % param.blockSize != 0) {
-		printf("[WARNING] Parse file not at block boundary! param.offset:%ld param.blocksize:%ld, remainder:%ld\n",
-				param.offset, param.blockSize, param.offset % param.blockSize);
-	}
-	for (j = param.offset; j < param.length; j += param.blockSize) {
-		readOffset = j;
-		readLen = std::min<int64_t>(param.blockSize, param.length - j);
-		// NOTE: Log file holds set of blocks of data. We need to parse the data block by block and get the kv pair(version, serialized_mutations)
-		// The set of mutations at the same version may be splitted into multiple kv pairs ACROSS multiple data blocks when the size of serialized_mutations is larger than 20000.
-		wait( _parseLogFileToMutationsOnLoader(rd, bc, param.version, param.filename, readOffset, readLen, param.restoreRange, param.addPrefix, param.removePrefix, param.mutationLogPrefix) );
-		++beginBlock;
-	}
-	printf("[INFO][Loader] Node:%s CMDUID:%s finishes parsing the data block into kv pairs (version, serialized_mutations) for file:%s\n",
-			rd->describeNode().c_str(), req.cmdID.toString().c_str(),
-			param.filename.c_str());
-	parseSerializedMutation(rd, false);
-
-	printf("[INFO][Loader] Node:%s CMDUID:%s finishes process Log file:%s\n",
-			rd->describeNode().c_str(), req.cmdID.toString().c_str(),
-			param.filename.c_str());
-	printf("[INFO][Loader] Node:%s CMDUID:%s will send log mutations to applier\n",
-			rd->describeNode().c_str(), req.cmdID.toString().c_str());
-	wait( registerMutationsToApplier(rd) ); // Send the parsed mutation to applier who will apply the mutation to DB
-
-	req.reply.send(RestoreCommonReply(interf.id(), req.cmdID)); // master node is waiting
-	rd->processedFiles[param.filename] =  1;
-	rd->processedCmd[req.cmdID] = 1;
-
-	rd->clearInProgressFlag(RestoreCommandEnum::Assign_Loader_Log_File);
-	
-	return Void();
-}
-
-// Applier receive mutation from loader
-ACTOR Future<Void> handleSendMutationVectorRequest(RestoreSendMutationVectorRequest req, Reference<RestoreData> rd, RestoreInterface interf) {
-	state int numMutations = 0;
-
-	//wait( delay(1.0) ); //Q: Why adding this delay will cause segmentation fault?
-	if ( debug_verbose ) {
-		printf("[VERBOSE_DEBUG] Node:%s receive mutation number:%d\n", rd->describeNode().c_str(), req.mutations.size());
-	}
-
-	// NOTE: We have insert operation to rd->kvOps. For the same worker, we should only allow one actor of this kind to run at any time!
-	// Otherwise, race condition may happen!
-	while (rd->isInProgress(RestoreCommandEnum::Loader_Send_Mutations_To_Applier)) {
-		printf("[DEBUG] NODE:%s sendMutation wait for 1s\n",  rd->describeNode().c_str());
-		wait(delay(1.0));
-	}
-
-	// Handle duplicat cmd
-	if ( rd->isCmdProcessed(req.cmdID) ) {
-		//printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", rd->describeNode().c_str(), req.cmdID.toString().c_str());
-		//printf("[DEBUG] Skipped mutation:%s\n", req.mutation.toString().c_str());
-		req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));	
-		return Void();
-	}
-	rd->setInProgressFlag(RestoreCommandEnum::Loader_Send_Mutations_To_Applier);
-
-	// Applier will cache the mutations at each version. Once receive all mutations, applier will apply them to DB
-	state uint64_t commitVersion = req.commitVersion;
-	VectorRef<MutationRef> mutations(req.mutations);
-	printf("[DEBUG] Node:%s receive %d mutations at version:%ld\n", rd->describeNode().c_str(), mutations.size(), commitVersion);
-	if ( rd->kvOps.find(commitVersion) == rd->kvOps.end() ) {
-		rd->kvOps.insert(std::make_pair(commitVersion, VectorRef<MutationRef>()));
-	}
-	state int mIndex = 0;
-	for (mIndex = 0; mIndex < mutations.size(); mIndex++) {
-		MutationRef mutation = mutations[mIndex];
-		rd->kvOps[commitVersion].push_back_deep(rd->kvOps[commitVersion].arena(), mutation);
-		numMutations++;
-		if ( numMutations % 100000 == 1 ) { // Should be different value in simulation and in real mode
-			printf("[INFO][Applier] Node:%s Receives %d mutations. cur_mutation:%s\n",
-					rd->describeNode().c_str(), numMutations, mutation.toString().c_str());
-		}
-	}
-	
-	req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-	// Avoid race condition when this actor is called twice on the same command
-	rd->processedCmd[req.cmdID] = 1;
-	rd->clearInProgressFlag(RestoreCommandEnum::Loader_Send_Mutations_To_Applier);
-
-	return Void();
-}
-
-ACTOR Future<Void> handleSendSampleMutationVectorRequest(RestoreSendMutationVectorRequest req, Reference<RestoreData> rd, RestoreInterface interf) {
-	state int numMutations = 0;
-	rd->numSampledMutations = 0;
-	//wait( delay(1.0) );
-	//ASSERT(req.cmd == (RestoreCommandEnum) req.cmdID.phase);
-
-	// NOTE: We have insert operation to rd->kvOps. For the same worker, we should only allow one actor of this kind to run at any time!
-	// Otherwise, race condition may happen!
-	while (rd->isInProgress(RestoreCommandEnum::Loader_Send_Sample_Mutation_To_Applier)) {
-		printf("[DEBUG] NODE:%s handleSendSampleMutationVectorRequest wait for 1s\n",  rd->describeNode().c_str());
-		wait(delay(1.0));
-	}
-
-	// Handle duplicate message
-	if (rd->isCmdProcessed(req.cmdID)) {
-		printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", rd->describeNode().c_str(), req.cmdID.toString().c_str());
-		req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-		return Void();
-	}
-	rd->setInProgressFlag(RestoreCommandEnum::Loader_Send_Sample_Mutation_To_Applier);
-
-	// Applier will cache the mutations at each version. Once receive all mutations, applier will apply them to DB
-	state uint64_t commitVersion = req.commitVersion;
-	// TODO: Change the req.mutation to a vector of mutations
-	VectorRef<MutationRef> mutations(req.mutations);
-
-	state int mIndex = 0;
-	for (mIndex = 0; mIndex < mutations.size(); mIndex++) {
-		MutationRef mutation = mutations[mIndex];
-		if ( rd->keyOpsCount.find(mutation.param1) == rd->keyOpsCount.end() ) {
-			rd->keyOpsCount.insert(std::make_pair(mutation.param1, 0));
-		}
-		// NOTE: We may receive the same mutation more than once due to network package lost.
-		// Since sampling is just an estimation and the network should be stable enough, we do NOT handle the duplication for now
-		// In a very unreliable network, we may get many duplicate messages and get a bad key-range splits for appliers. But the restore should still work except for running slower.
-		rd->keyOpsCount[mutation.param1]++;
-		rd->numSampledMutations++;
-
-		if ( debug_verbose && rd->numSampledMutations % 1000 == 1 ) {
-			printf("[Sampling][Applier] Node:%s Receives %d sampled mutations. cur_mutation:%s\n",
-					rd->describeNode().c_str(), rd->numSampledMutations, mutation.toString().c_str());
-		}
-	}
-	
-	req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-	rd->processedCmd[req.cmdID] = 1;
-
-	rd->clearInProgressFlag(RestoreCommandEnum::Loader_Send_Sample_Mutation_To_Applier);
-
-	return Void();
-}
-
- ACTOR Future<Void> handleApplyToDBRequest(RestoreSimpleRequest req, Reference<RestoreData> rd, RestoreInterface interf, Database cx) {
- 	state bool isPrint = false; //Debug message
- 	state std::string typeStr = "";
-
-	// Wait in case the  applyToDB request was delivered twice;
-	while (rd->inProgressApplyToDB) {
-		printf("[DEBUG] NODE:%s inProgressApplyToDB wait for 5s\n",  rd->describeNode().c_str());
-		wait(delay(5.0));
-	}
-	
-	if ( rd->isCmdProcessed(req.cmdID) ) {
-		printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", rd->describeNode().c_str(), req.cmdID.toString().c_str());
-		req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-		return Void();
-	}
-
-	rd->inProgressApplyToDB = true;
-
-	// Assume the process will not crash when it apply mutations to DB. The reply message can be lost though
-	if (rd->kvOps.empty()) {
-		printf("Node:%s kvOps is empty. No-op for apply to DB\n", rd->describeNode().c_str());
-		req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-		rd->processedCmd[req.cmdID] = 1;
-		rd->inProgressApplyToDB = false;
-		return Void();
-	}
-	
-	sanityCheckMutationOps(rd);
-
- 	if ( debug_verbose ) {
-		TraceEvent("ApplyKVOPsToDB").detail("MapSize", rd->kvOps.size());
-		printf("ApplyKVOPsToDB num_of_version:%ld\n", rd->kvOps.size());
- 	}
- 	state std::map<Version, Standalone<VectorRef<MutationRef>>>::iterator it = rd->kvOps.begin();
-	state std::map<Version, Standalone<VectorRef<MutationRef>>>::iterator prevIt = it;
-	state int index = 0;
-	state int prevIndex = index;
- 	state int count = 0;
-	state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
-	state int numVersion = 0;
-	state double transactionSize = 0;
-	loop {
-		try {
-			tr->reset();
-			tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
-			tr->setOption(FDBTransactionOptions::LOCK_AWARE);
-			transactionSize = 0;
-
-			for ( ; it != rd->kvOps.end(); ++it ) {
-				numVersion++;
-				if ( debug_verbose ) {
-					TraceEvent("ApplyKVOPsToDB\t").detail("Version", it->first).detail("OpNum", it->second.size());
-				}
-				//printf("ApplyKVOPsToDB numVersion:%d Version:%08lx num_of_ops:%d, \n", numVersion, it->first, it->second.size());
-
-				state MutationRef m;
-				for ( ; index < it->second.size(); ++index ) {
-					m = it->second[index];
-					if (  m.type >= MutationRef::Type::SetValue && m.type <= MutationRef::Type::MAX_ATOMIC_OP )
-						typeStr = typeString[m.type];
-					else {
-						printf("ApplyKVOPsToDB MutationType:%d is out of range\n", m.type);
-					}
-
-					if ( debug_verbose && count % 1000 == 1 ) {
-						printf("ApplyKVOPsToDB Node:%s num_mutation:%d Version:%08lx num_of_ops:%d\n",
-								rd->describeNode().c_str(), count, it->first, it->second.size());
-					}
-
-					if ( debug_verbose ) {
-						printf("[VERBOSE_DEBUG] Node:%s apply mutation:%s\n", rd->describeNode().c_str(), m.toString().c_str());
-					}
-
-					if ( m.type == MutationRef::SetValue ) {
-						tr->set(m.param1, m.param2);
-					} else if ( m.type == MutationRef::ClearRange ) {
-						KeyRangeRef mutationRange(m.param1, m.param2);
-						tr->clear(mutationRange);
-					} else if ( isAtomicOp((MutationRef::Type) m.type) ) {
-						//// Now handle atomic operation from this if statement
-						// TODO: Have not de-duplicated the mutations for multiple network delivery
-						// ATOMIC_MASK = (1 << AddValue) | (1 << And) | (1 << Or) | (1 << Xor) | (1 << AppendIfFits) | (1 << Max) | (1 << Min) | (1 << SetVersionstampedKey) | (1 << SetVersionstampedValue) | (1 << ByteMin) | (1 << ByteMax) | (1 << MinV2) | (1 << AndV2),
-						//atomicOp( const KeyRef& key, const ValueRef& operand, uint32_t operationType )
-						tr->atomicOp(m.param1, m.param2, m.type);
-					} else {
-						printf("[WARNING] mtype:%d (%s) unhandled\n", m.type, typeStr.c_str());
-					}
-					++count;
-					transactionSize += m.expectedSize();
-					
-					if ( transactionSize >= transactionBatchSizeThreshold ) { // commit per 1000 mutations
-						wait(tr->commit());
-						tr->reset();
-						tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
-						tr->setOption(FDBTransactionOptions::LOCK_AWARE);
-						prevIt = it;
-						prevIndex = index;
-						transactionSize = 0;
-					}
-
-					if ( isPrint ) {
-						printf("\tApplyKVOPsToDB Version:%016lx MType:%s K:%s, V:%s K_size:%d V_size:%d\n", it->first, typeStr.c_str(),
-							getHexString(m.param1).c_str(), getHexString(m.param2).c_str(), m.param1.size(), m.param2.size());
-
-						TraceEvent("ApplyKVOPsToDB\t\t").detail("Version", it->first)
-								.detail("MType", m.type).detail("MTypeStr", typeStr)
-								.detail("MKey", getHexString(m.param1))
-								.detail("MValueSize", m.param2.size())
-								.detail("MValue", getHexString(m.param2));
-					}
-				}
-				index = 0;
-			}
-			// Last transaction
-			if (transactionSize > 0) {
-				wait(tr->commit());
-			}
-			break;
-		} catch(Error &e) {
-			printf("ApplyKVOPsToDB transaction error:%s.\n", e.what());
-			wait(tr->onError(e));
-			it = prevIt;
-			index = prevIndex;
-			transactionSize = 0;
-		}
-	}
-
- 	rd->kvOps.clear();
- 	printf("Node:%s ApplyKVOPsToDB number of kv mutations:%d\n", rd->describeNode().c_str(), count);
-
-	req.reply.send(RestoreCommonReply(interf.id(), req.cmdID));
-	printf("rd->processedCmd size:%d req.cmdID:%s\n", rd->processedCmd.size(), req.cmdID.toString().c_str());
-	rd->processedCmd[req.cmdID] = 1;
-	rd->inProgressApplyToDB = false;
-
- 	return Void();
-}
-
-ACTOR Future<Void> workerCore(Reference<RestoreData> rd, RestoreInterface ri, Database cx) {
-	state ActorCollection actors(false);
+ACTOR Future<Void> startRestoreWorker(Reference<RestoreWorkerData> self, RestoreWorkerInterface interf, Database cx) {
 	state double lastLoopTopTime;
+	state ActorCollection actors(false); // Collect the main actor for each role
+	
 	loop {
-		
 		double loopTopTime = now();
 		double elapsedTime = loopTopTime - lastLoopTopTime;
 		if( elapsedTime > 0.050 ) {
 			if (g_random->random01() < 0.01)
-				TraceEvent(SevWarn, "SlowRestoreLoaderLoopx100").detail("NodeDesc", rd->describeNode()).detail("Elapsed", elapsedTime);
+				TraceEvent(SevWarn, "SlowRestoreLoaderLoopx100").detail("NodeDesc", self->describeNode()).detail("Elapsed", elapsedTime);
 		}
 		lastLoopTopTime = loopTopTime;
 		state std::string requestTypeStr = "[Init]";
 
 		try {
 			choose {
-				when ( RestoreSimpleRequest req = waitNext(ri.heartbeat.getFuture()) ) {
+				when ( RestoreSimpleRequest req = waitNext(interf.heartbeat.getFuture()) ) {
 					requestTypeStr = "heartbeat";
-					wait(handleHeartbeat(req, rd, ri));
+					actors.add( handleHeartbeat(req, interf.id()) );
 				}
-				when ( RestoreSetRoleRequest req = waitNext(ri.setRole.getFuture()) ) {
-					requestTypeStr = "setRole";
-					wait(handleSetRoleRequest(req, rd, ri));
+				when ( RestoreRecruitRoleRequest req = waitNext(interf.recruitRole.getFuture()) ) {
+					requestTypeStr = "recruitRole";
+					actors.add( handleRecruitRoleRequest(req, self, &actors, cx) );
 				}
-				when ( RestoreLoadFileRequest req = waitNext(ri.sampleRangeFile.getFuture()) ) {
-					requestTypeStr = "sampleRangeFile";
-					initBackupContainer(rd, req.param.url);
-					ASSERT(rd->getRole() == RestoreRole::Loader);
-					actors.add( handleSampleRangeFileRequest(req, rd, ri) );
-				}
-				when ( RestoreLoadFileRequest req = waitNext(ri.sampleLogFile.getFuture()) ) {
-					initBackupContainer(rd, req.param.url);
-					requestTypeStr = "sampleLogFile";
-					ASSERT(rd->getRole() == RestoreRole::Loader);
-					actors.add( handleSampleLogFileRequest(req, rd, ri) );
-				}
-				when ( RestoreGetApplierKeyRangeRequest req = waitNext(ri.getApplierKeyRangeRequest.getFuture()) ) {
-					requestTypeStr = "getApplierKeyRangeRequest";
-					wait(handleGetApplierKeyRangeRequest(req, rd, ri));	
-				}
-				when ( RestoreSetApplierKeyRangeRequest req = waitNext(ri.setApplierKeyRangeRequest.getFuture()) ) {
-					requestTypeStr = "setApplierKeyRangeRequest";
-					wait(handleSetApplierKeyRangeRequest(req, rd, ri));
-				}
-				when ( RestoreSetApplierKeyRangeVectorRequest req = waitNext(ri.setApplierKeyRangeVectorRequest.getFuture()) ) {
-					requestTypeStr = "setApplierKeyRangeVectorRequest";
-					wait(handleSetApplierKeyRangeVectorRequest(req, rd, ri));
-				}
-				when ( RestoreLoadFileRequest req = waitNext(ri.loadRangeFile.getFuture()) ) {
-					requestTypeStr = "loadRangeFile";
-					ASSERT(rd->getRole() == RestoreRole::Loader);
-					initBackupContainer(rd, req.param.url);
-					actors.add( handleLoadRangeFileRequest(req, rd, ri) );
-				}
-				when ( RestoreLoadFileRequest req = waitNext(ri.loadLogFile.getFuture()) ) {
-					requestTypeStr = "loadLogFile";
-					ASSERT(rd->getRole() == RestoreRole::Loader);
-					initBackupContainer(rd, req.param.url);
-					actors.add( handleLoadLogFileRequest(req, rd, ri) );
-				}
-
-				when ( RestoreCalculateApplierKeyRangeRequest req = waitNext(ri.calculateApplierKeyRange.getFuture()) ) {
-					requestTypeStr = "calculateApplierKeyRange";
-					ASSERT(rd->getRole() == RestoreRole::Applier);
-					wait(handleCalculateApplierKeyRangeRequest(req, rd, ri));
-				}
-				when ( RestoreSendMutationVectorRequest req = waitNext(ri.sendSampleMutationVector.getFuture()) ) {
-					requestTypeStr = "sendSampleMutationVector";
-					ASSERT(rd->getRole() == RestoreRole::Applier);
-					actors.add( handleSendSampleMutationVectorRequest(req, rd, ri));
-				} 
-				when ( RestoreSendMutationVectorRequest req = waitNext(ri.sendMutationVector.getFuture()) ) {
-					requestTypeStr = "sendMutationVector";
-					ASSERT(rd->getRole() == RestoreRole::Applier);
-					actors.add( handleSendMutationVectorRequest(req, rd, ri) );
-				}
-				when ( RestoreSimpleRequest req = waitNext(ri.applyToDB.getFuture()) ) {
-					requestTypeStr = "applyToDB";
-					actors.add( handleApplyToDBRequest(req, rd, ri, cx) );
-				}
-
-				when ( RestoreVersionBatchRequest req = waitNext(ri.initVersionBatch.getFuture()) ) {
-					requestTypeStr = "initVersionBatch";
-					wait(handleVersionBatchRequest(req, rd, ri));
-				}
-
-				when ( RestoreSimpleRequest req = waitNext(ri.setWorkerInterface.getFuture()) ) {
-					// Step: Find other worker's interfaces
-					// NOTE: This must be after wait(configureRolesHandler()) because we must ensure all workers have registered their interfaces into DB before we can read the interface.
-					// TODO: Wait until all workers have registered their interface.
-					wait( setWorkerInterface(req, rd, ri, cx) );
-				}
-
-				when ( RestoreSimpleRequest req = waitNext(ri.finishRestore.getFuture()) ) {
+				when ( RestoreSimpleRequest req = waitNext(interf.terminateWorker.getFuture()) ) {
 					// Destroy the worker at the end of the restore
 					// TODO: Cancel its own actors
-					wait( handleFinishRestoreReq(req, rd, ri, cx) );
+					requestTypeStr = "terminateWorker";
+					actors.add( handlerTerminateWorkerRequest(req, self, interf, cx) );
 					return Void();
 				}
 			}
 
 		} catch (Error &e) {
-			// TODO: Handle the command reply timeout error
-			if (e.code() != error_code_io_timeout) {
-				fprintf(stdout, "[ERROR] Loader handle received request:%s timeout\n", requestTypeStr.c_str());
-			} else {
-				fprintf(stdout, "[ERROR] Loader handle received request:%s error. error code:%d, error message:%s\n",
-						requestTypeStr.c_str(), e.code(), e.what());
-			}
-
+			fprintf(stdout, "[ERROR] Loader handle received request:%s error. error code:%d, error message:%s\n",
+					requestTypeStr.c_str(), e.code(), e.what());
 			if ( requestTypeStr.find("[Init]") != std::string::npos ) {
 				printf("Exit due to error at requestType:%s", requestTypeStr.c_str());
 				break;
@@ -4094,62 +415,89 @@ ACTOR Future<Void> workerCore(Reference<RestoreData> rd, RestoreInterface ri, Da
 	return Void();
 }
 
-ACTOR Future<Void> masterCore(Reference<RestoreData> rd, RestoreInterface interf, Database cx) {
-	//we are the leader
-	// We must wait for enough time to make sure all restore workers have registered their interfaces into the DB
-	printf("[INFO][Master] NodeID:%s Restore master waits for agents to register their workerKeys\n",
-			interf.id().toString().c_str());
-	wait( delay(10.0) );
+ACTOR Future<Void> _restoreWorker(Database cx_input, LocalityData locality) {
+	state Database cx = cx_input;
+	state RestoreWorkerInterface workerInterf;
+	workerInterf.initEndpoints();
+	state Optional<RestoreWorkerInterface> leaderInterf;
+	//Global data for the worker
+	state Reference<RestoreWorkerData> self = Reference<RestoreWorkerData>(new RestoreWorkerData());
 
-	rd->localNodeStatus.init(RestoreRole::Master);
-	rd->localNodeStatus.nodeID = interf.id();
-	printf("[INFO][Master]  NodeID:%s starts configuring roles for workers\n", interf.id().toString().c_str());
+	self->workerID = workerInterf.id();
 
-	wait( collectWorkerInterface(rd, cx, MIN_NUM_WORKERS) );
+	initRestoreWorkerConfig(); //TODO: Change to a global struct to store the restore configuration
 
-	Future<Void> workersFailureMonitor = monitorWorkerLiveness(rd);
-
-	// configureRoles must be after collectWorkerInterface
-	// Why do I need to put an extra wait() to make sure the above wait is executed after the below wwait?
-	wait( delay(1.0) );
-
-	wait( configureRoles(rd) );
-
-	wait( delay(1.0) );
-	wait( notifyWorkersToSetWorkersInterface(rd) );
-
-	state int restoreId = 0;
-	state int checkNum = 0;
+	// Compete in registering its restoreInterface as the leader.
+	state Transaction tr(cx);
 	loop {
-		printf("Node:%s---Wait on restore requests...---\n", rd->describeNode().c_str());
-		state Standalone<VectorRef<RestoreRequest>> restoreRequests = wait( collectRestoreRequests(cx) );
-
-		printf("Node:%s ---Received  restore requests as follows---\n", rd->describeNode().c_str());
-		// Print out the requests info
-		for ( auto &it : restoreRequests ) {
-			printf("\t[INFO][Master]Node:%s RestoreRequest info:%s\n", rd->describeNode().c_str(), it.toString().c_str());
+		try {
+			tr.reset();
+			tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
+			tr.setOption(FDBTransactionOptions::LOCK_AWARE);
+			Optional<Value> leader = wait(tr.get(restoreLeaderKey));
+			if(leader.present()) {
+				leaderInterf = BinaryReader::fromStringRef<RestoreWorkerInterface>(leader.get(), IncludeVersion());
+				// NOTE: Handle the situation that the leader's commit of its key causes error(commit_unknown_result)
+				// In this situation, the leader will try to register its key again, which will never succeed.
+				// We should let leader escape from the infinite loop
+				if ( leaderInterf.get().id() == workerInterf.id() ) {
+					printf("[Worker] NodeID:%s is the leader and has registered its key in commit_unknown_result error. Let it set the key again\n",
+							leaderInterf.get().id().toString().c_str());
+					tr.set(restoreLeaderKey, BinaryWriter::toValue(workerInterf, IncludeVersion()));
+					wait(tr.commit());
+					 // reset leaderInterf to invalid for the leader process
+					 // because a process will not execute leader's logic unless leaderInterf is invalid
+					leaderInterf = Optional<RestoreWorkerInterface>();
+					break;
+				}
+				printf("[Worker] Leader key exists:%s. Worker registers its restore workerInterface id:%s\n",
+						leaderInterf.get().id().toString().c_str(), workerInterf.id().toString().c_str());
+				tr.set(restoreWorkerKeyFor(workerInterf.id()), restoreWorkerInterfaceValue(workerInterf));
+				wait(tr.commit());
+				break;
+			}
+			printf("[Worker] NodeID:%s competes register its workerInterface as leader\n", workerInterf.id().toString().c_str());
+			tr.set(restoreLeaderKey, BinaryWriter::toValue(workerInterf, IncludeVersion()));
+			wait(tr.commit());
+			break;
+		} catch( Error &e ) {
+			// We may have error commit_unknown_result, the commit may or may not succeed!
+			// We must handle this error, otherwise, if the leader does not know its key has been registered, the leader will stuck here!
+			printf("[INFO] NodeID:%s restoreWorker select leader error, error code:%d error info:%s\n",
+					workerInterf.id().toString().c_str(), e.code(), e.what());
+			wait( tr.onError(e) );
 		}
+	}
 
-		// Step: Perform the restore requests
-		for ( auto &it : restoreRequests ) {
-			TraceEvent("LeaderGotRestoreRequest").detail("RestoreRequestInfo", it.toString());
-			printf("Node:%s Got RestoreRequestInfo:%s\n", rd->describeNode().c_str(), it.toString().c_str());
-			Version ver = wait( processRestoreRequest(interf, rd, cx, it) );
-		}
+	
+	if(leaderInterf.present()) { // Logic for restoer workers (restore loader and restore applier)
+		wait( startRestoreWorker(self, workerInterf, cx) );
+	} else { // Logic for restore master
+		self->masterData = Reference<RestoreMasterData>(new RestoreMasterData());
+		// We must wait for enough time to make sure all restore workers have registered their workerInterfaces into the DB
+		printf("[INFO][Master] NodeID:%s Restore master waits for agents to register their workerKeys\n",
+				workerInterf.id().toString().c_str());
+		wait( delay(10.0) );
 
-		// Step: Notify all restore requests have been handled by cleaning up the restore keys
-		wait( delay(5.0) );
-		printf("Finish my restore now!\n");
-		//wait( finishRestore(rd) );
-		wait( finishRestore(rd, cx, restoreRequests) ); 
+		printf("[INFO][Master]  NodeID:%s starts configuring roles for workers\n", workerInterf.id().toString().c_str());
 
-		printf("[INFO] MXRestoreEndHere RestoreID:%d\n", restoreId);
-		TraceEvent("MXRestoreEndHere").detail("RestoreID", restoreId++);
-		wait( delay(5.0) );
-		//NOTE: we have to break the loop so that the tester.actor can receive the return of this test workload.
-		//Otherwise, this special workload never returns and tester will think the test workload is stuck and the tester will timesout
-		break; //TODO: this break will be removed later since we need the restore agent to run all the time!
+		wait( collectRestoreWorkerInterface(self, cx, MIN_NUM_WORKERS) );
+
+		state Future<Void> workersFailureMonitor = monitorWorkerLiveness(self);
+
+		// configureRoles must be after collectWorkerInterface
+		// TODO: remove the delay() Why do I need to put an extra wait() to make sure the above wait is executed after the below wwait?
+		wait( delay(1.0) );
+		wait( recruitRestoreRoles(self) );
+
+		wait( startRestoreMaster(self->masterData, cx) );
 	}
 
+	return Void();
+}
+
+ACTOR Future<Void> restoreWorker(Reference<ClusterConnectionFile> ccf, LocalityData locality) {
+	Database cx = Database::createDatabase(ccf->getFilename(), Database::API_VERSION_LATEST,locality);
+	wait(_restoreWorker(cx, locality));
 	return Void();
 }
\ No newline at end of file
diff --git a/fdbserver/RestoreApplier.actor.cpp b/fdbserver/RestoreApplier.actor.cpp
index e9019ea056..a6614d6661 100644
--- a/fdbserver/RestoreApplier.actor.cpp
+++ b/fdbserver/RestoreApplier.actor.cpp
@@ -17,3 +17,453 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+
+#include "fdbclient/NativeAPI.actor.h"
+#include "fdbclient/SystemData.h"
+
+// Backup agent header
+#include "fdbclient/BackupAgent.actor.h"
+//#include "FileBackupAgent.h"
+#include "fdbclient/ManagementAPI.actor.h"
+#include "fdbclient/MutationList.h"
+#include "fdbclient/BackupContainer.h"
+
+#include "fdbserver/RestoreCommon.actor.h"
+#include "fdbserver/RestoreUtil.h"
+#include "fdbserver/RestoreRoleCommon.actor.h"
+#include "fdbserver/RestoreApplier.actor.h"
+
+
+#include "flow/actorcompiler.h"  // This must be the last #include.
+
+ACTOR Future<Void> handleGetApplierKeyRangeRequest(RestoreGetApplierKeyRangeRequest req, Reference<RestoreApplierData> self);
+ACTOR Future<Void> handleSetApplierKeyRangeRequest(RestoreSetApplierKeyRangeRequest req, Reference<RestoreApplierData> self);
+ACTOR Future<Void> handleCalculateApplierKeyRangeRequest(RestoreCalculateApplierKeyRangeRequest req, Reference<RestoreApplierData> self);
+ACTOR Future<Void> handleSendSampleMutationVectorRequest(RestoreSendMutationVectorRequest req, Reference<RestoreApplierData> self);
+ACTOR Future<Void> handleSendMutationVectorRequest(RestoreSendMutationVectorRequest req, Reference<RestoreApplierData> self);
+ACTOR Future<Void> handleApplyToDBRequest(RestoreSimpleRequest req, Reference<RestoreApplierData> self, Database cx);
+
+
+ACTOR Future<Void> restoreApplierCore(Reference<RestoreApplierData> self, RestoreApplierInterface applierInterf, Database cx) {
+	state ActorCollection actors(false);
+	state double lastLoopTopTime;
+	loop {
+		
+		double loopTopTime = now();
+		double elapsedTime = loopTopTime - lastLoopTopTime;
+		if( elapsedTime > 0.050 ) {
+			if (g_random->random01() < 0.01)
+				TraceEvent(SevWarn, "SlowRestoreLoaderLoopx100").detail("NodeDesc", self->describeNode()).detail("Elapsed", elapsedTime);
+		}
+		lastLoopTopTime = loopTopTime;
+		state std::string requestTypeStr = "[Init]";
+
+		try {
+			choose {
+				when ( RestoreSimpleRequest req = waitNext(applierInterf.heartbeat.getFuture()) ) {
+					requestTypeStr = "heartbeat";
+					wait(handleHeartbeat(req, applierInterf.id()));
+				}
+				when ( RestoreGetApplierKeyRangeRequest req = waitNext(applierInterf.getApplierKeyRangeRequest.getFuture()) ) {
+					requestTypeStr = "getApplierKeyRangeRequest";
+					wait(handleGetApplierKeyRangeRequest(req, self));
+				}
+				when ( RestoreSetApplierKeyRangeRequest req = waitNext(applierInterf.setApplierKeyRangeRequest.getFuture()) ) {
+					requestTypeStr = "setApplierKeyRangeRequest";
+					wait(handleSetApplierKeyRangeRequest(req, self));
+				}
+
+				when ( RestoreCalculateApplierKeyRangeRequest req = waitNext(applierInterf.calculateApplierKeyRange.getFuture()) ) {
+					requestTypeStr = "calculateApplierKeyRange";
+					wait(handleCalculateApplierKeyRangeRequest(req, self));
+				}
+				when ( RestoreSendMutationVectorRequest req = waitNext(applierInterf.sendSampleMutationVector.getFuture()) ) {
+					requestTypeStr = "sendSampleMutationVector";
+					actors.add( handleSendSampleMutationVectorRequest(req, self));
+				} 
+				when ( RestoreSendMutationVectorRequest req = waitNext(applierInterf.sendMutationVector.getFuture()) ) {
+					requestTypeStr = "sendMutationVector";
+					actors.add( handleSendMutationVectorRequest(req, self) );
+				}
+				when ( RestoreSimpleRequest req = waitNext(applierInterf.applyToDB.getFuture()) ) {
+					requestTypeStr = "applyToDB";
+					actors.add( handleApplyToDBRequest(req, self, cx) );
+				}
+
+				when ( RestoreVersionBatchRequest req = waitNext(applierInterf.initVersionBatch.getFuture()) ) {
+					requestTypeStr = "initVersionBatch";
+					wait(handleInitVersionBatchRequest(req, self));
+				}
+
+				// TODO: To modify the interface for the following 2 when condition
+				when ( RestoreSimpleRequest req = waitNext(applierInterf.collectRestoreRoleInterfaces.getFuture()) ) {
+					// Step: Find other worker's workerInterfaces
+					// NOTE: This must be after wait(configureRolesHandler()) because we must ensure all workers have registered their workerInterfaces into DB before we can read the workerInterface.
+					// TODO: Wait until all workers have registered their workerInterface.
+					wait( handleCollectRestoreRoleInterfaceRequest(req, self, cx) );
+				}
+			}
+
+		} catch (Error &e) {
+			fprintf(stdout, "[ERROR] Loader handle received request:%s error. error code:%d, error message:%s\n",
+					requestTypeStr.c_str(), e.code(), e.what());
+
+			if ( requestTypeStr.find("[Init]") != std::string::npos ) {
+				printf("Exit due to error at requestType:%s", requestTypeStr.c_str());
+				break;
+			}
+		}
+	}
+
+	return Void();
+}
+
+
+
+ACTOR Future<Void> handleCalculateApplierKeyRangeRequest(RestoreCalculateApplierKeyRangeRequest req, Reference<RestoreApplierData> self) {
+	state int numMutations = 0;
+	state std::vector<Standalone<KeyRef>> keyRangeLowerBounds;
+
+	while (self->isInProgress(RestoreCommandEnum::Calculate_Applier_KeyRange)) {
+		printf("[DEBUG] NODE:%s Calculate_Applier_KeyRange wait for 5s\n",  self->describeNode().c_str());
+		wait(delay(5.0));
+	}
+
+	wait( delay(1.0) );
+	// Handle duplicate message
+	// We need to recalculate the value for duplicate message! Because the reply to duplicate message may arrive earlier!
+	if (self->isCmdProcessed(req.cmdID) && !keyRangeLowerBounds.empty() ) {
+		printf("[DEBUG] Node:%s skip duplicate cmd:%s\n", self->describeNode().c_str(), req.cmdID.toString().c_str());
+		req.reply.send(GetKeyRangeNumberReply(keyRangeLowerBounds.size()));
+		return Void();
+	}
+	self->setInProgressFlag(RestoreCommandEnum::Calculate_Applier_KeyRange);
+
+	// Applier will calculate applier key range
+	printf("[INFO][Applier] CMD:%s, Node:%s Calculate key ranges for %d appliers\n",
+			req.cmdID.toString().c_str(), self->describeNode().c_str(), req.numAppliers);
+
+	//ASSERT(req.cmd == (RestoreCommandEnum) req.cmdID.phase);
+	if ( keyRangeLowerBounds.empty() ) {
+		keyRangeLowerBounds = self->calculateAppliersKeyRanges(req.numAppliers); // keyRangeIndex is the number of key ranges requested
+		self->keyRangeLowerBounds = keyRangeLowerBounds;
+	}
+	
+	printf("[INFO][Applier] CMD:%s, NodeID:%s: num of key ranges:%ld\n",
+			req.cmdID.toString().c_str(), self->describeNode().c_str(), keyRangeLowerBounds.size());
+	req.reply.send(GetKeyRangeNumberReply(keyRangeLowerBounds.size()));
+	self->processedCmd[req.cmdID] = 1; // We should not skip this command in the following phase. Otherwise, the handler in other phases may return a wrong number of appliers
+	self->clearInProgressFlag(RestoreCommandEnum::Calculate_Applier_KeyRange);
+
+	return Void();
+}
+
+ACTOR Future<Void> handleGetApplierKeyRangeRequest(RestoreGetApplierKeyRangeRequest req, Reference<RestoreApplierData> self) {
+	state int numMutations = 0;
+	//state std::vector<Standalone<KeyRef>> keyRangeLowerBounds = self->keyRangeLowerBounds;
+
+	while (self->isInProgress(RestoreCommandEnum::Get_Applier_KeyRange)) {
+		printf("[DEBUG] NODE:%s Calculate_Applier_KeyRange wait for 5s\n",  self->describeNode().c_str());
+		wait(delay(5.0));
+	}
+
+	wait( delay(1.0) );
+	//NOTE: Must reply a valid lowerBound and upperBound! Otherwise, the master will receive an invalid value!
+	// if (self->isCmdProcessed(req.cmdID) ) {
+	// 	printf("[DEBUG] Node:%s skip duplicate cmd:%s\n", self->describeNode().c_str(), req.cmdID.toString().c_str());
+	// 	req.reply.send(GetKeyRangeReply(workerInterf.id(), req.cmdID)); // Must wait until the previous command returns
+	// 	return Void();
+	// }
+	self->setInProgressFlag(RestoreCommandEnum::Get_Applier_KeyRange);
+	
+	if ( req.applierIndex < 0 || req.applierIndex >= self->keyRangeLowerBounds.size() ) {
+		printf("[INFO][Applier] NodeID:%s Get_Applier_KeyRange keyRangeIndex is out of range. keyIndex:%d keyRagneSize:%ld\n",
+				self->describeNode().c_str(), req.applierIndex,  self->keyRangeLowerBounds.size());
+	}
+	//ASSERT(req.cmd == (RestoreCommandEnum) req.cmdID.phase);
+
+	printf("[INFO][Applier] NodeID:%s replies Get_Applier_KeyRange. keyRangeIndex:%d lower_bound_of_keyRange:%s\n",
+			self->describeNode().c_str(), req.applierIndex, getHexString(self->keyRangeLowerBounds[req.applierIndex]).c_str());
+
+	KeyRef lowerBound = self->keyRangeLowerBounds[req.applierIndex];
+	KeyRef upperBound = (req.applierIndex + 1) < self->keyRangeLowerBounds.size() ? self->keyRangeLowerBounds[req.applierIndex+1] : normalKeys.end;
+
+	req.reply.send(GetKeyRangeReply(self->id(), req.cmdID, req.applierIndex, lowerBound, upperBound));
+	self->clearInProgressFlag(RestoreCommandEnum::Get_Applier_KeyRange);
+
+	return Void();
+
+}
+
+// Assign key range to applier
+ACTOR Future<Void> handleSetApplierKeyRangeRequest(RestoreSetApplierKeyRangeRequest req, Reference<RestoreApplierData> self) {
+	// Idempodent operation. OK to re-execute the duplicate cmd
+	// The applier should remember the key range it is responsible for
+	//ASSERT(req.cmd == (RestoreCommandEnum) req.cmdID.phase);
+	//self->applierStatus.keyRange = req.range;
+	while (self->isInProgress(RestoreCommandEnum::Assign_Applier_KeyRange)) {
+		printf("[DEBUG] NODE:%s handleSetApplierKeyRangeRequest wait for 1s\n",  self->describeNode().c_str());
+		wait(delay(1.0));
+	}
+	if ( self->isCmdProcessed(req.cmdID) ) {
+		req.reply.send(RestoreCommonReply(self->id(),req.cmdID));
+		return Void();
+	}
+	self->setInProgressFlag(RestoreCommandEnum::Assign_Applier_KeyRange);
+
+	self->range2Applier[req.range.begin] = req.applierID;
+
+	self->processedCmd[req.cmdID] = 1;
+	self->clearInProgressFlag(RestoreCommandEnum::Assign_Applier_KeyRange);
+
+	req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+
+	return Void();
+}
+
+
+
+// Applier receive mutation from loader
+ACTOR Future<Void> handleSendMutationVectorRequest(RestoreSendMutationVectorRequest req, Reference<RestoreApplierData> self) {
+	state int numMutations = 0;
+
+	//wait( delay(1.0) ); //Q: Why adding this delay will cause segmentation fault?
+	if ( debug_verbose ) {
+		printf("[VERBOSE_DEBUG] Node:%s receive mutation number:%d\n", self->describeNode().c_str(), req.mutations.size());
+	}
+
+	// NOTE: We have insert operation to self->kvOps. For the same worker, we should only allow one actor of this kind to run at any time!
+	// Otherwise, race condition may happen!
+	while (self->isInProgress(RestoreCommandEnum::Loader_Send_Mutations_To_Applier)) {
+		printf("[DEBUG] NODE:%s sendMutation wait for 1s\n",  self->describeNode().c_str());
+		wait(delay(1.0));
+	}
+
+	// Handle duplicat cmd
+	if ( self->isCmdProcessed(req.cmdID) ) {
+		//printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", self->describeNode().c_str(), req.cmdID.toString().c_str());
+		//printf("[DEBUG] Skipped mutation:%s\n", req.mutation.toString().c_str());
+		req.reply.send(RestoreCommonReply(self->id(), req.cmdID));	
+		return Void();
+	}
+	self->setInProgressFlag(RestoreCommandEnum::Loader_Send_Mutations_To_Applier);
+
+	// Applier will cache the mutations at each version. Once receive all mutations, applier will apply them to DB
+	state uint64_t commitVersion = req.commitVersion;
+	VectorRef<MutationRef> mutations(req.mutations);
+	printf("[DEBUG] Node:%s receive %d mutations at version:%ld\n", self->describeNode().c_str(), mutations.size(), commitVersion);
+	if ( self->kvOps.find(commitVersion) == self->kvOps.end() ) {
+		self->kvOps.insert(std::make_pair(commitVersion, VectorRef<MutationRef>()));
+	}
+	state int mIndex = 0;
+	for (mIndex = 0; mIndex < mutations.size(); mIndex++) {
+		MutationRef mutation = mutations[mIndex];
+		self->kvOps[commitVersion].push_back_deep(self->kvOps[commitVersion].arena(), mutation);
+		numMutations++;
+		if ( numMutations % 100000 == 1 ) { // Should be different value in simulation and in real mode
+			printf("[INFO][Applier] Node:%s Receives %d mutations. cur_mutation:%s\n",
+					self->describeNode().c_str(), numMutations, mutation.toString().c_str());
+		}
+	}
+	
+	req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+	// Avoid race condition when this actor is called twice on the same command
+	self->processedCmd[req.cmdID] = 1;
+	self->clearInProgressFlag(RestoreCommandEnum::Loader_Send_Mutations_To_Applier);
+
+	return Void();
+}
+
+ACTOR Future<Void> handleSendSampleMutationVectorRequest(RestoreSendMutationVectorRequest req, Reference<RestoreApplierData> self) {
+	state int numMutations = 0;
+	self->numSampledMutations = 0;
+	//wait( delay(1.0) );
+	//ASSERT(req.cmd == (RestoreCommandEnum) req.cmdID.phase);
+
+	// NOTE: We have insert operation to self->kvOps. For the same worker, we should only allow one actor of this kind to run at any time!
+	// Otherwise, race condition may happen!
+	while (self->isInProgress(RestoreCommandEnum::Loader_Send_Sample_Mutation_To_Applier)) {
+		printf("[DEBUG] NODE:%s handleSendSampleMutationVectorRequest wait for 1s\n",  self->describeNode().c_str());
+		wait(delay(1.0));
+	}
+
+	// Handle duplicate message
+	if (self->isCmdProcessed(req.cmdID)) {
+		printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", self->describeNode().c_str(), req.cmdID.toString().c_str());
+		req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+		return Void();
+	}
+	self->setInProgressFlag(RestoreCommandEnum::Loader_Send_Sample_Mutation_To_Applier);
+
+	// Applier will cache the mutations at each version. Once receive all mutations, applier will apply them to DB
+	state uint64_t commitVersion = req.commitVersion;
+	// TODO: Change the req.mutation to a vector of mutations
+	VectorRef<MutationRef> mutations(req.mutations);
+
+	state int mIndex = 0;
+	for (mIndex = 0; mIndex < mutations.size(); mIndex++) {
+		MutationRef mutation = mutations[mIndex];
+		if ( self->keyOpsCount.find(mutation.param1) == self->keyOpsCount.end() ) {
+			self->keyOpsCount.insert(std::make_pair(mutation.param1, 0));
+		}
+		// NOTE: We may receive the same mutation more than once due to network package lost.
+		// Since sampling is just an estimation and the network should be stable enough, we do NOT handle the duplication for now
+		// In a very unreliable network, we may get many duplicate messages and get a bad key-range splits for appliers. But the restore should still work except for running slower.
+		self->keyOpsCount[mutation.param1]++;
+		self->numSampledMutations++;
+
+		if ( debug_verbose && self->numSampledMutations % 1000 == 1 ) {
+			printf("[Sampling][Applier] Node:%s Receives %d sampled mutations. cur_mutation:%s\n",
+					self->describeNode().c_str(), self->numSampledMutations, mutation.toString().c_str());
+		}
+	}
+	
+	req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+	self->processedCmd[req.cmdID] = 1;
+
+	self->clearInProgressFlag(RestoreCommandEnum::Loader_Send_Sample_Mutation_To_Applier);
+
+	return Void();
+}
+
+ ACTOR Future<Void> handleApplyToDBRequest(RestoreSimpleRequest req, Reference<RestoreApplierData> self, Database cx) {
+ 	state bool isPrint = false; //Debug message
+ 	state std::string typeStr = "";
+
+	// Wait in case the  applyToDB request was delivered twice;
+	while (self->inProgressApplyToDB) {
+		printf("[DEBUG] NODE:%s inProgressApplyToDB wait for 5s\n",  self->describeNode().c_str());
+		wait(delay(5.0));
+	}
+	
+	if ( self->isCmdProcessed(req.cmdID) ) {
+		printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", self->describeNode().c_str(), req.cmdID.toString().c_str());
+		req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+		return Void();
+	}
+
+	self->inProgressApplyToDB = true;
+
+	// Assume the process will not crash when it apply mutations to DB. The reply message can be lost though
+	if (self->kvOps.empty()) {
+		printf("Node:%s kvOps is empty. No-op for apply to DB\n", self->describeNode().c_str());
+		req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+		self->processedCmd[req.cmdID] = 1;
+		self->inProgressApplyToDB = false;
+		return Void();
+	}
+	
+	self->sanityCheckMutationOps();
+
+ 	if ( debug_verbose ) {
+		TraceEvent("ApplyKVOPsToDB").detail("MapSize", self->kvOps.size());
+		printf("ApplyKVOPsToDB num_of_version:%ld\n", self->kvOps.size());
+ 	}
+ 	state std::map<Version, Standalone<VectorRef<MutationRef>>>::iterator it = self->kvOps.begin();
+	state std::map<Version, Standalone<VectorRef<MutationRef>>>::iterator prevIt = it;
+	state int index = 0;
+	state int prevIndex = index;
+ 	state int count = 0;
+	state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
+	state int numVersion = 0;
+	state double transactionSize = 0;
+	loop {
+		try {
+			tr->reset();
+			tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
+			tr->setOption(FDBTransactionOptions::LOCK_AWARE);
+			transactionSize = 0;
+
+			for ( ; it != self->kvOps.end(); ++it ) {
+				numVersion++;
+				if ( debug_verbose ) {
+					TraceEvent("ApplyKVOPsToDB\t").detail("Version", it->first).detail("OpNum", it->second.size());
+				}
+				//printf("ApplyKVOPsToDB numVersion:%d Version:%08lx num_of_ops:%d, \n", numVersion, it->first, it->second.size());
+
+				state MutationRef m;
+				for ( ; index < it->second.size(); ++index ) {
+					m = it->second[index];
+					if (  m.type >= MutationRef::Type::SetValue && m.type <= MutationRef::Type::MAX_ATOMIC_OP )
+						typeStr = typeString[m.type];
+					else {
+						printf("ApplyKVOPsToDB MutationType:%d is out of range\n", m.type);
+					}
+
+					if ( debug_verbose && count % 1000 == 1 ) {
+						printf("ApplyKVOPsToDB Node:%s num_mutation:%d Version:%08lx num_of_ops:%d\n",
+								self->describeNode().c_str(), count, it->first, it->second.size());
+					}
+
+					if ( debug_verbose ) {
+						printf("[VERBOSE_DEBUG] Node:%s apply mutation:%s\n", self->describeNode().c_str(), m.toString().c_str());
+					}
+
+					if ( m.type == MutationRef::SetValue ) {
+						tr->set(m.param1, m.param2);
+					} else if ( m.type == MutationRef::ClearRange ) {
+						KeyRangeRef mutationRange(m.param1, m.param2);
+						tr->clear(mutationRange);
+					} else if ( isAtomicOp((MutationRef::Type) m.type) ) {
+						//// Now handle atomic operation from this if statement
+						// TODO: Have not de-duplicated the mutations for multiple network delivery
+						// ATOMIC_MASK = (1 << AddValue) | (1 << And) | (1 << Or) | (1 << Xor) | (1 << AppendIfFits) | (1 << Max) | (1 << Min) | (1 << SetVersionstampedKey) | (1 << SetVersionstampedValue) | (1 << ByteMin) | (1 << ByteMax) | (1 << MinV2) | (1 << AndV2),
+						//atomicOp( const KeyRef& key, const ValueRef& operand, uint32_t operationType )
+						tr->atomicOp(m.param1, m.param2, m.type);
+					} else {
+						printf("[WARNING] mtype:%d (%s) unhandled\n", m.type, typeStr.c_str());
+					}
+					++count;
+					transactionSize += m.expectedSize();
+					
+					if ( transactionSize >= transactionBatchSizeThreshold ) { // commit per 1000 mutations
+						wait(tr->commit());
+						tr->reset();
+						tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
+						tr->setOption(FDBTransactionOptions::LOCK_AWARE);
+						prevIt = it;
+						prevIndex = index;
+						transactionSize = 0;
+					}
+
+					if ( isPrint ) {
+						printf("\tApplyKVOPsToDB Version:%016lx MType:%s K:%s, V:%s K_size:%d V_size:%d\n", it->first, typeStr.c_str(),
+							getHexString(m.param1).c_str(), getHexString(m.param2).c_str(), m.param1.size(), m.param2.size());
+
+						TraceEvent("ApplyKVOPsToDB\t\t").detail("Version", it->first)
+								.detail("MType", m.type).detail("MTypeStr", typeStr)
+								.detail("MKey", getHexString(m.param1))
+								.detail("MValueSize", m.param2.size())
+								.detail("MValue", getHexString(m.param2));
+					}
+				}
+				index = 0;
+			}
+			// Last transaction
+			if (transactionSize > 0) {
+				wait(tr->commit());
+			}
+			break;
+		} catch(Error &e) {
+			printf("ApplyKVOPsToDB transaction error:%s.\n", e.what());
+			wait(tr->onError(e));
+			it = prevIt;
+			index = prevIndex;
+			transactionSize = 0;
+		}
+	}
+
+ 	self->kvOps.clear();
+ 	printf("Node:%s ApplyKVOPsToDB number of kv mutations:%d\n", self->describeNode().c_str(), count);
+
+	req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+	printf("self->processedCmd size:%d req.cmdID:%s\n", self->processedCmd.size(), req.cmdID.toString().c_str());
+	self->processedCmd[req.cmdID] = 1;
+	self->inProgressApplyToDB = false;
+
+ 	return Void();
+}
+
+
+
diff --git a/fdbserver/RestoreApplier.actor.h b/fdbserver/RestoreApplier.actor.h
index 2295b6f9a6..2eddd58c99 100644
--- a/fdbserver/RestoreApplier.actor.h
+++ b/fdbserver/RestoreApplier.actor.h
@@ -21,7 +21,7 @@
 // Declear RestoreApplier interface and actors
 
 #pragma once
-#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_RestoreApplierInterface_H)
+#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_RestoreApplierInterface_G_H)
 	#define FDBSERVER_RestoreApplierInterface_G_H
 	#include "fdbserver/RestoreApplier.actor.g.h"
 #elif !defined(FDBSERVER_RestoreApplierInterface_H)
@@ -35,5 +35,150 @@
 #include "fdbserver/CoordinationInterface.h"
 #include "fdbrpc/Locality.h"
 
+#include "fdbserver/RestoreUtil.h"
+#include "fdbserver/RestoreRoleCommon.actor.h"
+#include "fdbserver/RestoreWorkerInterface.h"
 
+#include "flow/actorcompiler.h" // has to be last include
+
+extern double transactionBatchSizeThreshold;
+
+struct RestoreApplierData : RestoreRoleData, public ReferenceCounted<RestoreApplierData> { 
+	// range2Applier is in master and loader node. Loader node uses this to determine which applier a mutation should be sent
+	std::map<Standalone<KeyRef>, UID> range2Applier; // KeyRef is the inclusive lower bound of the key range the applier (UID) is responsible for
+	std::map<Standalone<KeyRef>, int> keyOpsCount; // The number of operations per key which is used to determine the key-range boundary for appliers
+	int numSampledMutations; // The total number of mutations received from sampled data.
+
+	// For master applier to hold the lower bound of key ranges for each appliers
+	std::vector<Standalone<KeyRef>> keyRangeLowerBounds;
+
+	// TODO: This block of variables may be moved to RestoreRoleData
+	bool inProgressApplyToDB = false;
+
+	// Temporary data structure for parsing range and log files into (version, <K, V, mutationType>)
+	std::map<Version, Standalone<VectorRef<MutationRef>>> kvOps;
+
+	void addref() { return ReferenceCounted<RestoreApplierData>::addref(); }
+	void delref() { return ReferenceCounted<RestoreApplierData>::delref(); }
+
+	RestoreApplierData() {
+		nodeID = g_random->randomUniqueID();
+		nodeIndex = 0;
+	}
+
+	~RestoreApplierData() {}
+
+	std::string describeNode() {
+		std::stringstream ss;
+		ss << "NodeID:" << nodeID.toString() << " nodeIndex:" << nodeIndex;
+		return ss.str();
+	}
+
+	void resetPerVersionBatch() {
+		RestoreRoleData::resetPerVersionBatch();
+
+		inProgressApplyToDB = false;
+		kvOps.clear();
+	}
+
+	void sanityCheckMutationOps() {
+		if (kvOps.empty())
+			return;
+
+		if ( isKVOpsSorted() ) {
+			printf("[CORRECT] KVOps is sorted by version\n");
+		} else {
+			printf("[ERROR]!!! KVOps is NOT sorted by version\n");
+		}
+
+		if ( allOpsAreKnown() ) {
+			printf("[CORRECT] KVOps all operations are known.\n");
+		} else {
+			printf("[ERROR]!!! KVOps has unknown mutation op. Exit...\n");
+		}
+	}
+
+	bool isKVOpsSorted() {
+		bool ret = true;
+		auto prev = kvOps.begin();
+		for ( auto it = kvOps.begin(); it != kvOps.end(); ++it ) {
+			if ( prev->first > it->first ) {
+				ret = false;
+				break;
+			}
+			prev = it;
+		}
+		return ret;
+	}
+
+	bool allOpsAreKnown() {
+		bool ret = true;
+		for ( auto it = kvOps.begin(); it != kvOps.end(); ++it ) {
+			for ( auto m = it->second.begin(); m != it->second.end(); ++m ) {
+				if ( m->type == MutationRef::SetValue || m->type == MutationRef::ClearRange
+					|| isAtomicOp((MutationRef::Type) m->type) )
+					continue;
+				else {
+					printf("[ERROR] Unknown mutation type:%d\n", m->type);
+					ret = false;
+				}
+			}
+
+		}
+
+		return ret;
+	}
+
+
+	std::vector<Standalone<KeyRef>> calculateAppliersKeyRanges(int numAppliers) {
+		ASSERT(numAppliers > 0);
+		std::vector<Standalone<KeyRef>> lowerBounds;
+		int numSampledMutations = 0;
+		for (auto &count : keyOpsCount) {
+			numSampledMutations += count.second;
+		}
+
+		//intervalLength = (numSampledMutations - remainder) / (numApplier - 1)
+		int intervalLength = std::max(numSampledMutations / numAppliers, 1); // minimal length is 1
+		int curCount = 0;
+		int curInterval = 0;
+
+		printf("[INFO] Node:%s calculateAppliersKeyRanges(): numSampledMutations:%d numAppliers:%d intervalLength:%d\n",
+				describeNode().c_str(),
+				numSampledMutations, numAppliers, intervalLength);
+		for (auto &count : keyOpsCount) {
+			if (curCount >= curInterval * intervalLength) {
+				printf("[INFO] Node:%s calculateAppliersKeyRanges(): Add a new key range  [%d]:%s: curCount:%d\n",
+						describeNode().c_str(), curInterval, count.first.toString().c_str(), curCount);
+				lowerBounds.push_back(count.first); // The lower bound of the current key range
+				curInterval++;
+			}
+			curCount += count.second;
+		}
+
+		if ( lowerBounds.size() != numAppliers ) {
+			printf("[WARNING] calculateAppliersKeyRanges() WE MAY NOT USE ALL APPLIERS efficiently! num_keyRanges:%ld numAppliers:%d\n",
+					lowerBounds.size(), numAppliers);
+			printLowerBounds(lowerBounds);
+		}
+
+		//ASSERT(lowerBounds.size() <= numAppliers + 1); // We may have at most numAppliers + 1 key ranges
+		if ( lowerBounds.size() >= numAppliers ) {
+			printf("[WARNING] Key ranges number:%ld > numAppliers:%d. Merge the last ones\n", lowerBounds.size(), numAppliers);
+		}
+
+		while ( lowerBounds.size() >= numAppliers ) {
+			printf("[WARNING] Key ranges number:%ld > numAppliers:%d. Merge the last ones\n", lowerBounds.size(), numAppliers);
+			lowerBounds.pop_back();
+		}
+
+		return lowerBounds;
+	}
+};
+
+
+ACTOR Future<Void> restoreApplierCore(Reference<RestoreApplierData> self, RestoreApplierInterface applierInterf, Database cx);
+
+
+#include "flow/unactorcompiler.h"
 #endif
\ No newline at end of file
diff --git a/fdbserver/RestoreCommon.actor.h b/fdbserver/RestoreCommon.actor.h
index ef778fef54..834f3f51a1 100644
--- a/fdbserver/RestoreCommon.actor.h
+++ b/fdbserver/RestoreCommon.actor.h
@@ -32,7 +32,7 @@
 #include "fdbrpc/IAsyncFile.h"
 #include "fdbclient/BackupAgent.actor.h"
 #include "flow/genericactors.actor.h"
-#include "flow/actorcompiler.h" // has to be last include
+
 
 // RestoreConfig copied from FileBackupAgent.actor.cpp
 // We copy RestoreConfig instead of using (and potentially changing) it in place to avoid conflict with the existing code
diff --git a/fdbserver/RestoreLoader.actor.cpp b/fdbserver/RestoreLoader.actor.cpp
index bc10f5226b..cfccddb442 100644
--- a/fdbserver/RestoreLoader.actor.cpp
+++ b/fdbserver/RestoreLoader.actor.cpp
@@ -17,3 +17,1132 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+#include "fdbclient/BackupContainer.h"
+#include "fdbserver/RestoreLoader.actor.h"
+
+#include "flow/actorcompiler.h"  // This must be the last #include.
+
+ACTOR Future<Void> handleSampleRangeFileRequest(RestoreLoadFileRequest req, Reference<RestoreLoaderData> self);
+ACTOR Future<Void> handleSampleLogFileRequest(RestoreLoadFileRequest req, Reference<RestoreLoaderData> self);
+ACTOR Future<Void> handleSetApplierKeyRangeVectorRequest(RestoreSetApplierKeyRangeVectorRequest req, Reference<RestoreLoaderData> self);
+ACTOR Future<Void> handleLoadRangeFileRequest(RestoreLoadFileRequest req, Reference<RestoreLoaderData> self);
+ACTOR Future<Void> handleLoadLogFileRequest(RestoreLoadFileRequest req, Reference<RestoreLoaderData> self);
+ACTOR Future<Void> registerMutationsToMasterApplier(Reference<RestoreLoaderData> self);
+
+ ACTOR static Future<Void> _parseLogFileToMutationsOnLoader(Reference<RestoreLoaderData> self,
+ 									Reference<IBackupContainer> bc, Version version,
+ 									std::string fileName, int64_t readOffset, int64_t readLen,
+ 									KeyRange restoreRange, Key addPrefix, Key removePrefix,
+ 									Key mutationLogPrefix);
+ACTOR static Future<Void> _parseRangeFileToMutationsOnLoader(Reference<RestoreLoaderData> self,
+ 									Reference<IBackupContainer> bc, Version version,
+ 									std::string fileName, int64_t readOffset_input, int64_t readLen_input,
+ 									KeyRange restoreRange, Key addPrefix, Key removePrefix);
+ACTOR Future<Void> registerMutationsToApplier(Reference<RestoreLoaderData> self);
+void parseSerializedMutation(Reference<RestoreLoaderData> self, bool isSampling);
+bool isRangeMutation(MutationRef m);
+void splitMutation(Reference<RestoreLoaderData> self,  MutationRef m, Arena& mvector_arena, VectorRef<MutationRef> mvector, Arena& nodeIDs_arena, VectorRef<UID> nodeIDs) ;
+
+
+ACTOR Future<Void> restoreLoaderCore(Reference<RestoreLoaderData> self, RestoreLoaderInterface loaderInterf, Database cx) {
+	state ActorCollection actors(false);
+	state double lastLoopTopTime;
+	loop {
+		
+		double loopTopTime = now();
+		double elapsedTime = loopTopTime - lastLoopTopTime;
+		if( elapsedTime > 0.050 ) {
+			if (g_random->random01() < 0.01)
+				TraceEvent(SevWarn, "SlowRestoreLoaderLoopx100").detail("NodeDesc", self->describeNode()).detail("Elapsed", elapsedTime);
+		}
+		lastLoopTopTime = loopTopTime;
+		state std::string requestTypeStr = "[Init]";
+
+		try {
+			choose {
+				when ( RestoreSimpleRequest req = waitNext(loaderInterf.heartbeat.getFuture()) ) {
+					requestTypeStr = "heartbeat";
+					wait(handleHeartbeat(req, loaderInterf.id()));
+				}
+				when ( RestoreLoadFileRequest req = waitNext(loaderInterf.sampleRangeFile.getFuture()) ) {
+					requestTypeStr = "sampleRangeFile";
+					self->initBackupContainer(req.param.url);
+					actors.add( handleSampleRangeFileRequest(req, self) );
+				}
+				when ( RestoreLoadFileRequest req = waitNext(loaderInterf.sampleLogFile.getFuture()) ) {
+					self->initBackupContainer(req.param.url);
+					requestTypeStr = "sampleLogFile";
+					actors.add( handleSampleLogFileRequest(req, self) );
+				}
+				when ( RestoreSetApplierKeyRangeVectorRequest req = waitNext(loaderInterf.setApplierKeyRangeVectorRequest.getFuture()) ) {
+					requestTypeStr = "setApplierKeyRangeVectorRequest";
+					wait(handleSetApplierKeyRangeVectorRequest(req, self));
+				}
+				when ( RestoreLoadFileRequest req = waitNext(loaderInterf.loadRangeFile.getFuture()) ) {
+					requestTypeStr = "loadRangeFile";
+					self->initBackupContainer(req.param.url);
+					actors.add( handleLoadRangeFileRequest(req, self) );
+				}
+				when ( RestoreLoadFileRequest req = waitNext(loaderInterf.loadLogFile.getFuture()) ) {
+					requestTypeStr = "loadLogFile";
+					self->initBackupContainer(req.param.url);
+					actors.add( handleLoadLogFileRequest(req, self) );
+				}
+
+				when ( RestoreVersionBatchRequest req = waitNext(loaderInterf.initVersionBatch.getFuture()) ) {
+					requestTypeStr = "initVersionBatch";
+					wait(handleInitVersionBatchRequest(req, self));
+				}
+
+                // TODO: To modify the following when conditions
+				when ( RestoreSimpleRequest req = waitNext(loaderInterf.collectRestoreRoleInterfaces.getFuture()) ) {
+					// Step: Find other worker's workerInterfaces
+					// NOTE: This must be after wait(configureRolesHandler()) because we must ensure all workers have registered their workerInterfaces into DB before we can read the workerInterface.
+					// TODO: Wait until all workers have registered their workerInterface.
+					wait( handleCollectRestoreRoleInterfaceRequest(req, self, cx) );
+				}
+			}
+
+		} catch (Error &e) {
+            fprintf(stdout, "[ERROR] Restore Loader handle received request:%s error. error code:%d, error message:%s\n",
+                    requestTypeStr.c_str(), e.code(), e.what());
+
+			if ( requestTypeStr.find("[Init]") != std::string::npos ) {
+				printf("Exit due to error at requestType:%s", requestTypeStr.c_str());
+				break;
+			}
+		}
+	}
+
+	return Void();
+}
+
+// Restore Loader
+ACTOR Future<Void> handleSetApplierKeyRangeVectorRequest(RestoreSetApplierKeyRangeVectorRequest req, Reference<RestoreLoaderData> self) {
+	// Idempodent operation. OK to re-execute the duplicate cmd
+	// The applier should remember the key range it is responsible for
+	//ASSERT(req.cmd == (RestoreCommandEnum) req.cmdID.phase);
+	//self->applierStatus.keyRange = req.range;
+	while (self->isInProgress(RestoreCommandEnum::Notify_Loader_ApplierKeyRange)) {
+		printf("[DEBUG] NODE:%s handleSetApplierKeyRangeVectorRequest wait for 1s\n",  self->describeNode().c_str());
+		wait(delay(1.0));
+	}
+	if ( self->isCmdProcessed(req.cmdID) ) {
+		req.reply.send(RestoreCommonReply(self->id(),req.cmdID));
+		return Void();
+	}
+	self->setInProgressFlag(RestoreCommandEnum::Notify_Loader_ApplierKeyRange);
+
+	VectorRef<UID> appliers = req.applierIDs;
+	VectorRef<KeyRange> ranges = req.ranges;
+	for ( int i = 0; i < appliers.size(); i++ ) {
+		self->range2Applier[ranges[i].begin] = appliers[i];
+	}
+	
+	self->processedCmd[req.cmdID] = 1;
+	self->clearInProgressFlag(RestoreCommandEnum::Notify_Loader_ApplierKeyRange);
+	req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+
+	return Void();
+}
+
+// TODO: Remove the RestoreLoaderInterface param., which is not needed in the handler functions
+// Restore Loader
+ACTOR Future<Void> handleSampleRangeFileRequest(RestoreLoadFileRequest req, Reference<RestoreLoaderData> self) {
+	//printf("[INFO] Node:%s Got Restore Command: cmdID:%s.\n", self->describeNode().c_str(), req.cmdID.toString().c_str());
+
+	state LoadingParam param = req.param;
+	state int beginBlock = 0;
+	state int j = 0;
+	state int readLen = 0;
+	state int64_t readOffset = param.offset;
+
+	while (self->isInProgress(RestoreCommandEnum::Sample_Range_File)) {
+		printf("[DEBUG] NODE:%s sampleRangeFile wait for 5s\n",  self->describeNode().c_str());
+		wait(delay(5.0));
+	}
+
+	// Handle duplicate, assuming cmdUID is always unique for the same workload
+	if ( self->isCmdProcessed(req.cmdID) ) {
+		printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", self->describeNode().c_str(), req.cmdID.toString().c_str());
+		req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+		return Void();
+	} 
+
+	self->setInProgressFlag(RestoreCommandEnum::Sample_Range_File);
+	printf("[Sample_Range_File][Loader] Node: %s, loading param:%s\n",
+			self->describeNode().c_str(), param.toString().c_str());
+
+	// TODO: This can be expensive
+	state Reference<IBackupContainer> bc =  self->bc;
+	printf("[INFO] node:%s open backup container for url:%s\n",
+			self->describeNode().c_str(),
+			param.url.toString().c_str());
+
+
+	self->kvOps.clear(); //Clear kvOps so that kvOps only hold mutations for the current data block. We will send all mutations in kvOps to applier
+	self->mutationMap.clear();
+	self->mutationPartMap.clear();
+
+	ASSERT( param.blockSize > 0 );
+	//state std::vector<Future<Void>> fileParserFutures;
+	if (param.offset % param.blockSize != 0) {
+		printf("[WARNING] Parse file not at block boundary! param.offset:%ld param.blocksize:%ld, remainder:%ld\n",
+				param.offset, param.blockSize, param.offset % param.blockSize);
+	}
+
+	ASSERT( param.offset + param.blockSize >= param.length ); // We only sample one data block or less (at the end of the file) of a file.
+	for (j = param.offset; j < param.length; j += param.blockSize) {
+		readOffset = j;
+		readLen = std::min<int64_t>(param.blockSize, param.length - j);
+		wait( _parseRangeFileToMutationsOnLoader(self, bc, param.version, param.filename, readOffset, readLen, param.restoreRange, param.addPrefix, param.removePrefix) );
+		++beginBlock;
+	}
+
+	printf("[Sampling][Loader] Node:%s finishes sample Range file:%s\n", self->describeNode().c_str(), param.filename.c_str());
+	// TODO: Send to applier to apply the mutations
+	printf("[Sampling][Loader] Node:%s will send sampled mutations to applier\n", self->describeNode().c_str());
+	wait( registerMutationsToMasterApplier(self) ); // Send the parsed mutation to applier who will apply the mutation to DB
+
+	//self->processedFiles.insert(std::make_pair(param.filename, 1));
+
+	//TODO: Send ack to master that loader has finished loading the data
+	req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+	self->processedCmd[req.cmdID] = 1; // Recoself the processed comand to handle duplicate command
+	//self->kvOps.clear(); 
+
+	self->clearInProgressFlag(RestoreCommandEnum::Sample_Range_File);
+
+	return Void();
+}
+
+ACTOR Future<Void> handleSampleLogFileRequest(RestoreLoadFileRequest req, Reference<RestoreLoaderData> self) {
+	state LoadingParam param = req.param;
+	state int beginBlock = 0;
+	state int j = 0;
+	state int readLen = 0;
+	state int64_t readOffset = param.offset;
+
+	while (self->isInProgress(RestoreCommandEnum::Sample_Log_File)) {
+		printf("[DEBUG] NODE:%s sampleLogFile wait for 5s\n",  self->describeNode().c_str());
+		wait(delay(5.0));
+	}
+
+	// Handle duplicate message
+	if ( self->isCmdProcessed(req.cmdID) ) {
+		printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", self->describeNode().c_str(), req.cmdID.toString().c_str());
+		req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+		return Void();
+	}
+
+	self->setInProgressFlag(RestoreCommandEnum::Sample_Log_File);
+	printf("[Sample_Log_File][Loader]  Node: %s, loading param:%s\n", self->describeNode().c_str(), param.toString().c_str());
+
+	// TODO: Expensive operation
+	state Reference<IBackupContainer> bc =  self->bc;
+	printf("[Sampling][Loader] Node:%s open backup container for url:%s\n",
+			self->describeNode().c_str(),
+			param.url.toString().c_str());
+	printf("[Sampling][Loader] Node:%s filename:%s blockSize:%ld\n",
+			self->describeNode().c_str(),
+			param.filename.c_str(), param.blockSize);
+
+	self->kvOps.clear(); //Clear kvOps so that kvOps only hold mutations for the current data block. We will send all mutations in kvOps to applier
+	self->mutationMap.clear();
+	self->mutationPartMap.clear();
+
+	ASSERT( param.blockSize > 0 );
+	//state std::vector<Future<Void>> fileParserFutures;
+	if (param.offset % param.blockSize != 0) {
+		printf("[WARNING] Parse file not at block boundary! param.offset:%ld param.blocksize:%ld, remainder:%ld\n",
+			param.offset, param.blockSize, param.offset % param.blockSize);
+	}
+	ASSERT( param.offset + param.blockSize >= param.length ); // Assumption: Only sample one data block or less
+	for (j = param.offset; j < param.length; j += param.blockSize) {
+		readOffset = j;
+		readLen = std::min<int64_t>(param.blockSize, param.length - j);
+		// NOTE: Log file holds set of blocks of data. We need to parse the data block by block and get the kv pair(version, serialized_mutations)
+		// The set of mutations at the same version may be splitted into multiple kv pairs ACROSS multiple data blocks when the size of serialized_mutations is larger than 20000.
+		wait( _parseLogFileToMutationsOnLoader(self, bc, param.version, param.filename, readOffset, readLen, param.restoreRange, param.addPrefix, param.removePrefix, param.mutationLogPrefix) );
+		++beginBlock;
+	}
+	printf("[Sampling][Loader] Node:%s finishes parsing the data block into kv pairs (version, serialized_mutations) for file:%s\n", self->describeNode().c_str(), param.filename.c_str());
+	parseSerializedMutation(self, true);
+
+	printf("[Sampling][Loader] Node:%s finishes process Log file:%s\n", self->describeNode().c_str(), param.filename.c_str());
+	printf("[Sampling][Loader] Node:%s will send log mutations to applier\n", self->describeNode().c_str());
+	wait( registerMutationsToMasterApplier(self) ); // Send the parsed mutation to applier who will apply the mutation to DB
+
+	req.reply.send(RestoreCommonReply(self->id(), req.cmdID)); // master node is waiting
+	self->processedFiles.insert(std::make_pair(param.filename, 1));
+	self->processedCmd[req.cmdID] = 1;
+
+	self->clearInProgressFlag(RestoreCommandEnum::Sample_Log_File);
+
+	return Void();
+}
+
+
+ACTOR Future<Void> handleLoadRangeFileRequest(RestoreLoadFileRequest req, Reference<RestoreLoaderData> self) {
+	//printf("[INFO] Worker Node:%s starts handleLoadRangeFileRequest\n", self->describeNode().c_str());
+
+	state LoadingParam param;
+	state int64_t beginBlock = 0;
+	state int64_t j = 0;
+	state int64_t readLen = 0;
+	state int64_t readOffset = 0;
+	state Reference<IBackupContainer> bc;
+
+	param = req.param;
+	beginBlock = 0;
+	j = 0;
+	readLen = 0;
+	readOffset = 0;
+	readOffset = param.offset;
+
+	while (self->isInProgress(RestoreCommandEnum::Assign_Loader_Range_File)) {
+		printf("[DEBUG] NODE:%s loadRangeFile wait for 5s\n",  self->describeNode().c_str());
+		wait(delay(5.0));
+	}
+
+	//Note: handle duplicate message delivery
+	if (self->processedFiles.find(param.filename) != self->processedFiles.end() ||
+		self->isCmdProcessed(req.cmdID)) {
+		// printf("[WARNING]Node:%s, CMDUID:%s file:%s is delivered more than once! Reply directly without loading the file\n",
+		// 		self->describeNode().c_str(), req.cmdID.toString().c_str(),
+		// 		param.filename.c_str());
+		req.reply.send(RestoreCommonReply(self->id(),req.cmdID));
+		return Void();
+	}
+
+	self->setInProgressFlag(RestoreCommandEnum::Assign_Loader_Range_File);
+
+	printf("[INFO][Loader] Node:%s, CMDUID:%s Execute: Assign_Loader_Range_File, loading param:%s\n",
+			self->describeNode().c_str(), req.cmdID.toString().c_str(),
+			param.toString().c_str());
+
+	bc = self->bc;
+	// printf("[INFO] Node:%s CMDUID:%s open backup container for url:%s\n",
+	// 		self->describeNode().c_str(), req.cmdID.toString().c_str(),
+	// 		param.url.toString().c_str());
+
+
+	self->kvOps.clear(); //Clear kvOps so that kvOps only hold mutations for the current data block. We will send all mutations in kvOps to applier
+	self->mutationMap.clear();
+	self->mutationPartMap.clear();
+
+	ASSERT( param.blockSize > 0 );
+	//state std::vector<Future<Void>> fileParserFutures;
+	if (param.offset % param.blockSize != 0) {
+		printf("[WARNING] Parse file not at block boundary! param.offset:%ld param.blocksize:%ld, remainder:%ld\n",
+				param.offset, param.blockSize, param.offset % param.blockSize);
+	}
+	for (j = param.offset; j < param.length; j += param.blockSize) {
+		readOffset = j;
+		readLen = std::min<int64_t>(param.blockSize, param.length - j);
+		printf("[DEBUG_TMP] _parseRangeFileToMutationsOnLoader starts\n");
+		wait( _parseRangeFileToMutationsOnLoader(self, bc, param.version, param.filename, readOffset, readLen, param.restoreRange, param.addPrefix, param.removePrefix) );
+		printf("[DEBUG_TMP] _parseRangeFileToMutationsOnLoader ends\n");
+		++beginBlock;
+	}
+
+	printf("[INFO][Loader] Node:%s CMDUID:%s finishes process Range file:%s\n",
+			self->describeNode().c_str(), req.cmdID.toString().c_str(),
+			param.filename.c_str());
+	// TODO: Send to applier to apply the mutations
+	// printf("[INFO][Loader] Node:%s CMDUID:%s will send range mutations to applier\n",
+	// 		self->describeNode().c_str(), self->cmdID.toString().c_str());
+	wait( registerMutationsToApplier(self) ); // Send the parsed mutation to applier who will apply the mutation to DB
+	wait ( delay(1.0) );
+	
+	self->processedFiles[param.filename] =  1;
+	self->processedCmd[req.cmdID] = 1;
+
+	self->clearInProgressFlag(RestoreCommandEnum::Assign_Loader_Range_File);
+	printf("[INFO][Loader] Node:%s CMDUID:%s clear inProgressFlag :%lx for Assign_Loader_Range_File.\n",
+			self->describeNode().c_str(), req.cmdID.toString().c_str(), self->inProgressFlag);
+
+	//Send ack to master that loader has finished loading the data
+	printf("[INFO][Loader] Node:%s CMDUID:%s send ack.\n",
+			self->describeNode().c_str(), self->cmdID.toString().c_str());
+	req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+
+	return Void();
+
+}
+
+
+ACTOR Future<Void> handleLoadLogFileRequest(RestoreLoadFileRequest req, Reference<RestoreLoaderData> self) {
+	printf("[INFO] Worker Node:%s starts handleLoadLogFileRequest\n", self->describeNode().c_str());
+
+	state LoadingParam param;
+	state int64_t beginBlock = 0;
+	state int64_t j = 0;
+	state int64_t readLen = 0;
+	state int64_t readOffset = 0;
+	state Reference<IBackupContainer> bc;
+
+	param = req.param;
+	beginBlock = 0;
+	j = 0;
+	readLen = 0;
+	readOffset = 0;
+	readOffset = param.offset;
+
+	while (self->isInProgress(RestoreCommandEnum::Assign_Loader_Log_File)) {
+		printf("[DEBUG] NODE:%s loadLogFile wait for 5s\n",  self->describeNode().c_str());
+		wait(delay(5.0));
+	}
+	
+	//Note: handle duplicate message delivery
+	if (self->processedFiles.find(param.filename) != self->processedFiles.end()
+	   || self->isCmdProcessed(req.cmdID)) {
+		printf("[WARNING] Node:%s CMDUID:%s file:%s is delivered more than once! Reply directly without loading the file\n",
+				self->describeNode().c_str(), req.cmdID.toString().c_str(),
+				param.filename.c_str());
+		req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+		return Void();
+	}
+
+	self->setInProgressFlag(RestoreCommandEnum::Assign_Loader_Log_File);
+
+	printf("[INFO][Loader] Node:%s CMDUID:%s Assign_Loader_Log_File loading param:%s\n",
+								self->describeNode().c_str(), req.cmdID.toString().c_str(),
+								param.toString().c_str());
+
+	bc = self->bc;
+	printf("[INFO][Loader] Node:%s CMDUID:%s open backup container for url:%s\n",
+			self->describeNode().c_str(), req.cmdID.toString().c_str(),
+			param.url.toString().c_str());
+	printf("[INFO][Loader] Node:%s CMDUID:%s filename:%s blockSize:%ld\n",
+			self->describeNode().c_str(), req.cmdID.toString().c_str(),
+			param.filename.c_str(), param.blockSize);
+
+	self->kvOps.clear(); //Clear kvOps so that kvOps only hold mutations for the current data block. We will send all mutations in kvOps to applier
+	self->mutationMap.clear();
+	self->mutationPartMap.clear();
+
+	ASSERT( param.blockSize > 0 );
+	//state std::vector<Future<Void>> fileParserFutures;
+	if (param.offset % param.blockSize != 0) {
+		printf("[WARNING] Parse file not at block boundary! param.offset:%ld param.blocksize:%ld, remainder:%ld\n",
+				param.offset, param.blockSize, param.offset % param.blockSize);
+	}
+	for (j = param.offset; j < param.length; j += param.blockSize) {
+		readOffset = j;
+		readLen = std::min<int64_t>(param.blockSize, param.length - j);
+		// NOTE: Log file holds set of blocks of data. We need to parse the data block by block and get the kv pair(version, serialized_mutations)
+		// The set of mutations at the same version may be splitted into multiple kv pairs ACROSS multiple data blocks when the size of serialized_mutations is larger than 20000.
+		wait( _parseLogFileToMutationsOnLoader(self, bc, param.version, param.filename, readOffset, readLen, param.restoreRange, param.addPrefix, param.removePrefix, param.mutationLogPrefix) );
+		++beginBlock;
+	}
+	printf("[INFO][Loader] Node:%s CMDUID:%s finishes parsing the data block into kv pairs (version, serialized_mutations) for file:%s\n",
+			self->describeNode().c_str(), req.cmdID.toString().c_str(),
+			param.filename.c_str());
+	parseSerializedMutation(self, false);
+
+	printf("[INFO][Loader] Node:%s CMDUID:%s finishes process Log file:%s\n",
+			self->describeNode().c_str(), req.cmdID.toString().c_str(),
+			param.filename.c_str());
+	printf("[INFO][Loader] Node:%s CMDUID:%s will send log mutations to applier\n",
+			self->describeNode().c_str(), req.cmdID.toString().c_str());
+	wait( registerMutationsToApplier(self) ); // Send the parsed mutation to applier who will apply the mutation to DB
+
+	req.reply.send(RestoreCommonReply(self->id(), req.cmdID)); // master node is waiting
+	self->processedFiles[param.filename] =  1;
+	self->processedCmd[req.cmdID] = 1;
+
+	self->clearInProgressFlag(RestoreCommandEnum::Assign_Loader_Log_File);
+	
+	return Void();
+}
+
+
+
+// Loader: Register sampled mutations
+ACTOR Future<Void> registerMutationsToMasterApplier(Reference<RestoreLoaderData> self) {
+	printf("[Sampling] Node:%s registerMutationsToMaster() self->masterApplierInterf:%s\n",
+			self->describeNode().c_str(), self->masterApplierInterf.toString().c_str());
+
+	state RestoreApplierInterface applierCmdInterf = self->masterApplierInterf;
+	state int packMutationNum = 0;
+	state int packMutationThreshold = 1;
+	state int kvCount = 0;
+	state std::vector<Future<RestoreCommonReply>> cmdReplies;
+
+	state int splitMutationIndex = 0;
+	state std::map<Version, Standalone<VectorRef<MutationRef>>>::iterator kvOp;
+	state int mIndex;
+	state uint64_t commitVersion;
+	state MutationRef kvm;
+
+	state Standalone<VectorRef<MutationRef>> mutationsBuffer; // The mutation vector to be sent to master applier
+	state double mutationsSize = 0;
+	//state double mutationVectorThreshold = 1; //1024 * 10; // Bytes
+	loop {
+		try {
+			cmdReplies.clear();
+			mutationsBuffer.pop_front(mutationsBuffer.size());
+			mutationsSize = 0;
+			packMutationNum = 0;
+			self->cmdID.initPhase(RestoreCommandEnum::Loader_Send_Sample_Mutation_To_Applier); 
+			// TODO: Consider using a different EndPoint for loader and applier communication.
+			// Otherwise, applier may receive loader's message while applier is waiting for master to assign key-range
+			for ( kvOp = self->kvOps.begin(); kvOp != self->kvOps.end(); kvOp++) {
+				commitVersion = kvOp->first;
+				
+				for (mIndex = 0; mIndex < kvOp->second.size(); mIndex++) {
+					kvm = kvOp->second[mIndex];
+					self->cmdID.nextCmd();
+					if ( debug_verbose || true ) { // Debug deterministic bug
+						printf("[VERBOSE_DEBUG] send mutation to applier, mIndex:%d mutation:%s\n", mIndex, kvm.toString().c_str());
+					}
+					mutationsBuffer.push_back(mutationsBuffer.arena(), kvm);
+					mutationsSize += kvm.expectedSize();
+					if ( mutationsSize >= mutationVectorThreshold ) {
+						self->cmdID.nextCmd();
+						cmdReplies.push_back(applierCmdInterf.sendSampleMutationVector.getReply(
+							RestoreSendMutationVectorRequest(self->cmdID, commitVersion, mutationsBuffer)));
+							mutationsBuffer.pop_front(mutationsBuffer.size());
+							mutationsSize = 0;
+						if ( debug_verbose ) {
+							printf("[INFO][Loader] Waits for master applier to receive %ld mutations\n", mutationsBuffer.size());
+						}
+						std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) );
+						//std::vector<RestoreCommonReply> reps = wait( getAll(cmdReplies) );
+						cmdReplies.clear();
+					}
+
+					kvCount++;
+				}
+			}
+
+			// The leftover mutationVector whose size is < mutationVectorThreshold
+			if ( mutationsSize > 0 ) {
+				self->cmdID.nextCmd();
+				cmdReplies.push_back(applierCmdInterf.sendSampleMutationVector.getReply(
+					RestoreSendMutationVectorRequest(self->cmdID, commitVersion, mutationsBuffer)));
+					mutationsBuffer.pop_front(mutationsBuffer.size());
+					mutationsSize = 0;
+			}
+
+
+			if (!cmdReplies.empty()) {
+				printf("[INFO][Loader] Last waits for master applier to receive %ld mutations\n", mutationsBuffer.size());
+				//std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout) );
+				std::vector<RestoreCommonReply> reps = wait( getAll(cmdReplies) );
+				cmdReplies.clear();
+			}
+
+			printf("[Sample Summary][Loader] Node:%s produces %d mutation operations\n", self->describeNode().c_str(), kvCount);
+			break;
+		} catch (Error &e) {
+			// TODO: Handle the command reply timeout error
+			if (e.code() != error_code_io_timeout) {
+				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s timeout\n", self->describeNode().c_str(), self->cmdID.toString().c_str());
+			} else {
+				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", self->describeNode().c_str(),
+						self->cmdID.toString().c_str(), e.code(), e.what());
+			}
+			printf("[WARNING] Node:%s timeout at waiting on replies of Loader_Send_Sample_Mutation_To_Applier. Retry...\n", self->describeNode().c_str());
+		}
+	}
+
+	return Void();
+}
+
+
+
+ACTOR Future<Void> registerMutationsToApplier(Reference<RestoreLoaderData> self) {
+	printf("[INFO][Loader] Node:%s self->masterApplierInterf:%s, registerMutationsToApplier\n",
+			self->describeNode().c_str(), self->masterApplierInterf.toString().c_str());
+
+	state RestoreApplierInterface applierCmdInterf;
+	state int packMutationNum = 0;
+	state int packMutationThreshold = 10;
+	state int kvCount = 0;
+	state std::vector<Future<RestoreCommonReply>> cmdReplies;
+
+	state int splitMutationIndex = 0;
+
+	self->printAppliersKeyRange();
+
+	//state double mutationVectorThreshold = 1;//1024 * 10; // Bytes.
+	state std::map<UID, Standalone<VectorRef<MutationRef>>> applierMutationsBuffer; // The mutation vector to be sent to each applier
+	state std::map<UID, double> applierMutationsSize; // buffered mutation vector size for each applier
+	// Initialize the above two maps
+	state std::vector<UID> applierIDs = self->getWorkingApplierIDs();
+	loop {
+		try {
+			packMutationNum = 0;
+			splitMutationIndex = 0;
+			kvCount = 0;
+			state std::map<Version, Standalone<VectorRef<MutationRef>>>::iterator kvOp;
+			self->cmdID.initPhase(RestoreCommandEnum::Loader_Send_Mutations_To_Applier);
+			// In case try-catch has error and loop back
+			applierMutationsBuffer.clear();
+			applierMutationsSize.clear();
+			for (auto &applierID : applierIDs) {
+				applierMutationsBuffer[applierID] = Standalone<VectorRef<MutationRef>>(VectorRef<MutationRef>());
+				applierMutationsSize[applierID] = 0.0;
+			}
+			for ( kvOp = self->kvOps.begin(); kvOp != self->kvOps.end(); kvOp++) {
+				state uint64_t commitVersion = kvOp->first;
+				state int mIndex;
+				state MutationRef kvm;
+				for (mIndex = 0; mIndex < kvOp->second.size(); mIndex++) {
+					kvm = kvOp->second[mIndex];
+					if ( debug_verbose ) {
+						printf("[VERBOSE_DEBUG] mutation to sent to applier, mutation:%s\n", kvm.toString().c_str());
+					}
+					// Send the mutation to applier
+					if (isRangeMutation(kvm)) {
+						// Because using a vector of mutations causes overhead, and the range mutation should happen rarely;
+						// We handle the range mutation and key mutation differently for the benefit of avoiding memory copy
+						state Standalone<VectorRef<MutationRef>> mvector;
+						state Standalone<VectorRef<UID>> nodeIDs;
+						// '' Bug may be here! The splitMutation() may be wrong!
+						splitMutation(self, kvm, mvector.arena(), mvector.contents(), nodeIDs.arena(), nodeIDs.contents());
+						ASSERT(mvector.size() == nodeIDs.size());
+
+						for (splitMutationIndex = 0; splitMutationIndex < mvector.size(); splitMutationIndex++ ) {
+							MutationRef mutation = mvector[splitMutationIndex];
+							UID applierID = nodeIDs[splitMutationIndex];
+							applierCmdInterf = self->appliersInterf[applierID];
+							applierMutationsBuffer[applierID].push_back(applierMutationsBuffer[applierID].arena(), mutation); // Q: Maybe push_back_deep()?
+							applierMutationsSize[applierID] += mutation.expectedSize();
+
+							kvCount++;
+						}
+
+						for (auto &applierID : applierIDs) {
+							if ( applierMutationsSize[applierID] >= mutationVectorThreshold ) {
+								self->cmdID.nextCmd();
+								cmdReplies.push_back(applierCmdInterf.sendMutationVector.getReply(
+									RestoreSendMutationVectorRequest(self->cmdID, commitVersion, applierMutationsBuffer[applierID])));
+								applierMutationsBuffer[applierID].pop_front(applierMutationsBuffer[applierID].size());
+								applierMutationsSize[applierID] = 0;
+
+								printf("[INFO][Loader] Waits for applier to receive %ld range mutations\n", cmdReplies.size());
+								std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) );
+								cmdReplies.clear();
+							}
+						}
+					} else { // mutation operates on a particular key
+						std::map<Standalone<KeyRef>, UID>::iterator itlow = self->range2Applier.lower_bound(kvm.param1); // lower_bound returns the iterator that is >= m.param1
+						// make sure itlow->first <= m.param1
+						if ( itlow == self->range2Applier.end() || itlow->first > kvm.param1 ) {
+							--itlow;
+						}
+						ASSERT( itlow->first <= kvm.param1 );
+						MutationRef mutation = kvm;
+						UID applierID = itlow->second;
+						applierCmdInterf = self->appliersInterf[applierID];
+						kvCount++;
+
+						applierMutationsBuffer[applierID].push_back(applierMutationsBuffer[applierID].arena(), mutation); // Q: Maybe push_back_deep()?
+						applierMutationsSize[applierID] += mutation.expectedSize();
+						if ( applierMutationsSize[applierID] >= mutationVectorThreshold ) {
+							self->cmdID.nextCmd();
+							cmdReplies.push_back(applierCmdInterf.sendMutationVector.getReply(
+												RestoreSendMutationVectorRequest(self->cmdID, commitVersion, applierMutationsBuffer[applierID])));
+							applierMutationsBuffer[applierID].pop_front(applierMutationsBuffer[applierID].size());
+							applierMutationsSize[applierID] = 0;
+
+							printf("[INFO][Loader] Waits for applier to receive %ld range mutations\n", cmdReplies.size());
+							std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) );
+							cmdReplies.clear();
+						}
+					}
+				}
+
+			}
+
+			// In case the mutation vector is not larger than mutationVectorThreshold
+			printf("[DEBUG][Loader] sendMutationVector sends the remaining applierMutationsBuffer, applierIDs.size:%d\n", applierIDs.size());
+			for (auto &applierID : applierIDs) {
+				if (applierMutationsBuffer[applierID].empty()) { //&& applierMutationsSize[applierID] >= 1
+					continue;
+				}
+				printf("[DEBUG][Loader] sendMutationVector for applierID:%s\n", applierID.toString().c_str());
+				self->cmdID.nextCmd();
+				cmdReplies.push_back(applierCmdInterf.sendMutationVector.getReply(
+									RestoreSendMutationVectorRequest(self->cmdID, commitVersion, applierMutationsBuffer[applierID])));
+				applierMutationsBuffer[applierID].pop_front(applierMutationsBuffer[applierID].size());
+				applierMutationsSize[applierID] = 0;
+				printf("[INFO][Loader] Waits for applier to receive %ld range mutations\n", cmdReplies.size());
+				std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) ); // Q: We need to wait for each reply, otherwise, correctness has error. Why?
+				cmdReplies.clear();
+			}
+
+			if (!cmdReplies.empty()) {
+				printf("[INFO][Loader] Last Waits for applier to receive %ld range mutations\n", cmdReplies.size());
+				std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) );
+				//std::vector<RestoreCommonReply> reps =  wait( getAll(cmdReplies) );
+				cmdReplies.clear();
+			}
+			printf("[Summary][Loader] Node:%s Last CMDUID:%s produces %d mutation operations\n",
+					self->describeNode().c_str(), self->cmdID.toString().c_str(), kvCount);
+
+			break;
+
+		} catch (Error &e) {
+			// Handle the command reply timeout error
+			fprintf(stdout, "[ERROR] registerMutationsToApplier Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", self->describeNode().c_str(),
+					self->cmdID.toString().c_str(), e.code(), e.what());
+		}
+	};
+
+	return Void();
+}
+
+
+
+void splitMutation(Reference<RestoreLoaderData> self,  MutationRef m, Arena& mvector_arena, VectorRef<MutationRef> mvector, Arena& nodeIDs_arena, VectorRef<UID> nodeIDs) {
+	// mvector[i] should be mapped to nodeID[i]
+	ASSERT(mvector.empty());
+	ASSERT(nodeIDs.empty());
+	// key range [m->param1, m->param2)
+	//std::map<Standalone<KeyRef>, UID>;
+	std::map<Standalone<KeyRef>, UID>::iterator itlow, itup; //we will return [itlow, itup)
+	itlow = self->range2Applier.lower_bound(m.param1); // lower_bound returns the iterator that is >= m.param1
+	if ( itlow != self->range2Applier.begin()) { // m.param1 is not the smallest key \00
+		// (itlow-1) is the node whose key range includes m.param1
+		--itlow;
+	} else {
+		if (m.param1 != LiteralStringRef("\00")) {
+			printf("[ERROR] splitMutation has bug on range mutation:%s\n", m.toString().c_str());
+		}
+	}
+
+	itup = self->range2Applier.upper_bound(m.param2); // upper_bound returns the iterator that is > m.param2; return rmap::end if no keys are considered to go after m.param2.
+	ASSERT( itup == self->range2Applier.end() || itup->first >= m.param2 );
+	// Now adjust for the case: example: mutation range is [a, d); we have applier's ranges' inclusive lower bound values are: a, b, c, d, e; upper_bound(d) returns itup to e, but we want itup to d.
+	--itup;
+	ASSERT( itup->first <= m.param2 );
+	if ( itup->first < m.param2 ) {
+		++itup; //make sure itup is >= m.param2, that is, itup is the next key range >= m.param2
+	}
+
+	while (itlow->first < itup->first) {
+		MutationRef curm; //current mutation
+		curm.type = m.type;
+		curm.param1 = itlow->first;
+		itlow++;
+		if (itlow == self->range2Applier.end()) {
+			curm.param2 = normalKeys.end;
+		} else {
+			curm.param2 = itlow->first;
+		}
+		mvector.push_back(mvector_arena, curm);
+
+		nodeIDs.push_back(nodeIDs_arena, itlow->second);
+	}
+
+	return;
+}
+
+
+//key_input format: [logRangeMutation.first][hash_value_of_commit_version:1B][bigEndian64(commitVersion)][bigEndian32(part)]
+bool concatenateBackupMutationForLogFile(Reference<RestoreLoaderData> self, Standalone<StringRef> val_input, Standalone<StringRef> key_input) {
+	std::string prefix = "||\t";
+	std::stringstream ss;
+	const int version_size = 12;
+	const int header_size = 12;
+	StringRef val = val_input.contents();
+	StringRefReaderMX reader(val, restore_corrupted_data());
+	StringRefReaderMX readerKey(key_input, restore_corrupted_data()); //read key_input!
+	int logRangeMutationFirstLength = key_input.size() - 1 - 8 - 4;
+	bool concatenated = false;
+
+	if ( logRangeMutationFirstLength < 0 ) {
+		printf("[ERROR]!!! logRangeMutationFirstLength:%ld < 0, key_input.size:%ld\n", logRangeMutationFirstLength, key_input.size());
+	}
+
+	if ( debug_verbose ) {
+		printf("[DEBUG] Process key_input:%s\n", getHexKey(key_input, logRangeMutationFirstLength).c_str());
+	}
+
+	//PARSE key
+	Standalone<StringRef> id_old = key_input.substr(0, key_input.size() - 4); //Used to sanity check the decoding of key is correct
+	Standalone<StringRef> partStr = key_input.substr(key_input.size() - 4, 4); //part
+	StringRefReaderMX readerPart(partStr, restore_corrupted_data());
+	uint32_t part_direct = readerPart.consumeNetworkUInt32(); //Consume a bigEndian value
+	if ( debug_verbose  ) {
+		printf("[DEBUG] Process prefix:%s and partStr:%s part_direct:%08x fromm key_input:%s, size:%ld\n",
+			   getHexKey(id_old, logRangeMutationFirstLength).c_str(),
+			   getHexString(partStr).c_str(),
+			   part_direct,
+			   getHexKey(key_input, logRangeMutationFirstLength).c_str(),
+			   key_input.size());
+	}
+
+	StringRef longRangeMutationFirst;
+
+	if ( logRangeMutationFirstLength > 0 ) {
+		printf("readerKey consumes %dB\n", logRangeMutationFirstLength);
+		longRangeMutationFirst = StringRef(readerKey.consume(logRangeMutationFirstLength), logRangeMutationFirstLength);
+	}
+
+	uint8_t hashValue = readerKey.consume<uint8_t>();
+	uint64_t commitVersion = readerKey.consumeNetworkUInt64(); // Consume big Endian value encoded in log file, commitVersion is in littleEndian
+	uint64_t commitVersionBE = bigEndian64(commitVersion);
+	uint32_t part = readerKey.consumeNetworkUInt32(); //Consume big Endian value encoded in log file
+	uint32_t partBE = bigEndian32(part);
+	Standalone<StringRef> id2 = longRangeMutationFirst.withSuffix(StringRef(&hashValue,1)).withSuffix(StringRef((uint8_t*) &commitVersion, 8));
+
+	//Use commitVersion as id
+	Standalone<StringRef> id = StringRef((uint8_t*) &commitVersion, 8);
+
+	if ( debug_verbose ) {
+		printf("[DEBUG] key_input_size:%d longRangeMutationFirst:%s hashValue:%02x commitVersion:%016lx (BigEndian:%016lx) part:%08x (BigEndian:%08x), part_direct:%08x mutationMap.size:%ld\n",
+			   key_input.size(), longRangeMutationFirst.printable().c_str(), hashValue,
+			   commitVersion, commitVersionBE,
+			   part, partBE,
+			   part_direct, self->mutationMap.size());
+	}
+
+	if ( self->mutationMap.find(id) == self->mutationMap.end() ) {
+		self->mutationMap.insert(std::make_pair(id, val_input));
+		if ( part_direct != 0 ) {
+			printf("[ERROR]!!! part:%d != 0 for key_input:%s\n", part_direct, getHexString(key_input).c_str());
+		}
+		self->mutationPartMap.insert(std::make_pair(id, part_direct));
+	} else { // concatenate the val string
+//		printf("[INFO] Concatenate the log's val string at version:%ld\n", id.toString().c_str());
+		self->mutationMap[id] = self->mutationMap[id].contents().withSuffix(val_input.contents()); //Assign the new Areana to the map's value
+		if ( part_direct != (self->mutationPartMap[id] + 1) ) {
+			printf("[ERROR]!!! current part id:%d new part_direct:%d is not the next integer of key_input:%s\n", self->mutationPartMap[id], part_direct, getHexString(key_input).c_str());
+			printf("[HINT] Check if the same range or log file has been processed more than once!\n");
+		}
+		if ( part_direct != part ) {
+			printf("part_direct:%08x != part:%08x\n", part_direct, part);
+		}
+		self->mutationPartMap[id] = part_direct;
+		concatenated = true;
+	}
+
+	return concatenated;
+}
+
+bool isRangeMutation(MutationRef m) {
+	if (m.type == MutationRef::Type::ClearRange) {
+		if (m.type == MutationRef::Type::DebugKeyRange) {
+			printf("[ERROR] DebugKeyRange mutation is in backup data unexpectedly. We still handle it as a range mutation; the suspicious mutation:%s\n", m.toString().c_str());
+		}
+		return true;
+	} else {
+		if ( !(m.type == MutationRef::Type::SetValue ||
+				isAtomicOp((MutationRef::Type) m.type)) ) {
+			printf("[ERROR] %s mutation is in backup data unexpectedly. We still handle it as a key mutation; the suspicious mutation:%s\n", typeString[m.type], m.toString().c_str());
+
+		}
+		return false;
+	}
+}
+
+
+ // Parse the kv pair (version, serialized_mutation), which are the results parsed from log file.
+ void parseSerializedMutation(Reference<RestoreLoaderData> self, bool isSampling) {
+	// Step: Parse the concatenated KV pairs into (version, <K, V, mutationType>) pair
+ 	printf("[INFO] Parse the concatenated log data\n");
+ 	std::string prefix = "||\t";
+	std::stringstream ss;
+	const int version_size = 12;
+	const int header_size = 12;
+	int kvCount = 0;
+
+	for ( auto& m : self->mutationMap ) {
+		StringRef k = m.first.contents();
+		StringRefReaderMX readerVersion(k, restore_corrupted_data());
+		uint64_t commitVersion = readerVersion.consume<uint64_t>(); // Consume little Endian data
+
+
+		StringRef val = m.second.contents();
+		StringRefReaderMX reader(val, restore_corrupted_data());
+
+		int count_size = 0;
+		// Get the include version in the batch commit, which is not the commitVersion.
+		// commitVersion is in the key
+		uint64_t includeVersion = reader.consume<uint64_t>();
+		count_size += 8;
+		uint32_t val_length_decode = reader.consume<uint32_t>(); //Parse little endian value, confirmed it is correct!
+		count_size += 4;
+
+		if ( self->kvOps.find(commitVersion) == self->kvOps.end() ) {
+			self->kvOps.insert(std::make_pair(commitVersion, VectorRef<MutationRef>()));
+		}
+
+		if ( debug_verbose ) {
+			printf("----------------------------------------------------------Register Backup Mutation into KVOPs version:%08lx\n", commitVersion);
+			printf("To decode value:%s\n", getHexString(val).c_str());
+		}
+		// In sampling, the last mutation vector may be not complete, we do not concatenate for performance benefit
+		if ( val_length_decode != (val.size() - 12) ) {
+			//IF we see val.size() == 10000, It means val should be concatenated! The concatenation may fail to copy the data
+			if (isSampling) {
+				printf("[PARSE WARNING]!!! val_length_decode:%d != val.size:%d version:%ld(0x%lx)\n",  val_length_decode, val.size(),
+					commitVersion, commitVersion);
+				printf("[PARSE WARNING] Skipped the mutation! OK for sampling workload but WRONG for restoring the workload\n");
+				continue;
+			} else {
+				printf("[PARSE ERROR]!!! val_length_decode:%d != val.size:%d version:%ld(0x%lx)\n",  val_length_decode, val.size(),
+					commitVersion, commitVersion);
+			}
+		} else {
+			if ( debug_verbose ) {
+				printf("[PARSE SUCCESS] val_length_decode:%d == (val.size:%d - 12)\n", val_length_decode, val.size());
+			}
+		}
+
+		// Get the mutation header
+		while (1) {
+			// stop when reach the end of the string
+			if(reader.eof() ) { //|| *reader.rptr == 0xFF
+				//printf("Finish decode the value\n");
+				break;
+			}
+
+
+			uint32_t type = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
+			uint32_t kLen = reader.consume<uint32_t>();//reader.consumeNetworkUInkvOps[t32();
+			uint32_t vLen = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
+			const uint8_t *k = reader.consume(kLen);
+			const uint8_t *v = reader.consume(vLen);
+			count_size += 4 * 3 + kLen + vLen;
+
+			MutationRef mutation((MutationRef::Type) type, KeyRef(k, kLen), KeyRef(v, vLen));
+			self->kvOps[commitVersion].push_back_deep(self->kvOps[commitVersion].arena(), mutation);
+			kvCount++;
+
+			if ( kLen < 0 || kLen > val.size() || vLen < 0 || vLen > val.size() ) {
+				printf("%s[PARSE ERROR]!!!! kLen:%d(0x%04x) vLen:%d(0x%04x)\n", prefix.c_str(), kLen, kLen, vLen, vLen);
+			}
+
+			if ( debug_verbose ) {
+				printf("%s---LogFile parsed mutations. Prefix:[%d]: Version:%016lx Type:%d K:%s V:%s k_size:%d v_size:%d\n", prefix.c_str(),
+					   kvCount,
+					   commitVersion, type,  getHexString(KeyRef(k, kLen)).c_str(), getHexString(KeyRef(v, vLen)).c_str(), kLen, vLen);
+			}
+
+		}
+		//	printf("----------------------------------------------------------\n");
+	}
+
+	printf("[INFO] Produces %d mutation operations from concatenated kv pairs that are parsed from log\n",  kvCount);
+
+}
+
+
+ACTOR static Future<Void> _parseRangeFileToMutationsOnLoader(Reference<RestoreLoaderData> self,
+ 									Reference<IBackupContainer> bc, Version version,
+ 									std::string fileName, int64_t readOffset_input, int64_t readLen_input,
+ 									KeyRange restoreRange, Key addPrefix, Key removePrefix) {
+
+ 	state int64_t readOffset = readOffset_input;
+ 	state int64_t readLen = readLen_input;
+
+	if ( debug_verbose ) {
+		printf("[VERBOSE_DEBUG] Parse range file and get mutations 1, bc:%lx\n", bc.getPtr());
+	}
+ 	// The set of key value version is rangeFile.version. the key-value set in the same range file has the same version
+ 	Reference<IAsyncFile> inFile = wait(bc->readFile(fileName));
+
+	if ( debug_verbose ) {
+		printf("[VERBOSE_DEBUG] Parse range file and get mutations 2\n");
+	}
+ 	state Standalone<VectorRef<KeyValueRef>> blockData = wait(parallelFileRestore::decodeRangeFileBlock(inFile, readOffset, readLen));
+
+	if ( debug_verbose ) {
+		printf("[VERBOSE_DEBUG] Parse range file and get mutations 3\n");
+		int tmpi = 0;
+		for (tmpi = 0; tmpi < blockData.size(); tmpi++) {
+			printf("\t[VERBOSE_DEBUG] mutation: key:%s value:%s\n", blockData[tmpi].key.toString().c_str(), blockData[tmpi].value.toString().c_str());
+		}
+	}
+
+ 	// First and last key are the range for this file
+ 	state KeyRange fileRange = KeyRangeRef(blockData.front().key, blockData.back().key);
+ 	printf("[INFO] RangeFile:%s KeyRange:%s, restoreRange:%s\n",
+ 			fileName.c_str(), fileRange.toString().c_str(), restoreRange.toString().c_str());
+
+ 	// If fileRange doesn't intersect restore range then we're done.
+ 	if(!fileRange.intersects(restoreRange)) {
+ 		TraceEvent("ExtractApplyRangeFileToDB_MX").detail("NoIntersectRestoreRange", "FinishAndReturn");
+ 		return Void();
+ 	}
+
+ 	// We know the file range intersects the restore range but there could still be keys outside the restore range.
+ 	// Find the subvector of kv pairs that intersect the restore range.  Note that the first and last keys are just the range endpoints for this file
+	 // The blockData's first and last entries are metadata, not the real data
+ 	int rangeStart = 1; //1
+ 	int rangeEnd = blockData.size() -1; //blockData.size() - 1 // Q: the rangeStart and rangeEnd is [,)?
+	if ( debug_verbose ) {
+		printf("[VERBOSE_DEBUG] Range file decoded blockData\n");
+		for (auto& data : blockData ) {
+			printf("\t[VERBOSE_DEBUG] data key:%s val:%s\n", data.key.toString().c_str(), data.value.toString().c_str());
+		}
+	}
+
+ 	// Slide start forwaself, stop if something in range is found
+	// Move rangeStart and rangeEnd until they is within restoreRange
+ 	while(rangeStart < rangeEnd && !restoreRange.contains(blockData[rangeStart].key)) {
+		if ( debug_verbose ) {
+			printf("[VERBOSE_DEBUG] rangeStart:%d key:%s is not in the range:%s\n", rangeStart, blockData[rangeStart].key.toString().c_str(), restoreRange.toString().c_str());
+		}
+		++rangeStart;
+	 }
+ 	// Side end backwaself, stop if something in range is found
+ 	while(rangeEnd > rangeStart && !restoreRange.contains(blockData[rangeEnd - 1].key)) {
+		if ( debug_verbose ) {
+			printf("[VERBOSE_DEBUG] (rangeEnd:%d - 1) key:%s is not in the range:%s\n", rangeEnd, blockData[rangeStart].key.toString().c_str(), restoreRange.toString().c_str());
+		}
+		--rangeEnd;
+	 }
+
+ 	// MX: now data only contains the kv mutation within restoreRange
+ 	state VectorRef<KeyValueRef> data = blockData.slice(rangeStart, rangeEnd);
+ 	printf("[INFO] RangeFile:%s blockData entry size:%d recovered data size:%d\n", fileName.c_str(), blockData.size(), data.size());
+
+ 	// Shrink file range to be entirely within restoreRange and translate it to the new prefix
+ 	// First, use the untranslated file range to create the shrunk original file range which must be used in the kv range version map for applying mutations
+ 	state KeyRange originalFileRange = KeyRangeRef(std::max(fileRange.begin, restoreRange.begin), std::min(fileRange.end,   restoreRange.end));
+
+ 	// Now shrink and translate fileRange
+ 	Key fileEnd = std::min(fileRange.end,   restoreRange.end);
+ 	if(fileEnd == (removePrefix == StringRef() ? normalKeys.end : strinc(removePrefix)) ) {
+ 		fileEnd = addPrefix == StringRef() ? normalKeys.end : strinc(addPrefix);
+ 	} else {
+ 		fileEnd = fileEnd.removePrefix(removePrefix).withPrefix(addPrefix);
+ 	}
+ 	fileRange = KeyRangeRef(std::max(fileRange.begin, restoreRange.begin).removePrefix(removePrefix).withPrefix(addPrefix),fileEnd);
+
+ 	state int start = 0;
+ 	state int end = data.size();
+ 	//state int dataSizeLimit = BUGGIFY ? g_random->randomInt(256 * 1024, 10e6) : CLIENT_KNOBS->RESTORE_WRITE_TX_SIZE;
+	state int dataSizeLimit = CLIENT_KNOBS->RESTORE_WRITE_TX_SIZE;
+ 	state int kvCount = 0;
+
+ 	//MX: This is where the key-value pair in range file is applied into DB
+	loop {
+
+		state int i = start;
+		state int txBytes = 0;
+		state int iend = start;
+
+		// find iend that results in the desired transaction size
+		for(; iend < end && txBytes < dataSizeLimit; ++iend) {
+			txBytes += data[iend].key.expectedSize();
+			txBytes += data[iend].value.expectedSize();
+		}
+
+
+		for(; i < iend; ++i) {
+			//MXX: print out the key value version, and operations.
+			if ( debug_verbose ) {
+				printf("RangeFile [key:%s, value:%s, version:%ld, op:set]\n", data[i].key.printable().c_str(), data[i].value.printable().c_str(), version);
+			}
+// 				TraceEvent("PrintRangeFile_MX").detail("Key", data[i].key.printable()).detail("Value", data[i].value.printable())
+// 					.detail("Version", rangeFile.version).detail("Op", "set");
+////				printf("PrintRangeFile_MX: mType:set param1:%s param2:%s param1_size:%d, param2_size:%d\n",
+////						getHexString(data[i].key.c_str(), getHexString(data[i].value).c_str(), data[i].key.size(), data[i].value.size());
+
+			//NOTE: Should NOT removePrefix and addPrefix for the backup data!
+			// In other woselfs, the following operation is wrong:  data[i].key.removePrefix(removePrefix).withPrefix(addPrefix)
+			MutationRef m(MutationRef::Type::SetValue, data[i].key, data[i].value); //ASSUME: all operation in range file is set.
+			++kvCount;
+
+			// TODO: we can commit the kv operation into DB.
+			// Right now, we cache all kv operations into kvOps, and apply all kv operations later in one place
+			if ( self->kvOps.find(version) == self->kvOps.end() ) { // Create the map's key if mutation m is the first on to be inserted
+				//kvOps.insert(std::make_pair(rangeFile.version, Standalone<VectorRef<MutationRef>>(VectorRef<MutationRef>())));
+				self->kvOps.insert(std::make_pair(version, VectorRef<MutationRef>()));
+			}
+
+			ASSERT(self->kvOps.find(version) != self->kvOps.end());
+			self->kvOps[version].push_back_deep(self->kvOps[version].arena(), m);
+
+		}
+
+		// Commit succeeded, so advance starting point
+		start = i;
+
+		if(start == end) {
+			//TraceEvent("ExtraApplyRangeFileToDB_MX").detail("Progress", "DoneApplyKVToDB");
+			printf("[INFO][Loader] NodeID:%s Parse RangeFile:%s: the number of kv operations = %d\n",
+					 self->describeNode().c_str(), fileName.c_str(), kvCount);
+			return Void();
+		}
+ 	}
+
+ }
+
+ ACTOR static Future<Void> _parseLogFileToMutationsOnLoader(Reference<RestoreLoaderData> self,
+ 									Reference<IBackupContainer> bc, Version version,
+ 									std::string fileName, int64_t readOffset, int64_t readLen,
+ 									KeyRange restoreRange, Key addPrefix, Key removePrefix,
+ 									Key mutationLogPrefix) {
+
+	// Step: concatenate the backuped param1 and param2 (KV) at the same version.
+ 	//state Key mutationLogPrefix = mutationLogPrefix;
+ 	//TraceEvent("ReadLogFileStart").detail("LogFileName", fileName);
+ 	state Reference<IAsyncFile> inFile = wait(bc->readFile(fileName));
+ 	//TraceEvent("ReadLogFileFinish").detail("LogFileName", fileName);
+
+ 	printf("Parse log file:%s readOffset:%d readLen:%ld\n", fileName.c_str(), readOffset, readLen);
+ 	//TODO: NOTE: decodeLogFileBlock() should read block by block! based on my serial version. This applies to decode range file as well
+ 	state Standalone<VectorRef<KeyValueRef>> data = wait(parallelFileRestore::decodeLogFileBlock(inFile, readOffset, readLen));
+ 	//state Standalone<VectorRef<MutationRef>> data = wait(fileBackup::decodeLogFileBlock_MX(inFile, readOffset, readLen)); //Decode log file
+ 	TraceEvent("ReadLogFileFinish").detail("LogFileName", fileName).detail("DecodedDataSize", data.contents().size());
+ 	printf("ReadLogFile, raw data size:%d\n", data.size());
+
+ 	state int start = 0;
+ 	state int end = data.size();
+ 	//state int dataSizeLimit = BUGGIFY ? g_random->randomInt(256 * 1024, 10e6) : CLIENT_KNOBS->RESTORE_WRITE_TX_SIZE;
+	state int dataSizeLimit = CLIENT_KNOBS->RESTORE_WRITE_TX_SIZE;
+	state int kvCount = 0;
+	state int numConcatenated = 0;
+	loop {
+ 		try {
+// 			printf("Process start:%d where end=%d\n", start, end);
+ 			if(start == end) {
+ 				printf("ReadLogFile: finish reading the raw data and concatenating the mutation at the same version\n");
+ 				break;
+ 			}
+
+ 			state int i = start;
+ 			state int txBytes = 0;
+ 			for(; i < end && txBytes < dataSizeLimit; ++i) {
+ 				Key k = data[i].key.withPrefix(mutationLogPrefix);
+ 				ValueRef v = data[i].value;
+ 				txBytes += k.expectedSize();
+ 				txBytes += v.expectedSize();
+ 				//MXX: print out the key value version, and operations.
+ 				//printf("LogFile [key:%s, value:%s, version:%ld, op:NoOp]\n", k.printable().c_str(), v.printable().c_str(), logFile.version);
+ //				printf("LogFile [KEY:%s, VALUE:%s, VERSION:%ld, op:NoOp]\n", getHexString(k).c_str(), getHexString(v).c_str(), logFile.version);
+ //				printBackupMutationRefValueHex(v, " |\t");
+ //				printf("[DEBUG]||Concatenate backup mutation:fileInfo:%s, data:%d\n", logFile.toString().c_str(), i);
+ 				bool concatenated = concatenateBackupMutationForLogFile(self, data[i].value, data[i].key);
+ 				numConcatenated += ( concatenated ? 1 : 0);
+ //				//TODO: Decode the value to get the mutation type. Use NoOp to distinguish from range kv for now.
+ //				MutationRef m(MutationRef::Type::NoOp, data[i].key, data[i].value); //ASSUME: all operation in log file is NoOp.
+ //				if ( self->kvOps.find(logFile.version) == self->kvOps.end() ) {
+ //					self->kvOps.insert(std::make_pair(logFile.version, std::vector<MutationRef>()));
+ //				} else {
+ //					self->kvOps[logFile.version].push_back(m);
+ //				}
+ 			}
+
+ 			start = i;
+
+ 		} catch(Error &e) {
+ 			if(e.code() == error_code_transaction_too_large)
+ 				dataSizeLimit /= 2;
+ 		}
+ 	}
+
+ 	printf("[INFO] raw kv number:%d parsed from log file, concatenated:%d kv, num_log_versions:%d\n", data.size(), numConcatenated, self->mutationMap.size());
+
+	return Void();
+ }
diff --git a/fdbserver/RestoreLoader.actor.h b/fdbserver/RestoreLoader.actor.h
index c86e6442e2..36150b4fc2 100644
--- a/fdbserver/RestoreLoader.actor.h
+++ b/fdbserver/RestoreLoader.actor.h
@@ -21,7 +21,7 @@
 // Declear RestoreLoader interface and actors
 
 #pragma once
-#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_RestoreLoaderInterface_H)
+#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_RestoreLoaderInterface_G_H)
 	#define FDBSERVER_RestoreLoaderInterface_G_H
 	#include "fdbserver/RestoreLoader.actor.g.h"
 #elif !defined(FDBSERVER_RestoreLoaderInterface_H)
@@ -35,5 +35,111 @@
 #include "fdbserver/CoordinationInterface.h"
 #include "fdbrpc/Locality.h"
 
+#include "fdbserver/RestoreUtil.h"
+#include "fdbserver/RestoreCommon.actor.h"
+#include "fdbserver/RestoreRoleCommon.actor.h"
+#include "fdbserver/RestoreWorkerInterface.h"
+#include "fdbclient/BackupContainer.h"
 
+#include "flow/actorcompiler.h" // has to be last include
+
+struct RestoreLoaderData : RestoreRoleData, public ReferenceCounted<RestoreLoaderData> {
+public:	
+	// range2Applier is in master and loader node. Loader node uses this to determine which applier a mutation should be sent
+	std::map<Standalone<KeyRef>, UID> range2Applier; // KeyRef is the inclusive lower bound of the key range the applier (UID) is responsible for
+	std::map<Standalone<KeyRef>, int> keyOpsCount; // The number of operations per key which is used to determine the key-range boundary for appliers
+	int numSampledMutations; // The total number of mutations received from sampled data.
+
+	// Loader's state to handle the duplicate delivery of loading commands
+	std::map<std::string, int> processedFiles; //first is filename of processed file, second is not used
+
+	// Temporary data structure for parsing range and log files into (version, <K, V, mutationType>)
+	std::map<Version, Standalone<VectorRef<MutationRef>>> kvOps;
+	// Must use StandAlone to save mutations, otherwise, the mutationref memory will be corrupted
+	std::map<Standalone<StringRef>, Standalone<StringRef>> mutationMap; // Key is the unique identifier for a batch of mutation logs at the same version
+	std::map<Standalone<StringRef>, uint32_t> mutationPartMap; // Recoself the most recent
+
+
+	Reference<IBackupContainer> bc; // Backup container is used to read backup files
+	Key bcUrl; // The url used to get the bc
+
+	CMDUID cmdID;
+
+    // Performance statistics
+    double curWorkloadSize;
+
+	void addref() { return ReferenceCounted<RestoreLoaderData>::addref(); }
+	void delref() { return ReferenceCounted<RestoreLoaderData>::delref(); }
+
+	RestoreLoaderData() {
+		nodeID = g_random->randomUniqueID();
+		nodeIndex = 0;
+	}
+
+	~RestoreLoaderData() {}
+
+	std::string describeNode() {
+		std::stringstream ss;
+		ss << "[Role: Loader] [NodeID:" << nodeID.toString().c_str()
+			<< "] [NodeIndex:" << std::to_string(nodeIndex) << "]";
+		return ss.str();
+	}
+
+    void resetPerVersionBatch() {
+		printf("[INFO]Node:%s resetPerVersionBatch\n", nodeID.toString().c_str());
+		RestoreRoleData::resetPerVersionBatch();
+
+		range2Applier.clear();
+		keyOpsCount.clear();
+		numSampledMutations = 0;
+		
+		processedFiles.clear();
+		
+        kvOps.clear();
+        mutationMap.clear();
+        mutationPartMap.clear();
+
+		curWorkloadSize = 0;
+	}
+
+	vector<UID> getBusyAppliers() {
+		vector<UID> busyAppliers;
+		for (auto &app : range2Applier) {
+			busyAppliers.push_back(app.second);
+		}
+		return busyAppliers;
+	}
+
+    std::vector<UID> getWorkingApplierIDs() {
+        std::vector<UID> applierIDs;
+        for ( auto &applier : range2Applier ) {
+            applierIDs.push_back(applier.second);
+        }
+
+        ASSERT( !applierIDs.empty() );
+        return applierIDs;
+    }
+
+	void initBackupContainer(Key url) {
+		if ( bcUrl == url && bc.isValid() ) {
+			return;
+		}
+		printf("initBackupContainer, url:%s\n", url.toString().c_str());
+		bcUrl = url;
+		bc = IBackupContainer::openContainer(url.toString());
+	}
+
+	void printAppliersKeyRange() {
+		printf("[INFO] The mapping of KeyRange_start --> Applier ID\n");
+		// applier type: std::map<Standalone<KeyRef>, UID>
+		for (auto &applier : range2Applier) {
+			printf("\t[INFO]%s -> %s\n", getHexString(applier.first).c_str(), applier.second.toString().c_str());
+		}
+	}
+};
+
+
+ACTOR Future<Void> restoreLoaderCore(Reference<RestoreLoaderData> self, RestoreLoaderInterface loaderInterf, Database cx);
+
+#include "flow/unactorcompiler.h"
 #endif
\ No newline at end of file
diff --git a/fdbserver/RestoreMaster.actor.cpp b/fdbserver/RestoreMaster.actor.cpp
new file mode 100644
index 0000000000..c414a24f1c
--- /dev/null
+++ b/fdbserver/RestoreMaster.actor.cpp
@@ -0,0 +1,1326 @@
+/*
+ * RestoreMaster.actor.cpp
+ *
+ * This source file is part of the FoundationDB open source project
+ *
+ * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
+ *
+ * Licensed 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.
+ */
+
+
+#include "fdbclient/NativeAPI.actor.h"
+#include "fdbclient/SystemData.h"
+
+// Backup agent header
+#include "fdbclient/BackupAgent.actor.h"
+//#include "FileBackupAgent.h"
+#include "fdbclient/ManagementAPI.actor.h"
+#include "fdbclient/MutationList.h"
+#include "fdbclient/BackupContainer.h"
+
+#include "fdbserver/RestoreCommon.actor.h"
+#include "fdbserver/RestoreRoleCommon.actor.h"
+#include "fdbserver/RestoreMaster.actor.h"
+#include "fdbserver/RestoreApplier.actor.h"
+#include "fdbserver/RestoreLoader.actor.h"
+
+#include "flow/actorcompiler.h"  // This must be the last #include.
+
+ACTOR Future<Void> askLoadersToCollectRestoreAppliersInterfaces(Reference<RestoreMasterData> self);
+ACTOR Future<Standalone<VectorRef<RestoreRequest>>> collectRestoreRequests(Database cx);
+ACTOR static Future<Version> processRestoreRequest(RestoreRequest request, Reference<RestoreMasterData> self, Database cx);
+ACTOR static Future<Void> finishRestore(Reference<RestoreMasterData> self, Database cx, Standalone<VectorRef<RestoreRequest>> restoreRequests);
+
+ACTOR static Future<Void> _collectBackupFiles(Reference<RestoreMasterData> self, Database cx, RestoreRequest request);
+ACTOR Future<Void> initializeVersionBatch(Reference<RestoreMasterData> self);
+ACTOR static Future<Void> distributeWorkloadPerVersionBatch(Reference<RestoreMasterData> self, Database cx, RestoreRequest request, Reference<RestoreConfig> restoreConfig);
+ACTOR static Future<Void> unlockDB(Database cx, UID uid);
+ACTOR static Future<Void> _clearDB(Reference<ReadYourWritesTransaction> tr);
+ACTOR static Future<Void> _lockDB(Database cx, UID uid, bool lockDB);
+ACTOR static Future<Void> registerStatus(Database cx, struct FastRestoreStatus status);
+ACTOR static Future<Void> sampleWorkload(Reference<RestoreMasterData> self, RestoreRequest request, Reference<RestoreConfig> restoreConfig, int64_t sampleMB_input);
+ACTOR Future<Void> notifyAppliersKeyRangeToLoader(Reference<RestoreMasterData> self, Database cx);
+ACTOR Future<Void> assignKeyRangeToAppliers(Reference<RestoreMasterData> self, Database cx);
+ACTOR Future<Void> notifyApplierToApplyMutations(Reference<RestoreMasterData> self);
+
+
+ACTOR Future<Void> startRestoreMaster(Reference<RestoreMasterData> self, Database cx) {
+	try {
+		wait( delay(1.0) );
+		wait( _collectRestoreRoleInterfaces(self, cx) );
+
+		wait( delay(1.0) );
+		wait( askLoadersToCollectRestoreAppliersInterfaces(self) );
+
+		state int restoreId = 0;
+		state int checkNum = 0;
+		loop {
+			printf("Node:%s---Wait on restore requests...---\n", self->describeNode().c_str());
+			state Standalone<VectorRef<RestoreRequest>> restoreRequests = wait( collectRestoreRequests(cx) );
+
+			printf("Node:%s ---Received  restore requests as follows---\n", self->describeNode().c_str());
+			// Print out the requests info
+			for ( auto &it : restoreRequests ) {
+				printf("\t[INFO][Master]Node:%s RestoreRequest info:%s\n", self->describeNode().c_str(), it.toString().c_str());
+			}
+
+			// Step: Perform the restore requests
+			for ( auto &it : restoreRequests ) {
+				TraceEvent("LeaderGotRestoreRequest").detail("RestoreRequestInfo", it.toString());
+				printf("Node:%s Got RestoreRequestInfo:%s\n", self->describeNode().c_str(), it.toString().c_str());
+				Version ver = wait( processRestoreRequest(it, self, cx) );
+			}
+
+			// Step: Notify all restore requests have been handled by cleaning up the restore keys
+			wait( delay(5.0) );
+			printf("Finish my restore now!\n");
+			//wait( finishRestore(self) );
+			wait( finishRestore(self, cx, restoreRequests) ); 
+
+			printf("[INFO] MXRestoreEndHere RestoreID:%d\n", restoreId);
+			TraceEvent("MXRestoreEndHere").detail("RestoreID", restoreId++);
+			wait( delay(5.0) );
+			//NOTE: we have to break the loop so that the tester.actor can receive the return of this test workload.
+			//Otherwise, this special workload never returns and tester will think the test workload is stuck and the tester will timesout
+			break; //TODO: this break will be removed later since we need the restore agent to run all the time!
+		}
+
+		return Void();
+
+	} catch (Error &e) {
+		fprintf(stdout, "[ERROR] Restoer Master encounters error. error code:%d, error message:%s\n",
+				e.code(), e.what());
+	}
+
+	return Void();
+}
+
+
+
+ACTOR static Future<Version> processRestoreRequest(RestoreRequest request, Reference<RestoreMasterData> self, Database cx) {
+	state Key tagName = request.tagName;
+	state Key url = request.url;
+	state bool waitForComplete = request.waitForComplete;
+	state Version targetVersion = request.targetVersion;
+	state bool verbose = request.verbose;
+	state KeyRange range = request.range;
+	state Key addPrefix = request.addPrefix;
+	state Key removePrefix = request.removePrefix;
+	state bool lockDB = request.lockDB;
+	state UID randomUid = request.randomUid;
+
+	//MX: Lock DB if it is not locked
+	printf("RestoreRequest lockDB:%d\n", lockDB);
+	if ( lockDB == false ) {
+		printf("[WARNING] RestoreRequest lockDB:%d; we will overwrite request.lockDB to true and forcely lock db\n", lockDB);
+		lockDB = true;
+		request.lockDB = true;
+	}
+
+	state long curBackupFilesBeginIndex = 0;
+	state long curBackupFilesEndIndex = 0;
+
+	state double totalWorkloadSize = 0;
+	state double totalRunningTime = 0; // seconds
+	state double curRunningTime = 0; // seconds
+	state double curStartTime = 0;
+	state double curEndTime = 0;
+	state double curWorkloadSize = 0; //Bytes
+
+	
+	state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
+	state Reference<RestoreConfig> restoreConfig(new RestoreConfig(randomUid));
+
+	// lock DB for restore
+	wait( _lockDB(cx, randomUid, lockDB) );
+	wait( _clearDB(tr) );
+
+	// Step: Collect all backup files
+	printf("===========Restore request start!===========\n");
+	state double startTime = now();
+	wait( _collectBackupFiles(self, cx, request) );
+	printf("[Perf] Node:%s collectBackupFiles takes %.2f seconds\n", self->describeNode().c_str(), now() - startTime);
+	self->constructFilesWithVersionRange();
+	self->files.clear(); // Ensure no mistakely use self->files
+	
+	// Sort the backup files based on end version.
+	sort(self->allFiles.begin(), self->allFiles.end());
+	self->printAllBackupFilesInfo();
+
+	self->buildForbiddenVersionRange();
+	self->printForbiddenVersionRange();
+	if ( self->isForbiddenVersionRangeOverlapped() ) {
+		fprintf(stderr, "[ERROR] forbidden version ranges are overlapped! Check out the forbidden version range above\n");
+	}
+
+	self->batchIndex = 0;
+	state int prevBatchIndex = 0;
+	state long prevCurBackupFilesBeginIndex = 0;
+	state long prevCurBackupFilesEndIndex = 0;
+	state double prevCurWorkloadSize = 0;
+	state double prevtotalWorkloadSize = 0;
+
+	loop {
+		try {
+			curStartTime = now();
+			self->files.clear();
+			self->resetPerVersionBatch();
+			self->cmdID.setBatch(self->batchIndex);
+			// Checkpoint the progress of the previous version batch
+			prevBatchIndex = self->batchIndex;
+			prevCurBackupFilesBeginIndex = self->curBackupFilesBeginIndex;
+			prevCurBackupFilesEndIndex = self->curBackupFilesEndIndex;
+			prevCurWorkloadSize = self->curWorkloadSize;
+			prevtotalWorkloadSize = self->totalWorkloadSize;
+			
+			bool hasBackupFilesToProcess = self->collectFilesForOneVersionBatch();
+			if ( !hasBackupFilesToProcess ) { // No more backup files to restore
+				printf("No backup files to process any more\n");
+				break;
+			}
+
+			printf("[Progress][Start version batch] Node:%s, restoreBatchIndex:%d, curWorkloadSize:%.2f------\n", self->describeNode().c_str(), self->batchIndex, self->curWorkloadSize);
+
+			wait( initializeVersionBatch(self) );
+
+			wait( delay(1.0) );
+
+			wait( distributeWorkloadPerVersionBatch(self, cx, request, restoreConfig) );
+
+			curEndTime = now();
+			curRunningTime = curEndTime - curStartTime;
+			ASSERT(curRunningTime >= 0);
+			totalRunningTime += curRunningTime;
+
+			struct FastRestoreStatus status;
+			status.curRunningTime = curRunningTime;
+			status.curWorkloadSize = self->curWorkloadSize;
+			status.curSpeed = self->curWorkloadSize /  curRunningTime;
+			status.totalRunningTime = totalRunningTime;
+			status.totalWorkloadSize = self->totalWorkloadSize;
+			status.totalSpeed = self->totalWorkloadSize / totalRunningTime;
+
+			printf("[Progress][Finish version batch] restoreBatchIndex:%d, curWorkloadSize:%.2f B, curWorkload:%.2f B curRunningtime:%.2f s curSpeed:%.2f B/s  totalWorkload:%.2f B totalRunningTime:%.2f s totalSpeed:%.2f B/s\n",
+					self->batchIndex, self->curWorkloadSize,
+					status.curWorkloadSize, status.curRunningTime, status.curSpeed, status.totalWorkloadSize, status.totalRunningTime, status.totalSpeed);
+
+			wait( registerStatus(cx, status) );
+			printf("[Progress] Finish 1 version batch. curBackupFilesBeginIndex:%ld curBackupFilesEndIndex:%ld allFiles.size():%ld",
+				self->curBackupFilesBeginIndex, self->curBackupFilesEndIndex, self->allFiles.size());
+
+			self->curBackupFilesBeginIndex = self->curBackupFilesEndIndex + 1;
+			self->curBackupFilesEndIndex++;
+			self->curWorkloadSize = 0;
+			self->batchIndex++;
+
+		} catch(Error &e) {
+			fprintf(stdout, "!!![MAY HAVE BUG] Reset the version batch state to the start of the current version batch, due to error:%s\n", e.what());
+			if(e.code() != error_code_restore_duplicate_tag) {
+				wait(tr->onError(e));
+			}
+			self->batchIndex = prevBatchIndex;
+			self->curBackupFilesBeginIndex = prevCurBackupFilesBeginIndex;
+			self->curBackupFilesEndIndex = prevCurBackupFilesEndIndex;
+			self->curWorkloadSize = prevCurWorkloadSize;
+			self->totalWorkloadSize = prevtotalWorkloadSize;
+		}
+	}
+
+	// Unlock DB  at the end of handling the restore request
+	wait( unlockDB(cx, randomUid) );
+	printf("Finish restore uid:%s \n", randomUid.toString().c_str());
+
+	return targetVersion;
+}
+
+// Distribution workload per version batch
+ACTOR static Future<Void> distributeWorkloadPerVersionBatch(Reference<RestoreMasterData> self, Database cx, RestoreRequest request, Reference<RestoreConfig> restoreConfig) {
+	state Key tagName = request.tagName;
+	state Key url = request.url;
+	state bool waitForComplete = request.waitForComplete;
+	state Version targetVersion = request.targetVersion;
+	state bool verbose = request.verbose;
+	state KeyRange restoreRange = request.range;
+	state Key addPrefix = request.addPrefix;
+	state Key removePrefix = request.removePrefix;
+	state bool lockDB = request.lockDB;
+	state UID randomUid = request.randomUid;
+	state Key mutationLogPrefix = restoreConfig->mutationLogPrefix();
+
+	if ( self->isBackupEmpty() ) {
+		printf("[WARNING] Node:%s distributeWorkloadPerVersionBatch() load an empty batch of backup. Print out the empty backup files info.\n", self->describeNode().c_str());
+		self->printBackupFilesInfo();
+		return Void();
+	}
+
+	printf("[INFO] Node:%s mutationLogPrefix:%s (hex value:%s)\n", self->describeNode().c_str(), mutationLogPrefix.toString().c_str(), getHexString(mutationLogPrefix).c_str());
+
+	// Determine the key range each applier is responsible for
+	int numLoaders = self->loadersInterf.size();
+	int numAppliers = self->appliersInterf.size();
+	ASSERT( numLoaders > 0 );
+	ASSERT( numAppliers > 0 );
+
+	state int loadingSizeMB = 0; //numLoaders * 1000; //NOTE: We want to load the entire file in the first version, so we want to make this as large as possible
+	int64_t sampleSizeMB = 0; //loadingSizeMB / 100; // Will be overwritten. The sampleSizeMB will be calculated based on the batch size
+
+	state double startTime = now();
+	state double startTimeBeforeSampling = now();
+	
+	wait( sampleWorkload(self, request, restoreConfig, sampleSizeMB) );
+	wait( delay(1.0) );
+
+	printf("[Progress] distributeWorkloadPerVersionBatch sampling time:%.2f seconds\n", now() - startTime);
+	state double startTimeAfterSampling = now();
+
+	// Notify each applier about the key range it is responsible for, and notify appliers to be ready to receive data
+	startTime = now();
+	wait( assignKeyRangeToAppliers(self, cx) );
+	wait( delay(1.0) );
+	printf("[Progress] distributeWorkloadPerVersionBatch assignKeyRangeToAppliers time:%.2f seconds\n", now() - startTime);
+
+	startTime = now();
+	wait( notifyAppliersKeyRangeToLoader(self, cx) );
+	wait( delay(1.0) );
+	printf("[Progress] distributeWorkloadPerVersionBatch notifyAppliersKeyRangeToLoader time:%.2f seconds\n", now() - startTime);
+
+	// Determine which backup data block (filename, offset, and length) each loader is responsible for and
+	// Notify the loader about the data block and send the cmd to the loader to start loading the data
+	// Wait for the ack from loader and repeats
+
+	// Prepare the file's loading status
+	for (int i = 0; i < self->files.size(); ++i) {
+		self->files[i].cursor = 0;
+	}
+
+	// Send loading cmd to available loaders whenever loaders become available
+	// NOTE: We must split the workload in the correct boundary:
+	// For range file, it's the block boundary;
+	// For log file, it is the version boundary.
+	// This is because
+	// (1) The set of mutations at a version may be encoded in multiple KV pairs in log files.
+	// We need to concatenate the related KVs to a big KV before we can parse the value into a vector of mutations at that version
+	// (2) The backuped KV are arranged in blocks in range file.
+	// For simplicity, we distribute at the granularity of files for now.
+
+	state int loadSizeB = loadingSizeMB * 1024 * 1024;
+	state int loadingCmdIndex = 0;
+
+	state int checkpointCurFileIndex = 0;
+	state long checkpointCurOffset = 0; 
+
+	startTime = now();
+	// We should load log file before we do range file
+	state RestoreCommandEnum phaseType = RestoreCommandEnum::Assign_Loader_Log_File;
+	state std::vector<Future<RestoreCommonReply>> cmdReplies;
+	loop {
+		state int curFileIndex = 0; // The smallest index of the files that has not been FULLY loaded
+		state long curOffset = 0;
+		state bool allLoadReqsSent = false;
+		loop {
+			try {
+				if ( allLoadReqsSent ) {
+					break; // All load requests have been handled
+				}
+				wait(delay(1.0));
+
+				cmdReplies.clear();
+				printf("[INFO] Number of backup files:%ld\n", self->files.size());
+				self->cmdID.initPhase(phaseType);
+				for (auto &loader : self->loadersInterf) {
+					UID loaderID = loader.first;
+					RestoreLoaderInterface loaderInterf = loader.second;
+
+					while (  curFileIndex < self->files.size() && self->files[curFileIndex].fileSize == 0 ) {
+						// NOTE: && self->files[curFileIndex].cursor >= self->files[curFileIndex].fileSize
+						printf("[INFO] File %ld:%s filesize:%ld skip the file\n", curFileIndex,
+								self->files[curFileIndex].fileName.c_str(), self->files[curFileIndex].fileSize);
+						curFileIndex++;
+						curOffset = 0;
+					}
+					if ( curFileIndex >= self->files.size() ) {
+						allLoadReqsSent = true;
+						break;
+					}
+					LoadingParam param;
+					//self->files[curFileIndex].cursor = 0; // This is a hacky way to make sure cursor is correct in current version when we load 1 file at a time
+					param.url = request.url;
+					param.version = self->files[curFileIndex].version;
+					param.filename = self->files[curFileIndex].fileName;
+					param.offset = curOffset; //self->files[curFileIndex].cursor;
+					param.length = std::min(self->files[curFileIndex].fileSize - curOffset, self->files[curFileIndex].blockSize);
+					//param.length = self->files[curFileIndex].fileSize;
+					loadSizeB = param.length;
+					param.blockSize = self->files[curFileIndex].blockSize;
+					param.restoreRange = restoreRange;
+					param.addPrefix = addPrefix;
+					param.removePrefix = removePrefix;
+					param.mutationLogPrefix = mutationLogPrefix;
+					if ( !(param.length > 0  &&  param.offset >= 0 && param.offset < self->files[curFileIndex].fileSize) ) {
+						printf("[ERROR] param: length:%ld offset:%ld fileSize:%ld for %ldth filename:%s\n",
+								param.length, param.offset, self->files[curFileIndex].fileSize, curFileIndex,
+								self->files[curFileIndex].fileName.c_str());
+					}
+					ASSERT( param.length > 0 );
+					ASSERT( param.offset >= 0 );
+					ASSERT( param.offset < self->files[curFileIndex].fileSize );
+					self->files[curFileIndex].cursor = self->files[curFileIndex].cursor +  param.length;
+
+					RestoreCommandEnum cmdType = RestoreCommandEnum::Assign_Loader_Range_File;
+					if (self->files[curFileIndex].isRange) {
+						cmdType = RestoreCommandEnum::Assign_Loader_Range_File;
+						self->cmdID.setPhase(RestoreCommandEnum::Assign_Loader_Range_File);
+					} else {
+						cmdType = RestoreCommandEnum::Assign_Loader_Log_File;
+						self->cmdID.setPhase(RestoreCommandEnum::Assign_Loader_Log_File);
+					}
+
+					if ( (phaseType == RestoreCommandEnum::Assign_Loader_Log_File && self->files[curFileIndex].isRange) 
+						|| (phaseType == RestoreCommandEnum::Assign_Loader_Range_File && !self->files[curFileIndex].isRange) ) {
+						self->files[curFileIndex].cursor = 0;
+						curFileIndex++;
+						curOffset = 0;
+					} else { // load the type of file in the phaseType
+						self->cmdID.nextCmd();
+						printf("[CMD] Loading fileIndex:%ld fileInfo:%s loadingParam:%s on node %s\n",
+							curFileIndex, self->files[curFileIndex].toString().c_str(), 
+							param.toString().c_str(), loaderID.toString().c_str()); // VERY USEFUL INFO
+						printf("[INFO] Node:%s CMDUID:%s cmdType:%d isRange:%d loaderNode:%s\n", self->describeNode().c_str(), self->cmdID.toString().c_str(),
+								(int) cmdType, (int) self->files[curFileIndex].isRange, loaderID.toString().c_str());
+						if (self->files[curFileIndex].isRange) {
+							cmdReplies.push_back( loaderInterf.loadRangeFile.getReply(RestoreLoadFileRequest(self->cmdID, param)) );
+						} else {
+							cmdReplies.push_back( loaderInterf.loadLogFile.getReply(RestoreLoadFileRequest(self->cmdID, param)) );
+						}
+						curOffset += param.length;
+
+						// Reach the end of the file
+						if ( param.length + param.offset >= self->files[curFileIndex].fileSize ) {
+							curFileIndex++;
+							curOffset = 0;
+						}
+						
+						// if (param.length <= loadSizeB) { // Reach the end of the file
+						// 	ASSERT( self->files[curFileIndex].cursor == self->files[curFileIndex].fileSize );
+						// 	curFileIndex++;
+						// }
+					}
+					
+					if ( curFileIndex >= self->files.size() ) {
+						allLoadReqsSent = true;
+						break;
+					}
+					//++loadingCmdIndex; // Replaced by cmdUID
+				}
+
+				printf("[INFO] Wait for %ld loaders to accept the cmd Assign_Loader_File\n", cmdReplies.size());
+
+				// Question: How to set reps to different value based on cmdReplies.empty()?
+				if ( !cmdReplies.empty() ) {
+					std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) ); //TODO: change to getAny. NOTE: need to keep the still-waiting replies
+					//std::vector<RestoreCommonReply> reps = wait( getAll(cmdReplies) ); 
+
+					cmdReplies.clear();
+					for (int i = 0; i < reps.size(); ++i) {
+						printf("[INFO] Get Ack reply:%s for Assign_Loader_File\n",
+								reps[i].toString().c_str());
+					}
+					checkpointCurFileIndex = curFileIndex; // Save the previous success point
+					checkpointCurOffset = curOffset;
+				}
+
+				// TODO: Let master print all nodes status. Note: We need a function to print out all nodes status
+
+				if (allLoadReqsSent) {
+					printf("[INFO] allLoadReqsSent has finished.\n");
+					break; // NOTE: need to change when change to wait on any cmdReplies
+				}
+
+			} catch (Error &e) {
+				// TODO: Handle the command reply timeout error
+				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", self->describeNode().c_str(),
+						self->cmdID.toString().c_str(), e.code(), e.what());
+				curFileIndex = checkpointCurFileIndex;
+				curOffset = checkpointCurOffset;
+			}
+		}
+
+		if (phaseType == RestoreCommandEnum::Assign_Loader_Log_File) {
+			phaseType = RestoreCommandEnum::Assign_Loader_Range_File;
+		} else if (phaseType == RestoreCommandEnum::Assign_Loader_Range_File) {
+			break;
+		}
+	}
+
+	wait( delay(1.0) );
+	printf("[Progress] distributeWorkloadPerVersionBatch loadFiles time:%.2f seconds\n", now() - startTime);
+
+	ASSERT( cmdReplies.empty() );
+	
+	wait( delay(5.0) );
+	// Notify the applier to applly mutation to DB
+
+	startTime = now();
+	wait( notifyApplierToApplyMutations(self) );
+	printf("[Progress] distributeWorkloadPerVersionBatch applyToDB time:%.2f seconds\n", now() - startTime);
+
+	state double endTime = now();
+
+	double runningTime = endTime - startTimeBeforeSampling;
+	printf("[Progress] Node:%s distributeWorkloadPerVersionBatch runningTime without sampling time:%.2f seconds, with sampling time:%.2f seconds\n",
+			self->describeNode().c_str(),
+			runningTime, endTime - startTimeAfterSampling);
+
+	return Void();
+
+}
+
+
+// RestoreMaster: Ask loaders to sample data and send mutations to master applier. Ask master applier to calculate the range for each applier
+ACTOR static Future<Void> sampleWorkload(Reference<RestoreMasterData> self, RestoreRequest request, Reference<RestoreConfig> restoreConfig, int64_t sampleMB_input) {
+	state Key tagName = request.tagName;
+	state Key url = request.url;
+	state bool waitForComplete = request.waitForComplete;
+	state Version targetVersion = request.targetVersion;
+	state bool verbose = request.verbose;
+	state KeyRange restoreRange = request.range;
+	state Key addPrefix = request.addPrefix;
+	state Key removePrefix = request.removePrefix;
+	state bool lockDB = request.lockDB;
+	state UID randomUid = request.randomUid;
+	state Key mutationLogPrefix = restoreConfig->mutationLogPrefix();
+
+	state bool allLoadReqsSent = false;
+	state int64_t sampleMB = sampleMB_input; //100;
+	state int64_t sampleB = sampleMB * 1024 * 1024; // Sample a block for every sampleB bytes. // Should adjust this value differently for simulation mode and real mode
+	state int64_t curFileIndex = 0;
+	state int64_t curFileOffset = 0;
+	state int64_t loadSizeB = 0;
+	state int64_t loadingCmdIndex = 0;
+	state int64_t sampleIndex = 0;
+	state double totalBackupSizeB = 0;
+	state double samplePercent = 0.05; // sample 1 data block per samplePercent (0.01) of data. num_sample = 1 / samplePercent
+
+	// We should sample 1% data
+	for (int i = 0; i < self->files.size(); i++) {
+		totalBackupSizeB += self->files[i].fileSize;
+	}
+	sampleB = std::max((int) (samplePercent * totalBackupSizeB), 10 * 1024 * 1024); // The minimal sample size is 10MB
+	printf("Node:%s totalBackupSizeB:%.1fB (%.1fMB) samplePercent:%.2f, sampleB:%ld\n", self->describeNode().c_str(),
+			totalBackupSizeB,  totalBackupSizeB / 1024 / 1024, samplePercent, sampleB);
+
+	// Step: Distribute sampled file blocks to loaders to sample the mutations
+	self->cmdID.initPhase(RestoreCommandEnum::Sample_Range_File);
+	curFileIndex = 0;
+	state CMDUID checkpointCMDUID = self->cmdID;
+	state int checkpointCurFileIndex = curFileIndex;
+	state int64_t checkpointCurFileOffset = 0;
+	state std::vector<Future<RestoreCommonReply>> cmdReplies;
+	state RestoreCommandEnum cmdType;
+	loop { // For retry on timeout
+		try {
+			if ( allLoadReqsSent ) {
+				break; // All load requests have been handled
+			}
+			wait(delay(1.0));
+
+			cmdReplies.clear();
+
+			printf("[Sampling] Node:%s We will sample the workload among %ld backup files.\n", self->describeNode().c_str(), self->files.size());
+			printf("[Sampling] Node:%s totalBackupSizeB:%.1fB (%.1fMB) samplePercent:%.2f, sampleB:%ld, loadSize:%dB sampleIndex:%ld\n", self->describeNode().c_str(),
+				totalBackupSizeB,  totalBackupSizeB / 1024 / 1024, samplePercent, sampleB, loadSizeB, sampleIndex);
+			for (auto &loader : self->loadersInterf) {
+				const UID &loaderID = loader.first;
+				RestoreLoaderInterface &loaderInterf= loader.second;
+
+				// Find the sample file
+				while ( curFileIndex < self->files.size() && self->files[curFileIndex].fileSize == 0 ) {
+					// NOTE: && self->files[curFileIndex].cursor >= self->files[curFileIndex].fileSize
+					printf("[Sampling] File %ld:%s filesize:%ld skip the file\n", curFileIndex,
+							self->files[curFileIndex].fileName.c_str(), self->files[curFileIndex].fileSize);
+					curFileOffset = 0;
+					curFileIndex++;
+				}
+				// Find the next sample point
+				while ( loadSizeB / sampleB < sampleIndex && curFileIndex < self->files.size() ) {
+					if (self->files[curFileIndex].fileSize == 0) {
+						// NOTE: && self->files[curFileIndex].cursor >= self->files[curFileIndex].fileSize
+						printf("[Sampling] File %ld:%s filesize:%ld skip the file\n", curFileIndex,
+								self->files[curFileIndex].fileName.c_str(), self->files[curFileIndex].fileSize);
+						curFileIndex++;
+						curFileOffset = 0;
+						continue;
+					}
+					if ( loadSizeB / sampleB >= sampleIndex ) {
+						break;
+					}
+					if (curFileIndex >= self->files.size()) {
+						break;
+					}
+					loadSizeB += std::min( self->files[curFileIndex].blockSize, std::max(self->files[curFileIndex].fileSize - curFileOffset * self->files[curFileIndex].blockSize, (int64_t) 0) );
+					curFileOffset++;
+					if ( self->files[curFileIndex].blockSize == 0 || curFileOffset >= self->files[curFileIndex].fileSize / self->files[curFileIndex].blockSize ) {
+						curFileOffset = 0;
+						curFileIndex++;
+					}
+				}
+				if ( curFileIndex >= self->files.size() ) {
+					allLoadReqsSent = true;
+					break;
+				}
+
+				//sampleIndex++;
+
+				// Notify loader to sample the file
+				LoadingParam param;
+				param.url = request.url;
+				param.version = self->files[curFileIndex].version;
+				param.filename = self->files[curFileIndex].fileName;
+				param.offset = curFileOffset * self->files[curFileIndex].blockSize; // The file offset in bytes
+				//param.length = std::min(self->files[curFileIndex].fileSize - self->files[curFileIndex].cursor, loadSizeB);
+				param.length = std::min(self->files[curFileIndex].blockSize, std::max((int64_t)0, self->files[curFileIndex].fileSize - param.offset));
+				loadSizeB += param.length;
+				sampleIndex = std::ceil(loadSizeB / sampleB);
+				curFileOffset++;
+
+				//loadSizeB = param.length;
+				param.blockSize = self->files[curFileIndex].blockSize;
+				param.restoreRange = restoreRange;
+				param.addPrefix = addPrefix;
+				param.removePrefix = removePrefix;
+				param.mutationLogPrefix = mutationLogPrefix;
+				if ( !(param.length > 0  &&  param.offset >= 0 && param.offset < self->files[curFileIndex].fileSize) ) {
+					printf("[ERROR] param: length:%ld offset:%ld fileSize:%ld for %ldth file:%s\n",
+							param.length, param.offset, self->files[curFileIndex].fileSize, curFileIndex,
+							self->files[curFileIndex].toString().c_str());
+				}
+
+
+				printf("[Sampling][File:%ld] filename:%s offset:%ld blockSize:%ld filesize:%ld loadSize:%ldB sampleIndex:%ld\n",
+						curFileIndex, self->files[curFileIndex].fileName.c_str(), curFileOffset,
+						self->files[curFileIndex].blockSize, self->files[curFileIndex].fileSize,
+						loadSizeB, sampleIndex);
+
+
+				ASSERT( param.length > 0 );
+				ASSERT( param.offset >= 0 );
+				ASSERT( param.offset <= self->files[curFileIndex].fileSize );
+
+				printf("[Sampling][CMD] Node:%s Loading %s on node %s\n", 
+						self->describeNode().c_str(), param.toString().c_str(), loaderID.toString().c_str());
+
+				self->cmdID.nextCmd(); // The cmd index is the i^th file (range or log file) to be processed
+				if (!self->files[curFileIndex].isRange) {
+					cmdType = RestoreCommandEnum::Sample_Log_File;
+					self->cmdID.setPhase(RestoreCommandEnum::Sample_Log_File);
+					cmdReplies.push_back( loaderInterf.sampleLogFile.getReply(RestoreLoadFileRequest(self->cmdID, param)) );
+				} else {
+					cmdType = RestoreCommandEnum::Sample_Range_File;
+					self->cmdID.setPhase(RestoreCommandEnum::Sample_Range_File);
+					cmdReplies.push_back( loaderInterf.sampleRangeFile.getReply(RestoreLoadFileRequest(self->cmdID, param)) );
+				}
+				
+				printf("[Sampling] Master cmdType:%d cmdUID:%s isRange:%d destinationNode:%s\n", 
+						(int) cmdType, self->cmdID.toString().c_str(), (int) self->files[curFileIndex].isRange,
+						loaderID.toString().c_str());
+				
+				if (param.offset + param.length >= self->files[curFileIndex].fileSize) { // Reach the end of the file
+					curFileIndex++;
+					curFileOffset = 0;
+				}
+				if ( curFileIndex >= self->files.size() ) {
+					allLoadReqsSent = true;
+					break;
+				}
+				++loadingCmdIndex;
+			}
+
+			printf("[Sampling] Wait for %ld loaders to accept the cmd Sample_Range_File or Sample_Log_File\n", cmdReplies.size());
+
+			if ( !cmdReplies.empty() ) {
+				//TODO: change to getAny. NOTE: need to keep the still-waiting replies
+				std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) ); 
+				//std::vector<RestoreCommonReply> reps = wait( getAll(cmdReplies) ); 
+
+				for (int i = 0; i < reps.size(); ++i) {
+					printf("[Sampling][%d out of %d] Get reply:%s for  Sample_Range_File or Sample_Log_File\n",
+							i, reps.size(), reps[i].toString().c_str());
+				}
+				checkpointCMDUID = self->cmdID;
+				checkpointCurFileIndex = curFileIndex;
+				checkpointCurFileOffset = curFileOffset;
+			}
+
+			if (allLoadReqsSent) {
+				printf("[Sampling] allLoadReqsSent, sampling finished\n");
+				break; // NOTE: need to change when change to wait on any cmdReplies
+			}
+
+		} catch (Error &e) {
+			// Handle the command reply timeout error
+			fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", self->describeNode().c_str(),
+					self->cmdID.toString().c_str(), e.code(), e.what());
+			self->cmdID = checkpointCMDUID;
+			curFileIndex = checkpointCurFileIndex;
+			curFileOffset = checkpointCurFileOffset;
+			allLoadReqsSent = false;
+			printf("[Sampling][Waring] Retry at CMDID:%s curFileIndex:%ld\n", self->cmdID.toString().c_str(), curFileIndex);
+		}
+	}
+
+	wait(delay(1.0));
+
+	// Ask master applier to calculate the key ranges for appliers
+	state int numKeyRanges = 0;
+	loop {
+		try {
+			printf("[Sampling][CMD] Ask master applier %s for the key ranges for appliers\n", self->masterApplierInterf.toString().c_str());
+
+			ASSERT(self->appliersInterf.size() > 0);
+			self->cmdID.initPhase(RestoreCommandEnum::Calculate_Applier_KeyRange);
+			self->cmdID.nextCmd();
+			GetKeyRangeNumberReply rep = wait( timeoutError( 
+				self->masterApplierInterf.calculateApplierKeyRange.getReply(RestoreCalculateApplierKeyRangeRequest(self->cmdID, self->appliersInterf.size())),  FastRestore_Failure_Timeout) );
+			printf("[Sampling][CMDRep] number of key ranges calculated by master applier:%d\n", rep.keyRangeNum);
+			numKeyRanges = rep.keyRangeNum;
+
+			if (numKeyRanges <= 0 || numKeyRanges >= self->appliersInterf.size() ) {
+				printf("[WARNING] Calculate_Applier_KeyRange receives wrong reply (numKeyRanges:%ld) from other phases. appliersInterf.size:%d Retry Calculate_Applier_KeyRange\n", numKeyRanges, self->appliersInterf.size());
+				continue;
+			}
+
+			if ( numKeyRanges < self->appliersInterf.size() ) {
+				printf("[WARNING][Sampling] numKeyRanges:%d < appliers number:%ld. %ld appliers will not be used!\n",
+						numKeyRanges, self->appliersInterf.size(), self->appliersInterf.size() - numKeyRanges);
+			}
+
+			break;
+		} catch (Error &e) {
+			// Handle the command reply timeout error
+			fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", self->describeNode().c_str(),
+					self->cmdID.toString().c_str(), e.code(), e.what());
+			printf("[Sampling] [Warning] Retry on Calculate_Applier_KeyRange\n");
+		}
+	}
+
+	wait(delay(1.0));
+
+	// Ask master applier to return the key range for appliers
+	state std::vector<Future<GetKeyRangeReply>> keyRangeReplies;
+	state std::map<UID, RestoreApplierInterface>::iterator applier;
+	loop {
+		try {
+			self->range2Applier.clear();
+			keyRangeReplies.clear(); // In case error happens in try loop
+			self->cmdID.initPhase(RestoreCommandEnum::Get_Applier_KeyRange);
+			//self->cmdID.nextCmd();
+			state int applierindex = 0;
+			for ( applier = self->appliersInterf.begin(); applier != self->appliersInterf.end(); applier++, applierindex++) {
+				self->cmdID.nextCmd();
+				printf("[Sampling][Master] Node:%s, CMDID:%s Ask masterApplierInterf:%s for the lower boundary of the key range for applier:%s\n",
+						self->describeNode().c_str(), self->cmdID.toString().c_str(),
+						self->masterApplierInterf.toString().c_str(), applier->first.toString().c_str());
+				keyRangeReplies.push_back( self->masterApplierInterf.getApplierKeyRangeRequest.getReply(
+					RestoreGetApplierKeyRangeRequest(self->cmdID, applierindex)) );
+			}
+			std::vector<GetKeyRangeReply> reps = wait( timeoutError( getAll(keyRangeReplies), FastRestore_Failure_Timeout) );
+
+			ASSERT( reps.size() <= self->appliersInterf.size() );
+
+			// TODO: Directly use the replied lowerBound and upperBound
+			applier = self->appliersInterf.begin();
+			for (int i = 0; i < reps.size() && i < numKeyRanges; ++i) {
+				UID applierID = applier->first;
+				Standalone<KeyRef> lowerBound = reps[i].lowerBound;
+				// if (i < numKeyRanges) {
+				// 	lowerBound = reps[i].lowerBound;
+				// } else {
+				// 	lowerBound = normalKeys.end;
+				// }
+
+				if (i == 0) {
+					lowerBound = LiteralStringRef("\x00"); // The first interval must starts with the smallest possible key
+				}
+				printf("[INFO] Node:%s Assign key-to-applier map: Key:%s -> applierID:%s\n", self->describeNode().c_str(),
+						getHexString(lowerBound).c_str(), applierID.toString().c_str());
+				self->range2Applier.insert(std::make_pair(lowerBound, applierID));
+				applier++;
+			}
+
+			break;
+		} catch (Error &e) {
+			// TODO: Handle the command reply timeout error
+			fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", self->describeNode().c_str(),
+					self->cmdID.toString().c_str(), e.code(), e.what());
+			printf("[Sampling] [Warning] Retry on Get_Applier_KeyRange\n");
+		}
+	}
+	printf("[Sampling] self->range2Applier has been set. Its size is:%d\n", self->range2Applier.size());
+	self->printAppliersKeyRange();
+
+	wait(delay(1.0));
+
+	return Void();
+
+}
+
+// Restore Master: Ask each restore loader to collect all appliers' interfaces
+ACTOR Future<Void> askLoadersToCollectRestoreAppliersInterfaces(Reference<RestoreMasterData> self) {
+	state int index = 0;
+	loop {
+		try {
+			wait(delay(1.0));
+			index = 0;
+			std::vector<Future<RestoreCommonReply>> cmdReplies;
+			for(auto& loaderInterf : self->loadersInterf) {
+				self->cmdID.nextCmd();
+				printf("[CMD:%s] Node:%s askLoadersToCollectRestoreAppliersInterfaces for node (index=%d uid=%s)\n", 
+						self->cmdID.toString().c_str(), self->describeNode().c_str(),
+						index, loaderInterf.first.toString().c_str());
+				cmdReplies.push_back( loaderInterf.second.collectRestoreRoleInterfaces.getReply(RestoreSimpleRequest(self->cmdID)) );
+				index++;
+			}
+			std::vector<RestoreCommonReply> reps = wait( timeoutError(getAll(cmdReplies), FastRestore_Failure_Timeout) );
+			printf("[setWorkerInterface] Finished\n");
+			break;
+		} catch (Error &e) {
+			fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", self->describeNode().c_str(),
+					self->cmdID.toString().c_str(), e.code(), e.what());
+			printf("Node:%s waits on replies time out. Current phase: setWorkerInterface, Retry all commands.\n", self->describeNode().c_str());
+		}
+	}
+
+	return Void();
+}
+
+
+
+// TODO: Revise the way to collect the restore request. We may make it into 1 transaction
+ACTOR Future<Standalone<VectorRef<RestoreRequest>>> collectRestoreRequests(Database cx) {
+	state int restoreId = 0;
+	state int checkNum = 0;
+	state Standalone<VectorRef<RestoreRequest>> restoreRequests;
+	state Future<Void> watch4RestoreRequest;
+
+	//wait for the restoreRequestTriggerKey to be set by the client/test workload
+	state ReadYourWritesTransaction tr(cx);
+
+	loop {
+		try {
+			tr.reset();
+			tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
+			tr.setOption(FDBTransactionOptions::LOCK_AWARE);
+			// Assumption: restoreRequestTriggerKey has not been set
+			// Question: What if  restoreRequestTriggerKey has been set? we will stuck here?
+			// Question: Can the following code handle the situation?
+			// Note: restoreRequestTriggerKey may be set before the watch is set or may have a conflict when the client sets the same key
+			// when it happens, will we  stuck at wait on the watch?
+
+			watch4RestoreRequest = tr.watch(restoreRequestTriggerKey);
+			wait(tr.commit());
+			printf("[INFO][Master] Finish setting up watch for restoreRequestTriggerKey\n");
+			break;
+		} catch(Error &e) {
+			printf("[WARNING] Transaction for restore request in watch restoreRequestTriggerKey. Error:%s\n", e.name());
+			wait(tr.onError(e));
+		}
+	};
+
+
+	loop {
+		try {
+			tr.reset();
+			tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
+			tr.setOption(FDBTransactionOptions::LOCK_AWARE);
+			// Assumption: restoreRequestTriggerKey has not been set
+			// Before we wait on the watch, we must make sure the key is not there yet!
+			//printf("[INFO][Master] Make sure restoreRequestTriggerKey does not exist before we wait on the key\n");
+			Optional<Value> triggerKey = wait( tr.get(restoreRequestTriggerKey) );
+			if ( triggerKey.present() ) {
+				printf("!!! restoreRequestTriggerKey (and restore requests) is set before restore agent waits on the request. Restore agent can immediately proceed\n");
+				break;
+			}
+			wait(watch4RestoreRequest);
+			printf("[INFO][Master] restoreRequestTriggerKey watch is triggered\n");
+			break;
+		} catch(Error &e) {
+			printf("[WARNING] Transaction for restore request at wait on watch restoreRequestTriggerKey. Error:%s\n", e.name());
+			wait(tr.onError(e));
+		}
+	};
+
+	loop {
+		try {
+			tr.reset();
+			tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
+			tr.setOption(FDBTransactionOptions::LOCK_AWARE);
+
+			state Optional<Value> numRequests = wait(tr.get(restoreRequestTriggerKey));
+			int num = decodeRestoreRequestTriggerValue(numRequests.get());
+			//TraceEvent("RestoreRequestKey").detail("NumRequests", num);
+			printf("[INFO] RestoreRequestNum:%d\n", num);
+
+			state Standalone<RangeResultRef> restoreRequestValues = wait(tr.getRange(restoreRequestKeys, CLIENT_KNOBS->TOO_MANY));
+			printf("Restore worker get restoreRequest: %s\n", restoreRequestValues.toString().c_str());
+
+			ASSERT(!restoreRequestValues.more);
+
+			if(restoreRequestValues.size()) {
+				for ( auto &it : restoreRequestValues ) {
+					printf("Now decode restore request value...\n");
+					restoreRequests.push_back(restoreRequests.arena(), decodeRestoreRequestValue(it.value));
+				}
+			}
+			break;
+		} catch(Error &e) {
+			printf("[WARNING] Transaction error: collect restore requests. Error:%s\n", e.name());
+			wait(tr.onError(e));
+		}
+	};
+
+	return restoreRequests;
+}
+
+// NOTE: This function can now get the backup file descriptors
+ACTOR static Future<Void> _collectBackupFiles(Reference<RestoreMasterData> self, Database cx, RestoreRequest request) {
+	state Key tagName = request.tagName;
+	state Key url = request.url;
+	state bool waitForComplete = request.waitForComplete;
+	state Version targetVersion = request.targetVersion;
+	state bool verbose = request.verbose;
+	state KeyRange range = request.range;
+	state Key addPrefix = request.addPrefix;
+	state Key removePrefix = request.removePrefix;
+	state bool lockDB = request.lockDB;
+	state UID randomUid = request.randomUid;
+
+	ASSERT( lockDB == true );
+
+	self->initBackupContainer(url);
+
+	state Reference<IBackupContainer> bc = self->bc;
+	state BackupDescription desc = wait(bc->describeBackup());
+
+	wait(desc.resolveVersionTimes(cx));
+
+	printf("[INFO] Backup Description\n%s", desc.toString().c_str());
+	printf("[INFO] Restore for url:%s, lockDB:%d\n", url.toString().c_str(), lockDB);
+	if(targetVersion == invalidVersion && desc.maxRestorableVersion.present())
+		targetVersion = desc.maxRestorableVersion.get();
+
+	printf("[INFO] collectBackupFiles: now getting backup files for restore request: %s\n", request.toString().c_str());
+	Optional<RestorableFileSet> restorable = wait(bc->getRestoreSet(targetVersion));
+
+	if(!restorable.present()) {
+		printf("[WARNING] restoreVersion:%ld (%lx) is not restorable!\n", targetVersion, targetVersion);
+		throw restore_missing_data();
+	}
+
+	if (!self->files.empty()) {
+		printf("[WARNING] global files are not empty! files.size() is %ld. We forcely clear files\n", self->files.size());
+		self->files.clear();
+	}
+
+	printf("[INFO] Found backup files: num of files:%ld\n", self->files.size());
+ 	for(const RangeFile &f : restorable.get().ranges) {
+ 		TraceEvent("FoundRangeFileMX").detail("FileInfo", f.toString());
+ 		printf("[INFO] FoundRangeFile, fileInfo:%s\n", f.toString().c_str());
+		RestoreFileFR file(f.version, f.fileName, true, f.blockSize, f.fileSize, f.version, f.version);
+ 		self->files.push_back(file);
+ 	}
+ 	for(const LogFile &f : restorable.get().logs) {
+ 		TraceEvent("FoundLogFileMX").detail("FileInfo", f.toString());
+		printf("[INFO] FoundLogFile, fileInfo:%s\n", f.toString().c_str());
+		RestoreFileFR file(f.beginVersion, f.fileName, false, f.blockSize, f.fileSize, f.endVersion, f.beginVersion);
+		self->files.push_back(file);
+ 	}
+
+	printf("[INFO] Restoring backup to version: %lld\n", (long long) targetVersion);
+
+	return Void();
+}
+
+
+ACTOR static Future<Void> _lockDB(Database cx, UID uid, bool lockDB) {
+	printf("[Lock] DB will be locked, uid:%s, lockDB:%d\n", uid.toString().c_str(), lockDB);
+	
+	ASSERT( lockDB );
+
+	loop {
+		try {
+			wait(lockDatabase(cx, uid));
+			break;
+		} catch( Error &e ) {
+			printf("Transaction Error when we lockDB. Error:%s\n", e.what());
+			wait(tr->onError(e));
+		}
+	}
+
+	state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
+	loop {
+		try {
+			tr->reset();
+			tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
+			tr->setOption(FDBTransactionOptions::LOCK_AWARE);
+
+			wait(checkDatabaseLock(tr, uid));
+
+			tr->commit();
+			break;
+		} catch( Error &e ) {
+			printf("Transaction Error when we lockDB. Error:%s\n", e.what());
+			wait(tr->onError(e));
+		}
+	}
+
+
+	return Void();
+}
+
+ACTOR static Future<Void> _clearDB(Reference<ReadYourWritesTransaction> tr) {
+	loop {
+		try {
+			tr->reset();
+			tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
+			tr->setOption(FDBTransactionOptions::LOCK_AWARE);
+			tr->clear(normalKeys);
+			tr->commit();
+			break;
+		} catch(Error &e) {
+			printf("Retry at clean up DB before restore. error code:%d message:%s. Retry...\n", e.code(), e.what());
+			if(e.code() != error_code_restore_duplicate_tag) {
+				wait(tr->onError(e));
+			}
+		}
+	}
+
+	return Void();
+}
+
+
+
+ACTOR Future<Void> initializeVersionBatch(Reference<RestoreMasterData> self) {
+	loop {
+		try {
+			wait(delay(1.0));
+			std::vector<Future<RestoreCommonReply>> cmdReplies;
+			self->cmdID.initPhase(RestoreCommandEnum::Reset_VersionBatch);
+			for (auto &loader : self->loadersInterf) {
+				cmdReplies.push_back( loader.second.initVersionBatch.getReply(RestoreVersionBatchRequest(self->cmdID, self->batchIndex)) );
+			}
+			for (auto &applier : self->appliersInterf) {
+				cmdReplies.push_back( applier.second.initVersionBatch.getReply(RestoreVersionBatchRequest(self->cmdID, self->batchIndex)) );
+			}
+
+			std::vector<RestoreCommonReply> reps = wait( timeoutError(getAll(cmdReplies), FastRestore_Failure_Timeout) );
+			printf("Initilaize Version Batch done\n");
+			break;
+		} catch (Error &e) {
+			fprintf(stdout, "[ERROR] Node:%s, Current phase: initializeVersionBatch, Commands before cmdID:%s error. error code:%d, error message:%s\n", self->describeNode().c_str(),
+					self->cmdID.toString().c_str(), e.code(), e.what());
+		}
+	}
+
+	return Void();
+}
+
+
+ACTOR Future<Void> notifyApplierToApplyMutations(Reference<RestoreMasterData> self) {
+	state std::vector<Future<RestoreCommonReply>> cmdReplies;
+	loop {
+		try {
+			self->cmdID.initPhase( RestoreCommandEnum::Apply_Mutation_To_DB );
+			for (auto& applier : self->appliersInterf) {
+				RestoreApplierInterface &applierInterf = applier.second;
+	
+				printf("[CMD] Node:%s Notify node:%s to apply mutations to DB\n", self->describeNode().c_str(), applier.first.toString().c_str());
+				cmdReplies.push_back( applier.second.applyToDB.getReply(RestoreSimpleRequest(self->cmdID)) );
+			}
+			printf("[INFO] Wait for %ld appliers to apply mutations to DB\n", self->appliersInterf.size());
+			std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) );
+			//std::vector<RestoreCommonReply> reps = wait( getAll(cmdReplies) );
+			printf("[INFO] %ld appliers finished applying mutations to DB\n", self->appliersInterf.size());
+
+			cmdReplies.clear();
+
+			wait(delay(5.0)); //TODO: Delete this wait and see if it can pass correctness
+
+			break;
+		} catch (Error &e) {
+			fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", self->describeNode().c_str(),
+					self->cmdID.toString().c_str(), e.code(), e.what());
+		}
+	}
+
+	return Void();
+}
+
+
+
+ACTOR Future<Void> assignKeyRangeToAppliers(Reference<RestoreMasterData> self, Database cx)  { //, VectorRef<RestoreWorkerInterface> ret_agents
+	//construct the key range for each applier
+	std::vector<KeyRef> lowerBounds;
+	std::vector<Standalone<KeyRangeRef>> keyRanges;
+	std::vector<UID> applierIDs;
+
+	// printf("[INFO] Node:%s, Assign key range to appliers. num_appliers:%ld\n", self->describeNode().c_str(), self->range2Applier.size());
+	for (auto& applier : self->range2Applier) {
+		lowerBounds.push_back(applier.first);
+		applierIDs.push_back(applier.second);
+		// printf("\t[INFO] ApplierID:%s lowerBound:%s\n",
+		// 		applierIDs.back().toString().c_str(),
+		// 		lowerBounds.back().toString().c_str());
+	}
+	for (int i  = 0; i < lowerBounds.size(); ++i) {
+		KeyRef startKey = lowerBounds[i];
+		KeyRef endKey;
+		if ( i < lowerBounds.size() - 1) {
+			endKey = lowerBounds[i+1];
+		} else {
+			endKey = normalKeys.end;
+		}
+
+		if (startKey > endKey) {
+			fprintf(stderr, "ERROR at assignKeyRangeToAppliers, startKey:%s > endKey:%s\n", startKey.toString().c_str(), endKey.toString().c_str());
+		}
+
+		keyRanges.push_back(KeyRangeRef(startKey, endKey));
+	}
+
+	ASSERT( applierIDs.size() == keyRanges.size() );
+	state std::map<UID, Standalone<KeyRangeRef>> appliers;
+	appliers.clear(); // If this function is called more than once in multiple version batches, appliers may carry over the data from earlier version batch
+	for (int i = 0; i < applierIDs.size(); ++i) {
+		if (appliers.find(applierIDs[i]) != appliers.end()) {
+			printf("[ERROR] ApplierID appear more than once. appliers size:%ld applierID: %s\n",
+					appliers.size(), applierIDs[i].toString().c_str());
+			printApplierKeyRangeInfo(appliers);
+		}
+		ASSERT( appliers.find(applierIDs[i]) == appliers.end() ); // we should not have a duplicate applierID respoinsbile for multiple key ranges
+		appliers.insert(std::make_pair(applierIDs[i], keyRanges[i]));
+	}
+
+	state std::vector<Future<RestoreCommonReply>> cmdReplies;
+	loop {
+		try {
+			cmdReplies.clear();
+			self->cmdID.initPhase(RestoreCommandEnum::Assign_Applier_KeyRange);
+			for (auto& applier : appliers) {
+				KeyRangeRef keyRange = applier.second;
+				UID applierID = applier.first;
+				printf("[CMD] Node:%s, Assign KeyRange:%s [begin:%s end:%s] to applier ID:%s\n", self->describeNode().c_str(),
+						keyRange.toString().c_str(),
+						getHexString(keyRange.begin).c_str(), getHexString(keyRange.end).c_str(),
+						applierID.toString().c_str());
+
+				ASSERT( self->appliersInterf.find(applierID) != self->appliersInterf.end() );
+				RestoreApplierInterface applierInterf = self->appliersInterf[applierID];
+				self->cmdID.nextCmd();
+				cmdReplies.push_back( applierInterf.setApplierKeyRangeRequest.getReply(RestoreSetApplierKeyRangeRequest(self->cmdID, applier.first, keyRange)) );
+
+			}
+			printf("[INFO] Wait for %ld applier to accept the cmd Assign_Applier_KeyRange\n", appliers.size());
+			std::vector<RestoreCommonReply> reps = wait( timeoutError(getAll(cmdReplies), FastRestore_Failure_Timeout) );
+			printf("All appliers have been assigned for ranges");
+			
+			break;
+		} catch (Error &e) {
+			fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s error. error code:%d, error message:%s\n", self->describeNode().c_str(),
+					self->cmdID.toString().c_str(), e.code(), e.what());
+		}
+	}
+
+	return Void();
+}
+
+// Restore Master: Notify loader about appliers' responsible key range
+ACTOR Future<Void> notifyAppliersKeyRangeToLoader(Reference<RestoreMasterData> self, Database cx)  {
+	state std::vector<UID> loaders = self->getLoaderIDs();
+	state std::vector<Future<RestoreCommonReply>> cmdReplies;
+	state Standalone<VectorRef<UID>> appliers;
+	state Standalone<VectorRef<KeyRange>> ranges;
+
+	state std::map<Standalone<KeyRef>, UID>::iterator applierRange;
+	for (applierRange = self->range2Applier.begin(); applierRange != self->range2Applier.end(); applierRange++) {
+		KeyRef beginRange = applierRange->first;
+		KeyRange range(KeyRangeRef(beginRange, beginRange)); // TODO: Use the end of key range
+		appliers.push_back(appliers.arena(), applierRange->second);
+		ranges.push_back(ranges.arena(), range);
+	}
+
+	printf("Notify_Loader_ApplierKeyRange: number of appliers:%d\n", appliers.size());
+	ASSERT( appliers.size() == ranges.size() && appliers.size() != 0 );
+
+	self->cmdID.initPhase( RestoreCommandEnum::Notify_Loader_ApplierKeyRange );
+	state std::map<UID, RestoreLoaderInterface>::iterator loader;
+	for (loader = self->loadersInterf.begin(); loader != self->loadersInterf.begin(); loader++) {
+		self->cmdID.nextCmd();
+		loop {
+			try {
+				cmdReplies.clear();
+				printf("[CMD] Node:%s Notify node:%s about appliers key range\n", self->describeNode().c_str(), loader->first.toString().c_str());
+				cmdReplies.push_back( loader->second.setApplierKeyRangeVectorRequest.getReply(RestoreSetApplierKeyRangeVectorRequest(self->cmdID, appliers, ranges)) );
+				printf("[INFO] Wait for node:%s to accept the cmd Notify_Loader_ApplierKeyRange\n", loader->first.toString().c_str());
+				std::vector<RestoreCommonReply> reps = wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout ) );
+				printf("Finished Notify_Loader_ApplierKeyRange: number of appliers:%d\n", appliers.size());
+				cmdReplies.clear();
+				break;
+			} catch (Error &e) {
+				fprintf(stdout, "[ERROR] Node:%s, Commands before cmdID:%s timeout\n", self->describeNode().c_str(), self->cmdID.toString().c_str());
+			}
+		}
+	}
+
+	return Void();
+}
+
+
+ACTOR static Future<Void> finishRestore(Reference<RestoreMasterData> self, Database cx, Standalone<VectorRef<RestoreRequest>> restoreRequests) {
+	// Make restore workers quit
+	state std::vector<Future<RestoreCommonReply>> cmdReplies;
+	state std::map<UID, RestoreLoaderInterface>::iterator loader;
+	state std::map<UID, RestoreApplierInterface>::iterator applier;
+	loop {
+		try {
+			cmdReplies.clear();
+			self->cmdID.initPhase(RestoreCommandEnum::Finish_Restore);
+
+			for ( loader = self->loadersInterf.begin(); loader != self->loadersInterf.end(); loader++ ) {
+				self->cmdID.nextCmd();
+				cmdReplies.push_back(loader->second.finishRestore.getReply(RestoreSimpleRequest(self->cmdID)));
+			}
+			for ( applier = self->appliersInterf.begin(); applier != self->appliersInterf.end(); applier++ ) {
+				self->cmdID.nextCmd();
+				cmdReplies.push_back(applier->second.finishRestore.getReply(RestoreSimpleRequest(self->cmdID)));
+			}
+
+			if (!cmdReplies.empty()) {
+				std::vector<RestoreCommonReply> reps =  wait( timeoutError( getAll(cmdReplies), FastRestore_Failure_Timeout / 100 ) );
+				//std::vector<RestoreCommonReply> reps =  wait( getAll(cmdReplies) );
+				cmdReplies.clear();
+			}
+			printf("All restore workers have quited\n");
+
+			break;
+		} catch(Error &e) {
+			printf("[ERROR] At sending finishRestore request. error code:%d message:%s. Retry...\n", e.code(), e.what());
+			self->loadersInterf.clear();
+			self->appliersInterf.clear();
+			cmdReplies.clear();
+			wait( _collectRestoreRoleInterfaces(self, cx) );
+		}
+	}
+
+	// Notify tester that the restore has finished
+	state ReadYourWritesTransaction tr3(cx);
+	loop {
+		try {
+			tr3.reset();
+			tr3.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
+			tr3.setOption(FDBTransactionOptions::LOCK_AWARE);
+			tr3.clear(restoreRequestTriggerKey);
+			tr3.clear(restoreRequestKeys);
+			tr3.set(restoreRequestDoneKey, restoreRequestDoneValue(restoreRequests.size()));
+			wait(tr3.commit());
+			TraceEvent("LeaderFinishRestoreRequest");
+			printf("[INFO] RestoreLeader write restoreRequestDoneKey\n");
+
+			break;
+		}  catch( Error &e ) {
+			TraceEvent("RestoreAgentLeaderErrorTr3").detail("ErrorCode", e.code()).detail("ErrorName", e.name());
+			printf("[Error] RestoreLead operation on restoreRequestDoneKey, error:%s\n", e.what());
+			wait( tr3.onError(e) );
+		}
+	};
+
+
+ 	// TODO:  Validate that the range version map has exactly the restored ranges in it.  This means that for any restore operation
+ 	// the ranges to restore must be within the backed up ranges, otherwise from the restore perspective it will appear that some
+ 	// key ranges were missing and so the backup set is incomplete and the restore has failed.
+ 	// This validation cannot be done currently because Restore only supports a single restore range but backups can have many ranges.
+
+ 	// Clear the applyMutations stuff, including any unapplied mutations from versions beyond the restored version.
+ 	//	restore.clearApplyMutationsKeys(tr);
+
+	printf("[INFO] Notify the end of the restore\n");
+	TraceEvent("NotifyRestoreFinished");
+
+	return Void();
+}
+
+
+
+ACTOR static Future<Void> unlockDB(Database cx, UID uid) {
+	state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
+	loop {
+		try {
+			tr->reset();
+			tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
+			tr->setOption(FDBTransactionOptions::LOCK_AWARE);
+			printf("CheckDBlock:%s START\n", uid.toString().c_str());
+			wait(checkDatabaseLock(tr, uid));
+			printf("CheckDBlock:%s DONE\n", uid.toString().c_str());
+
+			printf("UnlockDB now. Start.\n");
+			wait(unlockDatabase(tr, uid)); //NOTE: unlockDatabase didn't commit inside the function!
+
+			printf("CheckDBlock:%s START\n", uid.toString().c_str());
+			wait(checkDatabaseLock(tr, uid));
+			printf("CheckDBlock:%s DONE\n", uid.toString().c_str());
+
+			printf("UnlockDB now. Commit.\n");
+			wait( tr->commit() );
+
+			printf("UnlockDB now. Done.\n");
+			break;
+		} catch( Error &e ) {
+			printf("Error when we unlockDB. Error:%s\n", e.what());
+			wait(tr->onError(e));
+		}
+	};
+
+ 	return Void();
+ }
+
+ACTOR static Future<Void> registerStatus(Database cx, struct FastRestoreStatus status) {
+ 	state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
+	loop {
+		try {
+			printf("[Restore_Status][%d] curWorkload:%.2f curRunningtime:%.2f curSpeed:%.2f totalWorkload:%.2f totalRunningTime:%.2f totalSpeed:%.2f\n",
+					restoreStatusIndex, status.curWorkloadSize, status.curRunningTime, status.curSpeed, status.totalWorkloadSize, status.totalRunningTime, status.totalSpeed);
+
+			tr->reset();
+			tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
+			tr->setOption(FDBTransactionOptions::LOCK_AWARE);
+
+			tr->set(restoreStatusKeyFor(StringRef(std::string("curWorkload") + std::to_string(restoreStatusIndex))), restoreStatusValue(status.curWorkloadSize));
+			tr->set(restoreStatusKeyFor(StringRef(std::string("curRunningTime") + std::to_string(restoreStatusIndex))), restoreStatusValue(status.curRunningTime));
+			tr->set(restoreStatusKeyFor(StringRef(std::string("curSpeed") + std::to_string(restoreStatusIndex))), restoreStatusValue(status.curSpeed));
+
+			tr->set(restoreStatusKeyFor(StringRef(std::string("totalWorkload"))), restoreStatusValue(status.totalWorkloadSize));
+			tr->set(restoreStatusKeyFor(StringRef(std::string("totalRunningTime"))), restoreStatusValue(status.totalRunningTime));
+			tr->set(restoreStatusKeyFor(StringRef(std::string("totalSpeed"))), restoreStatusValue(status.totalSpeed));
+
+			wait( tr->commit() );
+			restoreStatusIndex++;
+
+			break;
+		} catch( Error &e ) {
+			printf("Transaction Error when we registerStatus. Error:%s\n", e.what());
+			wait(tr->onError(e));
+		}
+	 };
+
+	return Void();
+}
\ No newline at end of file
diff --git a/fdbserver/RestoreMaster.actor.h b/fdbserver/RestoreMaster.actor.h
new file mode 100644
index 0000000000..b6d29dfb7a
--- /dev/null
+++ b/fdbserver/RestoreMaster.actor.h
@@ -0,0 +1,264 @@
+/*
+ * RestoreMasterInterface.h
+ *
+ * This source file is part of the FoundationDB open source project
+ *
+ * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
+ *
+ * Licensed 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.
+ */
+
+// Declear RestoreMaster interface and actors
+
+#pragma once
+#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_RestoreMasterInterface_G_H)
+	#define FDBSERVER_RestoreMasterInterface_G_H
+	#include "fdbserver/RestoreMaster.actor.g.h"
+#elif !defined(FDBSERVER_RestoreMasterInterface_H)
+	#define FDBSERVER_RestoreMasterInterface_H
+
+#include <sstream>
+#include "flow/Stats.h"
+#include "fdbclient/FDBTypes.h"
+#include "fdbclient/CommitTransaction.h"
+#include "fdbrpc/fdbrpc.h"
+#include "fdbserver/CoordinationInterface.h"
+#include "fdbrpc/Locality.h"
+
+#include "fdbserver/RestoreUtil.h"
+#include "fdbserver/RestoreRoleCommon.actor.h"
+
+#include "flow/actorcompiler.h" // has to be last include
+
+extern double loadBatchSizeThresholdB;
+extern int restoreStatusIndex;
+
+struct RestoreMasterData :  RestoreRoleData, public ReferenceCounted<RestoreMasterData> {
+	// range2Applier is in master and loader node. Loader node uses this to determine which applier a mutation should be sent
+	std::map<Standalone<KeyRef>, UID> range2Applier; // KeyRef is the inclusive lower bound of the key range the applier (UID) is responsible for
+
+	CMDUID cmdID; // Command id to recoself the progress
+
+	// Temporary variables to hold files and data to restore
+	std::vector<RestoreFileFR> allFiles; // All backup files to be processed in all version batches
+	std::vector<RestoreFileFR> files; // Backup files to be parsed and applied: range and log files in 1 version batch
+	std::map<Version, Version> forbiddenVersions; // forbidden version range [first, second)
+
+	// In each version batch, we process the files in [curBackupFilesBeginIndex, curBackupFilesEndIndex] in RestoreMasterData.allFiles.
+	long curBackupFilesBeginIndex;
+	long curBackupFilesEndIndex;
+	double totalWorkloadSize;
+	double curWorkloadSize;
+	int batchIndex;
+
+	Reference<IBackupContainer> bc; // Backup container is used to read backup files
+	Key bcUrl; // The url used to get the bc
+
+	void addref() { return ReferenceCounted<RestoreMasterData>::addref(); }
+	void delref() { return ReferenceCounted<RestoreMasterData>::delref(); }
+
+	void printAllBackupFilesInfo() {
+		printf("[INFO] All backup files: num:%ld\n", allFiles.size());
+		for (int i = 0; i < allFiles.size(); ++i) {
+			printf("\t[INFO][File %d] %s\n", i, allFiles[i].toString().c_str());
+		}
+	}
+
+	std::string describeNode() {
+		std::stringstream ss;
+		ss << "Master versionBatch:"  << batchIndex;
+		return ss.str();
+	}
+
+	void constructFilesWithVersionRange() {
+		printf("[INFO] constructFilesWithVersionRange for num_files:%ld\n", files.size());
+		allFiles.clear();
+		for (int i = 0; i <  files.size(); i++) {
+			printf("\t[File:%d] Start %s\n", i,  files[i].toString().c_str());
+			Version beginVersion = 0;
+			Version endVersion = 0;
+			if ( files[i].isRange) {
+				// No need to parse range filename to get endVersion
+				beginVersion =  files[i].version;
+				endVersion = beginVersion;
+			} else { // Log file
+				//Refer to pathToLogFile() in BackupContainer.actor.cpp
+				long blockSize, len;
+				int pos =  files[i].fileName.find_last_of("/");
+				std::string fileName =  files[i].fileName.substr(pos);
+				printf("\t[File:%d] Log filename:%s, pos:%d\n", i, fileName.c_str(), pos);
+				sscanf(fileName.c_str(), "/log,%ld,%ld,%*[^,],%lu%ln", &beginVersion, &endVersion, &blockSize, &len);
+				printf("\t[File:%d] Log filename:%s produces beginVersion:%ld endVersion:%ld\n",i, fileName.c_str(), beginVersion, endVersion);
+			}
+			files[i].beginVersion = beginVersion;
+			files[i].endVersion = endVersion;
+			printf("\t[File:%d] End %s\n", i,  files[i].toString().c_str());
+			ASSERT(beginVersion <= endVersion);
+			allFiles.push_back( files[i]);
+		}
+	}
+
+	void printBackupFilesInfo() {
+		printf("[INFO] The backup files for current batch to load and apply: num:%ld\n", files.size());
+		for (int i = 0; i < files.size(); ++i) {
+			printf("\t[INFO][File %d] %s\n", i, files[i].toString().c_str());
+		}
+	}
+
+	void buildForbiddenVersionRange() {
+		printf("[INFO] Build forbidden version ranges for all backup files: num:%ld\n", allFiles.size());
+		for (int i = 0; i < allFiles.size(); ++i) {
+			if (!allFiles[i].isRange) {
+				forbiddenVersions.insert(std::make_pair(allFiles[i].beginVersion, allFiles[i].endVersion));
+			}
+		}
+	}
+
+	bool isForbiddenVersionRangeOverlapped() {
+		printf("[INFO] Check if forbidden version ranges is overlapped: num of ranges:%ld\n", forbiddenVersions.size());
+		if (forbiddenVersions.empty()) {
+			return false;
+		}
+
+		std::map<Version, Version>::iterator prevRange = forbiddenVersions.begin();
+		std::map<Version, Version>::iterator curRange = forbiddenVersions.begin();
+		curRange++; // Assume forbiddenVersions has at least one element!
+
+		while ( curRange != forbiddenVersions.end() ) {
+			if ( curRange->first < prevRange->second ) {
+				return true; // overlapped
+			}
+			curRange++;
+		}
+
+		return false; //not overlapped
+	}
+
+
+	void printForbiddenVersionRange() {
+		printf("[INFO] Number of forbidden version ranges:%ld\n", forbiddenVersions.size());
+		int i = 0;
+		for (auto &range : forbiddenVersions) {
+			printf("\t[INFO][Range%d] [%ld, %ld)\n", i, range.first, range.second);
+			++i;
+		}
+	}
+
+	// endVersion is begin version for range file, because range file takes snapshot at the same version
+	// endVersion is the end version (excluded) for mutations recoselfed in log file
+	bool isVersionInForbiddenRange(Version endVersion, bool isRange) {
+		bool isForbidden = false;
+		for (auto &range : forbiddenVersions) {
+			if ( isRange ) { //the range file includes mutations at the endVersion
+				if (endVersion >= range.first && endVersion < range.second) {
+					isForbidden = true;
+					break;
+				}
+			} else { // the log file does NOT include mutations at the endVersion
+				continue; // Log file's endVersion is always a valid version batch boundary as long as the forbidden version ranges do not overlap
+			}
+		}
+
+		return isForbidden;
+	}
+
+
+	void printAppliersKeyRange() {
+		printf("[INFO] The mapping of KeyRange_start --> Applier ID\n");
+		// applier type: std::map<Standalone<KeyRef>, UID>
+		for (auto &applier : range2Applier) {
+			printf("\t[INFO]%s -> %s\n", getHexString(applier.first).c_str(), applier.second.toString().c_str());
+		}
+	}
+
+	bool isBackupEmpty() {
+		for (int i = 0; i < files.size(); ++i) {
+			if (files[i].fileSize > 0) {
+				return false;
+			}
+		}
+		return true;
+	}
+
+
+	void initBackupContainer(Key url) {
+		if ( bcUrl == url && bc.isValid() ) {
+			return;
+		}
+		printf("initBackupContainer, url:%s\n", url.toString().c_str());
+		bcUrl = url;
+		bc = IBackupContainer::openContainer(url.toString());
+		//state BackupDescription desc = wait(self->bc->describeBackup());
+		//return Void();
+	}
+
+	// Collect the set of backup files to be used for a version batch
+	// Return true if there is still files to be restored; false otherwise.
+	// This function will change the process' RestoreMasterData
+	bool collectFilesForOneVersionBatch() {
+		files.clear();
+		curWorkloadSize = 0;
+		Version endVersion = -1;
+		bool isRange = false;
+		bool validVersion = false;
+		// Step: Find backup files in each version batch and restore them.
+		while ( curBackupFilesBeginIndex < allFiles.size() ) {
+			// Find the curBackupFilesEndIndex, such that the to-be-loaded files size (curWorkloadSize) is as close to loadBatchSizeThresholdB as possible,
+			// and curBackupFilesEndIndex must not belong to the forbidden version range!
+			if ( curBackupFilesEndIndex < allFiles.size() ) {
+				endVersion =  allFiles[curBackupFilesEndIndex].endVersion;
+				isRange = allFiles[curBackupFilesEndIndex].isRange;
+				validVersion = !isVersionInForbiddenRange(endVersion, isRange);
+				curWorkloadSize  += allFiles[curBackupFilesEndIndex].fileSize;
+				printf("[DEBUG][Batch:%d] Calculate backup files for a version batch: endVersion:%lld isRange:%d validVersion:%d curWorkloadSize:%.2fB curBackupFilesBeginIndex:%ld curBackupFilesEndIndex:%ld, files.size:%ld\n",
+					batchIndex, (long long) endVersion, isRange, validVersion, curWorkloadSize , curBackupFilesBeginIndex, curBackupFilesEndIndex, allFiles.size());
+			}
+			if ( (validVersion && curWorkloadSize  >= loadBatchSizeThresholdB) || curBackupFilesEndIndex >= allFiles.size() )  {
+				if ( curBackupFilesEndIndex >= allFiles.size() && curWorkloadSize <= 0 ) {
+					printf("Restore finishes: curBackupFilesEndIndex:%ld, allFiles.size:%ld, curWorkloadSize:%.2f\n",
+							curBackupFilesEndIndex, allFiles.size(), curWorkloadSize );
+					//break; // return result
+				}
+				// Construct the files [curBackupFilesBeginIndex, curBackupFilesEndIndex]
+				//resetPerVersionBatch();
+				//cmdID.setBatch(batchIndex);
+				if ( curBackupFilesBeginIndex < allFiles.size()) {
+					for (int fileIndex = curBackupFilesBeginIndex; fileIndex <= curBackupFilesEndIndex && fileIndex < allFiles.size(); fileIndex++) {
+						files.push_back(allFiles[fileIndex]);
+					}
+				}
+				printBackupFilesInfo();
+				totalWorkloadSize += curWorkloadSize;
+				break;
+			} else if (validVersion && curWorkloadSize < loadBatchSizeThresholdB) {
+				curBackupFilesEndIndex++;
+			} else if (!validVersion && curWorkloadSize < loadBatchSizeThresholdB) {
+				curBackupFilesEndIndex++;
+			} else if (!validVersion && curWorkloadSize >= loadBatchSizeThresholdB) {
+				// Now: just move to the next file. We will eventually find a valid version but load more than loadBatchSizeThresholdB
+				printf("[WARNING] The loading batch size will be larger than expected! curBatchSize:%.2fB, expectedBatchSize:%2.fB, endVersion:%ld\n",
+						curWorkloadSize, loadBatchSizeThresholdB, endVersion);
+				curBackupFilesEndIndex++;
+				// TODO: Roll back to find a valid version
+			}
+		}
+
+		return (files.size() > 0);
+	}
+};
+
+
+ACTOR Future<Void> startRestoreMaster(Reference<RestoreMasterData> self, Database cx);
+
+#include "flow/unactorcompiler.h"
+#endif
\ No newline at end of file
diff --git a/fdbserver/RestoreRoleCommon.actor.cpp b/fdbserver/RestoreRoleCommon.actor.cpp
new file mode 100644
index 0000000000..80a8d941db
--- /dev/null
+++ b/fdbserver/RestoreRoleCommon.actor.cpp
@@ -0,0 +1,324 @@
+/*
+ * RestoreRoleCommon.actor.cpp
+ *
+ * This source file is part of the FoundationDB open source project
+ *
+ * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
+ *
+ * Licensed 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.
+ */
+
+#include "fdbclient/NativeAPI.actor.h"
+#include "fdbclient/MutationList.h"
+
+#include "fdbserver/RestoreUtil.h"
+#include "fdbserver/RestoreRoleCommon.actor.h"
+#include "fdbserver/RestoreLoader.actor.h"
+#include "fdbserver/RestoreApplier.actor.h"
+#include "fdbserver/RestoreMaster.actor.h"
+
+#include "flow/actorcompiler.h"  // This must be the last #include.
+
+class Database;
+struct RestoreWorkerData;
+
+// id is the id of the worker to be monitored
+// This actor is used for both restore loader and restore applier
+ACTOR Future<Void> handleHeartbeat(RestoreSimpleRequest req, UID id) {
+	wait( delay(0.1) ); // To avoid warning
+	req.reply.send(RestoreCommonReply(id, req.cmdID));
+
+	return Void();
+}
+
+// Restore Worker: collect restore role interfaces locally by reading the specific system keys
+ACTOR Future<Void> _collectRestoreRoleInterfaces(Reference<RestoreRoleData> self, Database cx) {
+    state Transaction tr(cx);
+	//state Standalone<RangeResultRef> loaderAgentValues;
+	//state Standalone<RangeResultRef> applierAgentValues;
+	printf("[INFO][Worker] Node:%s Get the handleCollectRestoreRoleInterfaceRequest for all workers\n", self->describeNode().c_str());
+	loop {
+		try {
+			self->clearInterfaces();
+			tr.reset();
+			tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
+			tr.setOption(FDBTransactionOptions::LOCK_AWARE);
+			state Standalone<RangeResultRef> loaderAgentValues = wait( tr.getRange(restoreLoaderKeys, CLIENT_KNOBS->TOO_MANY) );
+			state Standalone<RangeResultRef> applierAgentValues = wait( tr.getRange(restoreApplierKeys, CLIENT_KNOBS->TOO_MANY) );
+			ASSERT(!loaderAgentValues.more);
+			ASSERT(!applierAgentValues.more);
+			// Save the loader and applier interfaces for the later operations
+			if (loaderAgentValues.size()) {
+				for(auto& it : loaderAgentValues) {
+					RestoreLoaderInterface loaderInterf = BinaryReader::fromStringRef<RestoreLoaderInterface>(it.value, IncludeVersion());
+					self->loadersInterf[loaderInterf.id()] = loaderInterf;
+				}
+			}
+			if (applierAgentValues.size()) {
+				for(auto& it : applierAgentValues) {
+					RestoreApplierInterface applierInterf = BinaryReader::fromStringRef<RestoreApplierInterface>(it.value, IncludeVersion());
+					self->appliersInterf[applierInterf.id()] = applierInterf;
+					self->masterApplierInterf = applierInterf; // TODO: Set masterApplier in a more deterministic way
+				}
+			}
+			//wait(tr.commit());
+			break;
+		} catch( Error &e ) {
+			printf("[WARNING] Node:%s handleCollectRestoreRoleInterfaceRequest() transaction error:%s\n", self->describeNode().c_str(), e.what());
+			wait( tr.onError(e) );
+		}
+		printf("[WARNING] Node:%s handleCollectRestoreRoleInterfaceRequest should always succeed in the first loop! Something goes wrong!\n", self->describeNode().c_str());
+	};
+
+    return Void();
+} 
+
+// Restore worker
+// RestoreRoleData will be casted to RestoreLoaderData or RestoreApplierData based on its type
+ACTOR Future<Void> handleCollectRestoreRoleInterfaceRequest(RestoreSimpleRequest req, Reference<RestoreRoleData> self, Database cx) {
+
+	while (self->isInProgress(RestoreCommandEnum::Collect_RestoreRoleInterface)) {
+		printf("[DEBUG] NODE:%s handleCollectRestoreRoleInterfaceRequest wait for 5s\n",  self->describeNode().c_str());
+		wait(delay(5.0));
+	}
+	// Handle duplicate, assuming cmdUID is always unique for the same workload
+	if ( self->isCmdProcessed(req.cmdID) ) {
+		printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", self->describeNode().c_str(), req.cmdID.toString().c_str());
+		req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+		return Void();
+	} 
+
+	self->setInProgressFlag(RestoreCommandEnum::Collect_RestoreRoleInterface);
+
+    wait( _collectRestoreRoleInterfaces(self, cx) );
+
+	req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+	self->processedCmd[req.cmdID] = 1;
+	self->clearInProgressFlag(RestoreCommandEnum::Collect_RestoreRoleInterface);
+
+	return Void();
+ }
+
+
+
+ACTOR Future<Void> handleInitVersionBatchRequest(RestoreVersionBatchRequest req, Reference<RestoreRoleData> self) {
+	// wait( delay(1.0) );
+	printf("[Batch:%d] Node:%s Start...\n", req.batchID, self->describeNode().c_str());
+	while (self->isInProgress(RestoreCommandEnum::Reset_VersionBatch)) {
+		printf("[DEBUG] NODE:%s handleVersionBatchRequest wait for 5s\n",  self->describeNode().c_str());
+		wait(delay(5.0));
+	}
+
+	// Handle duplicate, assuming cmdUID is always unique for the same workload
+	if ( self->isCmdProcessed(req.cmdID) ) {
+		printf("[DEBUG] NODE:%s skip duplicate cmd:%s\n", self->describeNode().c_str(), req.cmdID.toString().c_str());
+		req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+		return Void();
+	} 
+
+	self->setInProgressFlag(RestoreCommandEnum::Reset_VersionBatch);
+
+	self->resetPerVersionBatch();
+	req.reply.send(RestoreCommonReply(self->id(), req.cmdID));
+
+	self->processedCmd[req.cmdID] = 1;
+	self->clearInProgressFlag(RestoreCommandEnum::Reset_VersionBatch);
+
+	// This actor never returns. You may cancel it in master
+	return Void();
+}
+
+
+//-------Helper functions
+std::string getHexString(StringRef input) {
+	std::stringstream ss;
+	for (int i = 0; i<input.size(); i++) {
+		if ( i % 4 == 0 )
+			ss << " ";
+		if ( i == 12 ) { //The end of 12bytes, which is the version size for value
+			ss << "|";
+		}
+		if ( i == (12 + 12) ) { //The end of version + header
+			ss << "@";
+		}
+		ss << std::setfill('0') << std::setw(2) << std::hex << (int) input[i]; // [] operator moves the pointer in step of unit8
+	}
+	return ss.str();
+}
+
+std::string getHexKey(StringRef input, int skip) {
+	std::stringstream ss;
+	for (int i = 0; i<skip; i++) {
+		if ( i % 4 == 0 )
+			ss << " ";
+		ss << std::setfill('0') << std::setw(2) << std::hex << (int) input[i]; // [] operator moves the pointer in step of unit8
+	}
+	ss << "||";
+
+	//hashvalue
+	ss << std::setfill('0') << std::setw(2) << std::hex << (int) input[skip]; // [] operator moves the pointer in step of unit8
+	ss << "|";
+
+	// commitversion in 64bit
+	int count = 0;
+	for (int i = skip+1; i<input.size() && i < skip+1+8; i++) {
+		if ( count++ % 4 == 0 )
+			ss << " ";
+		ss << std::setfill('0') << std::setw(2) << std::hex << (int) input[i]; // [] operator moves the pointer in step of unit8
+	}
+	// part value
+	count = 0;
+	for (int i = skip+1+8; i<input.size(); i++) {
+		if ( count++ % 4 == 0 )
+			ss << " ";
+		ss << std::setfill('0') << std::setw(2) << std::hex << (int) input[i]; // [] operator moves the pointer in step of unit8
+	}
+	return ss.str();
+}
+
+
+void printMutationListRefHex(MutationListRef m, std::string prefix) {
+	MutationListRef::Iterator iter = m.begin();
+	for ( ;iter != m.end(); ++iter) {
+		printf("%s mType:%04x param1:%s param2:%s param1_size:%d, param2_size:%d\n", prefix.c_str(), iter->type,
+			   getHexString(iter->param1).c_str(), getHexString(iter->param2).c_str(), iter->param1.size(), iter->param2.size());
+	}
+	return;
+}
+
+//TODO: Print out the backup mutation log value. The backup log value (i.e., the value in the kv pair) has the following format
+//version(12B)|mutationRef|MutationRef|....
+//A mutationRef has the format: |type_4B|param1_size_4B|param2_size_4B|param1|param2.
+//Note: The data is stored in little endian! You need to convert it to BigEndian so that you know how long the param1 and param2 is and how to format them!
+void printBackupMutationRefValueHex(Standalone<StringRef> val_input, std::string prefix) {
+	std::stringstream ss;
+	const int version_size = 12;
+	const int header_size = 12;
+	StringRef val = val_input.contents();
+	StringRefReaderMX reader(val, restore_corrupted_data());
+
+	int count_size = 0;
+	// Get the version
+	uint64_t version = reader.consume<uint64_t>();
+	count_size += 8;
+	uint32_t val_length_decode = reader.consume<uint32_t>();
+	count_size += 4;
+
+	printf("----------------------------------------------------------\n");
+	printf("To decode value:%s\n", getHexString(val).c_str());
+	if ( val_length_decode != (val.size() - 12) ) {
+		fprintf(stderr, "%s[PARSE ERROR]!!! val_length_decode:%d != val.size:%d\n", prefix.c_str(), val_length_decode, val.size());
+	} else {
+		if ( debug_verbose ) {
+			printf("%s[PARSE SUCCESS] val_length_decode:%d == (val.size:%d - 12)\n", prefix.c_str(), val_length_decode, val.size());
+		}
+	}
+
+	// Get the mutation header
+	while (1) {
+		// stop when reach the end of the string
+		if(reader.eof() ) { //|| *reader.rptr == 0xFFCheckRestoreRequestDoneErrorMX
+			//printf("Finish decode the value\n");
+			break;
+		}
+
+
+		uint32_t type = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
+		uint32_t kLen = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
+		uint32_t vLen = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
+		const uint8_t *k = reader.consume(kLen);
+		const uint8_t *v = reader.consume(vLen);
+		count_size += 4 * 3 + kLen + vLen;
+
+		if ( kLen < 0 || kLen > val.size() || vLen < 0 || vLen > val.size() ) {
+			fprintf(stderr, "%s[PARSE ERROR]!!!! kLen:%d(0x%04x) vLen:%d(0x%04x)\n", prefix.c_str(), kLen, kLen, vLen, vLen);
+		}
+
+		if ( debug_verbose ) {
+			printf("%s---DedodeBackupMutation: Type:%d K:%s V:%s k_size:%d v_size:%d\n", prefix.c_str(),
+				   type,  getHexString(KeyRef(k, kLen)).c_str(), getHexString(KeyRef(v, vLen)).c_str(), kLen, vLen);
+		}
+
+	}
+	if ( debug_verbose ) {
+		printf("----------------------------------------------------------\n");
+	}
+}
+
+void printBackupLogKeyHex(Standalone<StringRef> key_input, std::string prefix) {
+	std::stringstream ss;
+	const int version_size = 12;
+	const int header_size = 12;
+	StringRef val = key_input.contents();
+	StringRefReaderMX reader(val, restore_corrupted_data());
+
+	int count_size = 0;
+	// Get the version
+	uint64_t version = reader.consume<uint64_t>();
+	count_size += 8;
+	uint32_t val_length_decode = reader.consume<uint32_t>();
+	count_size += 4;
+
+	printf("----------------------------------------------------------\n");
+	printf("To decode value:%s\n", getHexString(val).c_str());
+	if ( val_length_decode != (val.size() - 12) ) {
+		fprintf(stderr, "%s[PARSE ERROR]!!! val_length_decode:%d != val.size:%d\n", prefix.c_str(), val_length_decode, val.size());
+	} else {
+		printf("%s[PARSE SUCCESS] val_length_decode:%d == (val.size:%d - 12)\n", prefix.c_str(), val_length_decode, val.size());
+	}
+
+	// Get the mutation header
+	while (1) {
+		// stop when reach the end of the string
+		if(reader.eof() ) { //|| *reader.rptr == 0xFF
+			//printf("Finish decode the value\n");
+			break;
+		}
+
+
+		uint32_t type = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
+		uint32_t kLen = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
+		uint32_t vLen = reader.consume<uint32_t>();//reader.consumeNetworkUInt32();
+		const uint8_t *k = reader.consume(kLen);
+		const uint8_t *v = reader.consume(vLen);
+		count_size += 4 * 3 + kLen + vLen;
+
+		if ( kLen < 0 || kLen > val.size() || vLen < 0 || vLen > val.size() ) {
+			printf("%s[PARSE ERROR]!!!! kLen:%d(0x%04x) vLen:%d(0x%04x)\n", prefix.c_str(), kLen, kLen, vLen, vLen);
+		}
+
+		printf("%s---DedoceBackupMutation: Type:%d K:%s V:%s k_size:%d v_size:%d\n", prefix.c_str(),
+			   type,  getHexString(KeyRef(k, kLen)).c_str(), getHexString(KeyRef(v, vLen)).c_str(), kLen, vLen);
+
+	}
+	printf("----------------------------------------------------------\n");
+}
+
+void printLowerBounds(std::vector<Standalone<KeyRef>> lowerBounds) {
+	if ( debug_verbose == false )
+		return;
+
+	printf("[INFO] Print out %ld keys in the lowerbounds\n", lowerBounds.size());
+	for (int i = 0; i < lowerBounds.size(); i++) {
+		printf("\t[INFO][%d] %s\n", i, getHexString(lowerBounds[i]).c_str());
+	}
+}
+
+
+void printApplierKeyRangeInfo(std::map<UID, Standalone<KeyRangeRef>>  appliers) {
+	printf("[INFO] appliers num:%ld\n", appliers.size());
+	int index = 0;
+	for(auto &applier : appliers) {
+		printf("\t[INFO][Applier:%d] ID:%s --> KeyRange:%s\n", index, applier.first.toString().c_str(), applier.second.toString().c_str());
+	}
+}
diff --git a/fdbserver/RestoreRoleCommon.actor.h b/fdbserver/RestoreRoleCommon.actor.h
new file mode 100644
index 0000000000..073f02fad7
--- /dev/null
+++ b/fdbserver/RestoreRoleCommon.actor.h
@@ -0,0 +1,200 @@
+/*
+ * RestoreRoleCommon.h
+ *
+ * This source file is part of the FoundationDB open source project
+ *
+ * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
+ *
+ * Licensed 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.
+ */
+
+// Delcare commone struct and functions used in fast restore
+
+#pragma once
+#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_RestoreRoleCommon_G_H)
+	#define FDBSERVER_RestoreRoleCommon_G_H
+	#include "fdbserver/RestoreRoleCommon.actor.g.h"
+#elif !defined(FDBSERVER_RestoreRoleCommon_H)
+	#define FDBSERVER_RestoreRoleCommon_H
+
+#include <sstream>
+#include "flow/Stats.h"
+#include "fdbclient/FDBTypes.h"
+#include "fdbclient/CommitTransaction.h"
+#include "fdbrpc/fdbrpc.h"
+#include "fdbserver/CoordinationInterface.h"
+#include "fdbrpc/Locality.h"
+
+#include "fdbserver/RestoreUtil.h"
+#include "fdbserver/RestoreWorkerInterface.h"
+
+extern bool debug_verbose;
+extern double mutationVectorThreshold;
+
+struct RestoreRoleInterface;
+struct RestoreLoaderInterface;
+struct RestoreApplierInterface;
+
+struct RestoreRoleData;
+struct RestoreMasterData;
+
+struct RestoreSimpleRequest;
+
+ACTOR Future<Void> handleHeartbeat(RestoreSimpleRequest req, UID id);
+ACTOR Future<Void> handleCollectRestoreRoleInterfaceRequest(RestoreSimpleRequest req, Reference<RestoreRoleData> self, Database cx);
+ACTOR Future<Void> handleInitVersionBatchRequest(RestoreVersionBatchRequest req, Reference<RestoreRoleData> self);
+
+ACTOR Future<Void> _collectRestoreRoleInterfaces(Reference<RestoreRoleData> self, Database cx);
+
+// Helper class for reading restore data from a buffer and throwing the right errors.
+// This struct is mostly copied from StringRefReader. We add a sanity check in this struct.
+// TODO: Merge this struct with StringRefReader.
+struct StringRefReaderMX {
+	StringRefReaderMX(StringRef s = StringRef(), Error e = Error()) : rptr(s.begin()), end(s.end()), failure_error(e), str_size(s.size()) {}
+
+	// Return remainder of data as a StringRef
+	StringRef remainder() {
+		return StringRef(rptr, end - rptr);
+	}
+
+	// Return a pointer to len bytes at the current read position and advance read pos
+	//Consume a little-Endian data. Since we only run on little-Endian machine, the data on storage is little Endian
+	const uint8_t * consume(unsigned int len) {
+		if(rptr == end && len != 0)
+			throw end_of_stream();
+		const uint8_t *p = rptr;
+		rptr += len;
+		if(rptr > end) {
+			printf("[ERROR] StringRefReaderMX throw error! string length:%d\n", str_size);
+			printf("!!!!!!!!!!!![ERROR]!!!!!!!!!!!!!! Worker may die due to the error. Master will stuck when a worker die\n");
+			throw failure_error;
+		}
+		return p;
+	}
+
+	// Return a T from the current read position and advance read pos
+	template<typename T> const T consume() {
+		return *(const T *)consume(sizeof(T));
+	}
+
+	// Functions for consuming big endian (network byte oselfer) integers.
+	// Consumes a big endian number, swaps it to little endian, and returns it.
+	const int32_t  consumeNetworkInt32()  { return (int32_t)bigEndian32((uint32_t)consume< int32_t>());}
+	const uint32_t consumeNetworkUInt32() { return          bigEndian32(          consume<uint32_t>());}
+
+	const int64_t  consumeNetworkInt64()  { return (int64_t)bigEndian64((uint32_t)consume< int64_t>());}
+	const uint64_t consumeNetworkUInt64() { return          bigEndian64(          consume<uint64_t>());}
+
+	bool eof() { return rptr == end; }
+
+	const uint8_t *rptr, *end;
+	const int str_size;
+	Error failure_error;
+};
+
+struct RestoreRoleData :  NonCopyable, public ReferenceCounted<RestoreRoleData> {
+public:	
+	RestoreRole role;
+	UID nodeID; // RestoreLoader role ID
+	int nodeIndex; // RestoreLoader role index, which is continuous and easy for debuggging
+
+	std::map<UID, RestoreLoaderInterface> loadersInterf;
+	std::map<UID, RestoreApplierInterface> appliersInterf;
+	RestoreApplierInterface masterApplierInterf;
+
+	std::map<CMDUID, int> processedCmd;
+	uint32_t inProgressFlag = 0;
+
+	RestoreRoleData() : role(RestoreRole::Invalid) {};
+
+	~RestoreRoleData() {};
+
+	UID id() const { return nodeID; }
+
+	bool isCmdProcessed(CMDUID const &cmdID) {
+		return processedCmd.find(cmdID) != processedCmd.end();
+	}
+
+	// Helper functions to set/clear the flag when a worker is in the middle of processing an actor.
+	void setInProgressFlag(RestoreCommandEnum phaseEnum) {
+		int phase = (int) phaseEnum;
+		ASSERT(phase < 32);
+		inProgressFlag |= (1UL << phase);
+	}
+
+	void clearInProgressFlag(RestoreCommandEnum phaseEnum) {
+		int phase = (int) phaseEnum;
+		ASSERT(phase < 32);
+		inProgressFlag &= ~(1UL << phase);
+	}
+
+	bool isInProgress(RestoreCommandEnum phaseEnum) {
+		int phase = (int) phaseEnum;
+		ASSERT(phase < 32);
+		return (inProgressFlag & (1UL << phase));
+	}
+
+	void resetPerVersionBatch() {
+		processedCmd.clear();		
+		inProgressFlag = 0;
+	}
+
+	void clearInterfaces() {
+		loadersInterf.clear();
+		appliersInterf.clear();
+	}
+
+	std::string describeNode() {
+		std::stringstream ss;
+		ss << "RestoreRoleData role:" << getRoleStr(role);
+		return ss.str();
+	}
+
+	// TODO: To remove this function
+	std::vector<UID> getApplierIDs() {
+		std::vector<UID> applierIDs;
+		for (auto &applier : appliersInterf) {
+			applierIDs.push_back(applier.first);
+		}
+		return applierIDs;
+	}
+
+	// TODO: To remove this function
+	std::vector<UID> getLoaderIDs() {
+		std::vector<UID> loaderIDs;
+		for (auto &loader : loadersInterf) {
+			loaderIDs.push_back(loader.first);
+		}
+
+		return loaderIDs;
+	}
+
+	// TODO: To remove this function
+	std::vector<UID> getWorkerIDs() {
+		std::vector<UID> workerIDs;
+		for (auto &loader : loadersInterf) {
+			workerIDs.push_back(loader.first);
+		}
+		for (auto &applier : appliersInterf) {
+			workerIDs.push_back(applier.first);
+		}
+
+		return workerIDs;
+	}
+
+};
+
+void printLowerBounds(std::vector<Standalone<KeyRef>> lowerBounds);
+void printApplierKeyRangeInfo(std::map<UID, Standalone<KeyRangeRef>>  appliers);
+
+#endif
\ No newline at end of file
diff --git a/fdbserver/RestoreUtil.actor.cpp b/fdbserver/RestoreUtil.actor.cpp
new file mode 100644
index 0000000000..ed54d2ef6b
--- /dev/null
+++ b/fdbserver/RestoreUtil.actor.cpp
@@ -0,0 +1,70 @@
+/*
+ * RestoreUtil.cpp
+ *
+ * This source file is part of the FoundationDB open source project
+ *
+ * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
+ *
+ * Licensed 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.
+ */
+
+#include "fdbserver/RestoreUtil.h"
+
+#include "flow/actorcompiler.h"  // This must be the last #include.
+
+std::vector<std::string> RestoreRoleStr = {"Invalid", "Master", "Loader", "Applier"};
+int numRoles = RestoreRoleStr.size();
+
+std::string getRoleStr(RestoreRole role) {
+	if ( (int) role >= numRoles || (int) role < 0) {
+		printf("[ERROR] role:%d is out of scope\n", (int) role);
+		return "[Unset]";
+	}
+	return RestoreRoleStr[(int)role];
+}
+
+// CMDUID implementation
+void CMDUID::initPhase(RestoreCommandEnum newPhase) {
+	printf("CMDID, current phase:%d, new phase:%d\n", phase, newPhase);
+	phase = (uint16_t) newPhase;
+	cmdID = 0;
+}
+
+void CMDUID::nextPhase() {
+	phase++;
+	cmdID = 0;
+}
+
+void CMDUID::nextCmd() {
+	cmdID++;
+}
+
+RestoreCommandEnum CMDUID::getPhase() {
+	return (RestoreCommandEnum) phase;
+}
+
+void CMDUID::setPhase(RestoreCommandEnum newPhase) {
+	phase = (uint16_t) newPhase;
+}
+
+void CMDUID::setBatch(int newBatchIndex) {
+	batch = newBatchIndex;
+}
+
+uint64_t CMDUID::getIndex() {
+	return cmdID;
+}
+
+std::string CMDUID::toString() const {
+	return format("%04ld|%04ld|%016lld", batch, phase, cmdID);
+}
diff --git a/fdbserver/RestoreUtil.h b/fdbserver/RestoreUtil.h
new file mode 100644
index 0000000000..4e9ceed149
--- /dev/null
+++ b/fdbserver/RestoreUtil.h
@@ -0,0 +1,146 @@
+/*
+ * RestoreUtil.h
+ *
+ * This source file is part of the FoundationDB open source project
+ *
+ * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
+ *
+ * Licensed 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.
+ */
+
+// This file defines the commonly used data structure and functions
+// that are used by both RestoreWorker and RestoreRoles(Master, Loader, and Applier)
+
+#ifndef FDBSERVER_RESTOREUTIL_H
+#define FDBSERVER_RESTOREUTIL_H
+#pragma once
+
+#include "fdbclient/Tuple.h"
+#include "flow/flow.h"
+#include "flow/Stats.h"
+#include "fdbrpc/fdbrpc.h"
+#include "fdbrpc/IAsyncFile.h"
+
+
+// RestoreCommandEnum is also used as the phase ID for CMDUID
+enum class RestoreCommandEnum {Init = 0,
+		Set_Role, Set_Role_Done,
+		Sample_Range_File, Sample_Log_File, Sample_File_Done,
+		Loader_Send_Sample_Mutation_To_Applier, Loader_Send_Sample_Mutation_To_Applier_Done, //7
+		Calculate_Applier_KeyRange, Get_Applier_KeyRange, Get_Applier_KeyRange_Done, //10
+		Assign_Applier_KeyRange, Assign_Applier_KeyRange_Done, //12
+		Assign_Loader_Range_File, Assign_Loader_Log_File, Assign_Loader_File_Done,//15
+		Loader_Send_Mutations_To_Applier, Loader_Send_Mutations_To_Applier_Done,//17
+		Apply_Mutation_To_DB, Apply_Mutation_To_DB_Skip, //19
+		Loader_Notify_Appler_To_Apply_Mutation,
+		Notify_Loader_ApplierKeyRange, Notify_Loader_ApplierKeyRange_Done, //22
+		Finish_Restore, Reset_VersionBatch, Set_WorkerInterface, Collect_RestoreRoleInterface,
+		Heart_Beat}; //23
+BINARY_SERIALIZABLE(RestoreCommandEnum);
+
+enum class RestoreRole {Invalid = 0, Master = 1, Loader, Applier};
+BINARY_SERIALIZABLE( RestoreRole );
+
+extern std::vector<std::string> RestoreRoleStr;
+extern int numRoles;
+
+std::string getRoleStr(RestoreRole role);
+
+// Restore command's UID. uint64_t part[2];
+// part[0] is the phase id, part[1] is the command index in the phase.
+// TODO: Add another field to indicate version-batch round
+class CMDUID {
+public:
+	uint16_t batch;
+	uint16_t phase;
+	uint64_t cmdID;
+	CMDUID() : batch(0), phase(0), cmdID(0) { }
+	CMDUID( uint16_t a, uint64_t b ) { batch = 0; phase=a; cmdID=b; }
+	CMDUID(const CMDUID &cmd) { batch = cmd.batch; phase = cmd.phase; cmdID = cmd.cmdID; }
+
+	void initPhase(RestoreCommandEnum phase);
+
+	void nextPhase(); // Set to the next phase.
+
+	void nextCmd(); // Increase the command index at the same phase
+
+	RestoreCommandEnum getPhase();
+	void setPhase(RestoreCommandEnum newPhase);
+	void setBatch(int newBatchIndex);
+
+	uint64_t getIndex();
+
+	std::string toString() const;
+
+	bool operator == ( const CMDUID& r ) const { return batch == r.batch && phase == r.phase && cmdID == r.cmdID; }
+	bool operator != ( const CMDUID& r ) const { return batch != r.batch || phase != r.phase || cmdID != r.cmdID; }
+	bool operator < ( const CMDUID& r ) const { return batch < r.batch || (batch == r.batch && phase < r.phase) || (batch == r.batch && phase == r.phase && cmdID < r.cmdID); }
+
+	//uint64_t hash() const { return first(); }
+	//uint64_t first() const { return part[0]; }
+	//uint64_t second() const { return part[1]; }
+
+	template <class Ar>
+	void serialize_unversioned(Ar& ar) { // Changing this serialization format will affect key definitions, so can't simply be versioned!
+		serializer(ar, batch, phase, cmdID);
+	}
+};
+template <class Ar> void load( Ar& ar, CMDUID& uid ) { uid.serialize_unversioned(ar); }
+template <class Ar> void save( Ar& ar, CMDUID const& uid ) { const_cast<CMDUID&>(uid).serialize_unversioned(ar); }
+
+ struct FastRestoreStatus {
+	double curWorkloadSize;
+	double curRunningTime;
+	double curSpeed;
+
+	double totalWorkloadSize;
+	double totalRunningTime;
+	double totalSpeed;
+};
+
+// Common restore request/response interface
+// Reply type
+struct RestoreCommonReply { 
+	UID id; // unique ID of the server who sends the reply
+	CMDUID cmdID; // The restore command for the reply
+	
+	RestoreCommonReply() : id(UID()), cmdID(CMDUID()) {}
+	explicit RestoreCommonReply(UID id, CMDUID cmdID) : id(id), cmdID(cmdID) {}
+	
+	std::string toString() const {
+		std::stringstream ss;
+		ss << "ServerNodeID:" << id.toString() << " CMDID:" << cmdID.toString();
+		return ss.str();
+	}
+
+	template <class Ar>
+	void serialize(Ar& ar) {
+		serializer(ar, id, cmdID);
+	}
+};
+
+struct RestoreSimpleRequest : TimedRequest {
+	CMDUID cmdID;
+
+	ReplyPromise<RestoreCommonReply> reply;
+
+	RestoreSimpleRequest() : cmdID(CMDUID()) {}
+	explicit RestoreSimpleRequest(CMDUID cmdID) : cmdID(cmdID) {}
+
+	template <class Ar> 
+	void serialize( Ar& ar ) {
+		serializer(ar, cmdID, reply);
+	}
+};
+
+#endif //FDBSERVER_RESTOREUTIL_ACTOR_H
\ No newline at end of file
diff --git a/fdbserver/RestoreWorkerInterface.h b/fdbserver/RestoreWorkerInterface.h
index 35d4cdd255..cd1abd44f7 100644
--- a/fdbserver/RestoreWorkerInterface.h
+++ b/fdbserver/RestoreWorkerInterface.h
@@ -18,8 +18,10 @@
  * limitations under the License.
  */
 
-#ifndef FDBSERVER_RestoreWorkerInterface_H
-#define FDBSERVER_RestoreWorkerInterface_H
+// Declare and define the interface for restore worker/loader/applier
+
+#ifndef FDBSERVER_RESTORE_WORKER_INTERFACE_H
+#define FDBSERVER_RESTORE_WORKER_INTERFACE_H
 #pragma once
 
 #include <sstream>
@@ -30,11 +32,12 @@
 #include "fdbserver/CoordinationInterface.h"
 #include "fdbrpc/Locality.h"
 
+#include "fdbserver/RestoreUtil.h"
+//#include "fdbserver/RestoreRoleCommon.actor.h"
+
+#include "flow/actorcompiler.h" // has to be last include
 
 class RestoreConfig;
-enum class RestoreRole {Invalid = 0, Master = 1, Loader, Applier};
-extern std::vector<std::string> RestoreRoleStr;
-BINARY_SERIALIZABLE( RestoreRole );
 
 
 // Timeout threshold in seconds for restore commands
@@ -43,8 +46,7 @@ extern int FastRestore_Failure_Timeout;
 struct RestoreCommonReply;
 struct GetKeyRangeReply;
 struct GetKeyRangeReply;
-struct RestoreSetRoleRequest;
-struct RestoreSimpleRequest;
+struct RestoreRecruitRoleRequest;
 struct RestoreLoadFileRequest;
 struct RestoreGetApplierKeyRangeRequest;
 struct RestoreSetApplierKeyRangeRequest;
@@ -54,124 +56,87 @@ struct RestoreCalculateApplierKeyRangeRequest;
 struct RestoreSendMutationVectorRequest;
 struct RestoreSetApplierKeyRangeVectorRequest;
 
-// RestoreCommandEnum is also used as the phase ID for CMDUID
-enum class RestoreCommandEnum {Init = 0,
-		Set_Role, Set_Role_Done,
-		Sample_Range_File, Sample_Log_File, Sample_File_Done,
-		Loader_Send_Sample_Mutation_To_Applier, Loader_Send_Sample_Mutation_To_Applier_Done, //7
-		Calculate_Applier_KeyRange, Get_Applier_KeyRange, Get_Applier_KeyRange_Done, //10
-		Assign_Applier_KeyRange, Assign_Applier_KeyRange_Done, //12
-		Assign_Loader_Range_File, Assign_Loader_Log_File, Assign_Loader_File_Done,//15
-		Loader_Send_Mutations_To_Applier, Loader_Send_Mutations_To_Applier_Done,//17
-		Apply_Mutation_To_DB, Apply_Mutation_To_DB_Skip, //19
-		Loader_Notify_Appler_To_Apply_Mutation,
-		Notify_Loader_ApplierKeyRange, Notify_Loader_ApplierKeyRange_Done, //22
-		Finish_Restore, RESET_VersionBatch, Set_WorkerInterface}; //23
-BINARY_SERIALIZABLE(RestoreCommandEnum);
 
-// Restore command's UID. uint64_t part[2];
-// part[0] is the phase id, part[1] is the command index in the phase.
-// TODO: Add another field to indicate version-batch round
-class CMDUID {
-public:
-	uint16_t batch;
-	uint16_t phase;
-	uint64_t cmdID;
-	CMDUID() : batch(0), phase(0), cmdID(0) { }
-	CMDUID( uint16_t a, uint64_t b ) { batch = 0; phase=a; cmdID=b; }
-	CMDUID(const CMDUID &cmd) { batch = cmd.batch; phase = cmd.phase; cmdID = cmd.cmdID; }
+struct RestoreWorkerInterface {
+	UID interfID;
 
-	void initPhase(RestoreCommandEnum phase);
+	RequestStream<RestoreSimpleRequest> heartbeat;
+	RequestStream<RestoreRecruitRoleRequest> recruitRole;
+	RequestStream<RestoreSimpleRequest> terminateWorker;
 
-	void nextPhase(); // Set to the next phase.
+	bool operator == (RestoreWorkerInterface const& r) const { return id() == r.id(); }
+	bool operator != (RestoreWorkerInterface const& r) const { return id() != r.id(); }
 
-	void nextCmd(); // Increase the command index at the same phase
+	UID id() const { return interfID; } //cmd.getEndpoint().token;
 
-	RestoreCommandEnum getPhase();
-	void setPhase(RestoreCommandEnum newPhase);
-	void setBatch(int newBatchIndex);
+	NetworkAddress address() const { return recruitRole.getEndpoint().addresses.address; }
 
-	uint64_t getIndex();
+	void initEndpoints() {
+		heartbeat.getEndpoint( TaskClusterController );
+		recruitRole.getEndpoint( TaskClusterController );// Q: Why do we need this? 
+		terminateWorker.getEndpoint( TaskClusterController ); 
 
-	std::string toString() const;
-
-	bool operator == ( const CMDUID& r ) const { return batch == r.batch && phase == r.phase && cmdID == r.cmdID; }
-	bool operator != ( const CMDUID& r ) const { return batch != r.batch || phase != r.phase || cmdID != r.cmdID; }
-	bool operator < ( const CMDUID& r ) const { return batch < r.batch || (batch == r.batch && phase < r.phase) || (batch == r.batch && phase == r.phase && cmdID < r.cmdID); }
-
-	//uint64_t hash() const { return first(); }
-	//uint64_t first() const { return part[0]; }
-	//uint64_t second() const { return part[1]; }
+		interfID = g_random->randomUniqueID();
+	}
 
 	template <class Ar>
-	void serialize_unversioned(Ar& ar) { // Changing this serialization format will affect key definitions, so can't simply be versioned!
-		serializer(ar, batch, phase, cmdID);
+	void serialize( Ar& ar ) {
+		serializer(ar, interfID, heartbeat, recruitRole, terminateWorker);
 	}
 };
 
-template <class Ar> void load( Ar& ar, CMDUID& uid ) { uid.serialize_unversioned(ar); }
-template <class Ar> void save( Ar& ar, CMDUID const& uid ) { const_cast<CMDUID&>(uid).serialize_unversioned(ar); }
 
+struct RestoreRoleInterface {
+public:	
+	RestoreRole role;
 
-// NOTE: is cmd's Endpoint token the same with the request's token for the same node?
-struct RestoreInterface {
+	RestoreRoleInterface() {
+		role = RestoreRole::Invalid;
+	}
+};
+
+struct RestoreLoaderInterface : RestoreRoleInterface {
+public:	
 	UID nodeID;
 
 	RequestStream<RestoreSimpleRequest> heartbeat;
 
-	RequestStream<RestoreSetRoleRequest> setRole;
 	RequestStream<RestoreLoadFileRequest> sampleRangeFile;
 	RequestStream<RestoreLoadFileRequest> sampleLogFile;
-	RequestStream<RestoreSendMutationVectorRequest> sendSampleMutationVector;
 
-	RequestStream<RestoreCalculateApplierKeyRangeRequest> calculateApplierKeyRange;
-	RequestStream<RestoreGetApplierKeyRangeRequest> getApplierKeyRangeRequest;
-	RequestStream<RestoreSetApplierKeyRangeRequest> setApplierKeyRangeRequest; // To delete
 	RequestStream<RestoreSetApplierKeyRangeVectorRequest> setApplierKeyRangeVectorRequest;
 
 	RequestStream<RestoreLoadFileRequest> loadRangeFile;
 	RequestStream<RestoreLoadFileRequest> loadLogFile;
-	RequestStream<RestoreSendMutationVectorRequest> sendMutationVector;
-	RequestStream<RestoreSimpleRequest> applyToDB;
 
 	RequestStream<RestoreVersionBatchRequest> initVersionBatch;
 
-	RequestStream<RestoreSimpleRequest> setWorkerInterface;
+	RequestStream<RestoreSimpleRequest> collectRestoreRoleInterfaces; // TODO: Change to collectRestoreRoleInterfaces
 
 	RequestStream<RestoreSimpleRequest> finishRestore;
 
-	// ToDelete
-//	RequestStream< struct RestoreCommand > cmd; // Restore commands from master to loader and applier
-//	RequestStream< struct RestoreRequest > request; // Restore requests used by loader and applier
+	bool operator == (RestoreWorkerInterface const& r) const { return id() == r.id(); }
+	bool operator != (RestoreWorkerInterface const& r) const { return id() != r.id(); }
 
-	bool operator == (RestoreInterface const& r) const { return id() == r.id(); }
-	bool operator != (RestoreInterface const& r) const { return id() != r.id(); }
+	UID id() const { return nodeID; }
 
-	UID id() const { return nodeID; } //cmd.getEndpoint().token;
-
-	NetworkAddress address() const { return setRole.getEndpoint().addresses.address; }
+	NetworkAddress address() const { return heartbeat.getEndpoint().addresses.address; }
 
 	void initEndpoints() {
 		heartbeat.getEndpoint( TaskClusterController );
 		
-		setRole.getEndpoint( TaskClusterController );// Q: Why do we need this? 
 		sampleRangeFile.getEndpoint( TaskClusterController ); 
 		sampleLogFile.getEndpoint( TaskClusterController ); 
-		sendSampleMutationVector.getEndpoint( TaskClusterController ); 
 
-		calculateApplierKeyRange.getEndpoint( TaskClusterController ); 
-		getApplierKeyRangeRequest.getEndpoint( TaskClusterController ); 
-		setApplierKeyRangeRequest.getEndpoint( TaskClusterController );
 		setApplierKeyRangeVectorRequest.getEndpoint( TaskClusterController ); 
 
 		loadRangeFile.getEndpoint( TaskClusterController ); 
 		loadLogFile.getEndpoint( TaskClusterController ); 
-		sendMutationVector.getEndpoint( TaskClusterController ); 
-		applyToDB.getEndpoint( TaskClusterController ); 
 		
 		initVersionBatch.getEndpoint( TaskClusterController );
 
-		setWorkerInterface.getEndpoint( TaskClusterController ); 
+		collectRestoreRoleInterfaces.getEndpoint( TaskClusterController ); 
+
 		finishRestore.getEndpoint( TaskClusterController ); 
 
 		nodeID = g_random->randomUniqueID();
@@ -179,10 +144,73 @@ struct RestoreInterface {
 
 	template <class Ar>
 	void serialize( Ar& ar ) {
-		serializer(ar, nodeID, heartbeat, setRole, sampleRangeFile, sampleLogFile, sendSampleMutationVector,
-				calculateApplierKeyRange, getApplierKeyRangeRequest, setApplierKeyRangeRequest, setApplierKeyRangeVectorRequest,
-				loadRangeFile, loadLogFile, sendMutationVector, applyToDB, initVersionBatch, setWorkerInterface,
-				finishRestore);
+		serializer(ar, nodeID, heartbeat, sampleRangeFile, sampleLogFile,
+				setApplierKeyRangeVectorRequest, loadRangeFile, loadLogFile, 
+				initVersionBatch, collectRestoreRoleInterfaces, finishRestore);
+	}
+};
+
+
+struct RestoreApplierInterface : RestoreRoleInterface {
+public:
+	UID nodeID;
+
+	RequestStream<RestoreSimpleRequest> heartbeat;
+
+	RequestStream<RestoreCalculateApplierKeyRangeRequest> calculateApplierKeyRange;
+	RequestStream<RestoreGetApplierKeyRangeRequest> getApplierKeyRangeRequest;
+	RequestStream<RestoreSetApplierKeyRangeRequest> setApplierKeyRangeRequest;
+
+	RequestStream<RestoreSendMutationVectorRequest> sendSampleMutationVector;
+	RequestStream<RestoreSendMutationVectorRequest> sendMutationVector;
+
+	RequestStream<RestoreSimpleRequest> applyToDB;
+
+	RequestStream<RestoreVersionBatchRequest> initVersionBatch;
+
+	RequestStream<RestoreSimpleRequest> collectRestoreRoleInterfaces;
+
+	RequestStream<RestoreSimpleRequest> finishRestore;
+
+
+	bool operator == (RestoreWorkerInterface const& r) const { return id() == r.id(); }
+	bool operator != (RestoreWorkerInterface const& r) const { return id() != r.id(); }
+
+	UID id() const { return nodeID; }
+
+	NetworkAddress address() const { return heartbeat.getEndpoint().addresses.address; }
+
+	void initEndpoints() {
+		heartbeat.getEndpoint( TaskClusterController );
+	
+		calculateApplierKeyRange.getEndpoint( TaskClusterController ); 
+		getApplierKeyRangeRequest.getEndpoint( TaskClusterController ); 
+		setApplierKeyRangeRequest.getEndpoint( TaskClusterController );
+
+		sendSampleMutationVector.getEndpoint( TaskClusterController ); 
+		sendMutationVector.getEndpoint( TaskClusterController ); 
+
+		applyToDB.getEndpoint( TaskClusterController ); 
+		
+		initVersionBatch.getEndpoint( TaskClusterController );
+
+		collectRestoreRoleInterfaces.getEndpoint( TaskClusterController ); 
+
+		finishRestore.getEndpoint( TaskClusterController ); 
+
+		nodeID = g_random->randomUniqueID();
+	}
+
+	template <class Ar>
+	void serialize( Ar& ar ) {
+		serializer(ar, nodeID, heartbeat,  calculateApplierKeyRange, 
+				getApplierKeyRangeRequest, setApplierKeyRangeRequest,
+				sendSampleMutationVector, sendMutationVector,
+			    applyToDB, initVersionBatch, collectRestoreRoleInterfaces, finishRestore);
+	}
+
+	std::string toString() {
+		return nodeID.toString();
 	}
 };
 
@@ -215,21 +243,26 @@ struct LoadingParam {
 };
 
 
-struct RestoreSetRoleRequest : TimedRequest {
+struct RestoreRecruitRoleRequest : TimedRequest {
 	CMDUID cmdID;
 	RestoreRole role;
-	int nodeIndex;
-	UID masterApplierID;
+	int nodeIndex; // Each role is a node
 
 	ReplyPromise<RestoreCommonReply> reply;
 
-	RestoreSetRoleRequest() : cmdID(CMDUID()), role(RestoreRole::Invalid) {}
-	explicit RestoreSetRoleRequest(CMDUID cmdID, RestoreRole role, int nodeIndex, UID masterApplierID) : 
-				cmdID(cmdID), role(role), nodeIndex(nodeIndex), masterApplierID(masterApplierID) {}
+	RestoreRecruitRoleRequest() : cmdID(CMDUID()), role(RestoreRole::Invalid) {}
+	explicit RestoreRecruitRoleRequest(CMDUID cmdID, RestoreRole role, int nodeIndex) : 
+				cmdID(cmdID), role(role), nodeIndex(nodeIndex){}
 
 	template <class Ar> 
 	void serialize( Ar& ar ) {
-		serializer(ar, cmdID, role, nodeIndex, masterApplierID, reply);
+		serializer(ar, cmdID, role, nodeIndex, reply);
+	}
+
+	std::string printable() {
+		std::stringstream ss;
+		ss << "CMDID:" <<  cmdID.toString() <<  " Role:" << getRoleStr(role) << " NodeIndex:" << nodeIndex;
+		return ss.str();
 	}
 };
 
@@ -265,20 +298,6 @@ struct RestoreSendMutationVectorRequest : TimedRequest {
 	}
 };
 
-// CalculateApplierKeyRange, applyToDB
-struct RestoreSimpleRequest : TimedRequest {
-	CMDUID cmdID;
-
-	ReplyPromise<RestoreCommonReply> reply;
-
-	RestoreSimpleRequest() : cmdID(CMDUID()) {}
-	explicit RestoreSimpleRequest(CMDUID cmdID) : cmdID(cmdID) {}
-
-	template <class Ar> 
-	void serialize( Ar& ar ) {
-		serializer(ar, cmdID, reply);
-	}
-};
 
 struct RestoreCalculateApplierKeyRangeRequest : TimedRequest {
 	CMDUID cmdID;
@@ -358,28 +377,6 @@ struct RestoreSetApplierKeyRangeVectorRequest : TimedRequest {
 	}
 };
 
-
-
-// Reply type
-struct RestoreCommonReply { 
-	UID id; // unique ID of the server who sends the reply
-	CMDUID cmdID; // The restore command for the reply
-	
-	RestoreCommonReply() : id(UID()), cmdID(CMDUID()) {}
-	explicit RestoreCommonReply(UID id, CMDUID cmdID) : id(id), cmdID(cmdID) {}
-	
-	std::string toString() const {
-		std::stringstream ss;
-		ss << "ServerNodeID:" << id.toString() << " CMDID:" << cmdID.toString();
-		return ss.str();
-	}
-
-	template <class Ar>
-	void serialize(Ar& ar) {
-		serializer(ar, id, cmdID);
-	}
-};
-
 struct GetKeyRangeReply : RestoreCommonReply {
 	int index;
 	Standalone<KeyRef> lowerBound; // inclusive
diff --git a/fdbserver/fdbserver.vcxproj b/fdbserver/fdbserver.vcxproj
index d58d7fa156..0441e11575 100644
--- a/fdbserver/fdbserver.vcxproj
+++ b/fdbserver/fdbserver.vcxproj
@@ -53,7 +53,10 @@
     <ActorCompiler Include="workloads\SaveAndKill.actor.cpp" />
     <ActorCompiler Include="Resolver.actor.cpp" />
     <ActorCompiler Include="Restore.actor.cpp" />
+    <ActorCompiler Include="RestoreUtil.actor.cpp" />
     <ActorCompiler Include="RestoreCommon.actor.cpp" />
+    <ActorCompiler Include="RestoreRoleCommon.actor.cpp" />
+    <ActorCompiler Include="RestoreMaster.actor.cpp" />
     <ActorCompiler Include="RestoreLoader.actor.cpp" />
     <ActorCompiler Include="RestoreApplier.actor.cpp" />
     <ActorCompiler Include="LogSystemDiskQueueAdapter.actor.cpp" />
@@ -199,7 +202,13 @@
     <ClInclude Include="RatekeeperInterface.h" />
     <ClInclude Include="RecoveryState.h" />
     <ClInclude Include="ResolverInterface.h" />
-    <ClInclude Include="RestoreWorkerInterface.h" />
+    <ClInclude Include="RestoreUtil.h" />
+    <ActorCompiler Include="RestoreRoleCommon.actor.h">
+      <EnableCompile>false</EnableCompile>
+    </ActorCompiler>
+     <ActorCompiler Include="RestoreMaster.actor.h">
+      <EnableCompile>false</EnableCompile>
+    </ActorCompiler>
     <ActorCompiler Include="RestoreLoader.actor.h">
       <EnableCompile>false</EnableCompile>
     </ActorCompiler>
@@ -209,6 +218,7 @@
     <ActorCompiler Include="RestoreCommon.actor.h">
       <EnableCompile>false</EnableCompile>
     </ActorCompiler>
+    <ClInclude Include="RestoreWorkerInterface.h" />
     <ClInclude Include="ServerDBInfo.h" />
     <ClInclude Include="SimulatedCluster.h" />
     <ClInclude Include="sqlite\btree.h" />