You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2021/06/27 14:26:23 UTC

[iotdb] branch master updated: New features of cluster scalability and multi-raft (#3191)

This is an automated email from the ASF dual-hosted git repository.

qiaojialin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 00f2c30  New features of cluster scalability and multi-raft (#3191)
00f2c30 is described below

commit 00f2c30ff3cdaf453fc914bccc54d9064125090e
Author: Tianan Li <li...@163.com>
AuthorDate: Sun Jun 27 22:26:03 2021 +0800

    New features of cluster scalability and multi-raft (#3191)
---
 .../resources/conf/iotdb-cluster.properties        |   7 +
 cluster/src/assembly/resources/sbin/add-node.bat   |  39 +-
 cluster/src/assembly/resources/sbin/add-node.sh    |   2 +-
 .../sbin/{add-node.bat => remove-node.bat}         |  18 +-
 .../resources/sbin/{add-node.sh => remove-node.sh} |  33 +-
 cluster/src/assembly/resources/sbin/start-node.bat |   2 +-
 .../java/org/apache/iotdb/cluster/ClusterMain.java |  25 +-
 .../cluster/client/sync/SyncClientAdaptor.java     |  35 +-
 .../apache/iotdb/cluster/config/ClusterConfig.java |  24 +
 .../iotdb/cluster/config/ClusterConstant.java      |   4 +
 .../iotdb/cluster/config/ClusterDescriptor.java    |   9 +
 .../iotdb/cluster/coordinator/Coordinator.java     | 100 ++-
 .../ChangeMembershipException.java}                |  16 +-
 .../exception/CheckConsistencyException.java       |   4 +-
 .../org/apache/iotdb/cluster/log/LogParser.java    |  10 +
 .../iotdb/cluster/log/applier/BaseApplier.java     |   4 +-
 .../iotdb/cluster/log/applier/DataLogApplier.java  |  29 +-
 .../iotdb/cluster/log/applier/MetaLogApplier.java  |  41 +-
 .../iotdb/cluster/log/catchup/CatchUpTask.java     |  13 +-
 .../iotdb/cluster/log/catchup/LogCatchUpTask.java  |   7 +-
 .../cluster/log/catchup/SnapshotCatchUpTask.java   |   5 +-
 .../iotdb/cluster/log/logtypes/AddNodeLog.java     |  52 +-
 .../iotdb/cluster/log/logtypes/RemoveNodeLog.java  |  48 +-
 .../manage/FilePartitionedSnapshotLogManager.java  |  53 +-
 .../log/manage/PartitionedSnapshotLogManager.java  |  29 +-
 .../iotdb/cluster/log/manage/RaftLogManager.java   |  40 +-
 .../iotdb/cluster/log/snapshot/FileSnapshot.java   | 105 ++-
 .../cluster/log/snapshot/MetaSimpleSnapshot.java   |   4 +-
 .../cluster/log/snapshot/PartitionedSnapshot.java  |   6 +-
 .../cluster/log/snapshot/PullSnapshotTask.java     |  42 +-
 .../log/snapshot/PullSnapshotTaskDescriptor.java   |   3 +-
 .../cluster/log/snapshot/SnapshotInstaller.java    |   5 +-
 .../apache/iotdb/cluster/metadata/CMManager.java   |  21 +-
 .../apache/iotdb/cluster/metadata/MetaPuller.java  |  32 +-
 .../cluster/partition/NodeAdditionResult.java      |  13 +-
 .../iotdb/cluster/partition/NodeRemovalResult.java |  41 +-
 .../iotdb/cluster/partition/PartitionGroup.java    |  59 +-
 .../iotdb/cluster/partition/PartitionTable.java    |  41 +-
 .../partition/balancer/DefaultSlotBalancer.java    | 119 ++++
 .../SlotBalancer.java}                             |  27 +-
 .../iotdb/cluster/partition/slot/SlotManager.java  | 114 +++-
 .../partition/slot/SlotNodeAdditionResult.java     |   8 +-
 .../partition/slot/SlotNodeRemovalResult.java      |  45 +-
 .../cluster/partition/slot/SlotPartitionTable.java | 364 ++++++----
 .../iotdb/cluster/partition/slot/SlotStrategy.java |  17 +-
 .../cluster/query/ClusterDataQueryExecutor.java    |  12 +-
 .../iotdb/cluster/query/ClusterPlanExecutor.java   |  16 +-
 .../iotdb/cluster/query/ClusterPlanRouter.java     |  32 +-
 .../iotdb/cluster/query/LocalQueryExecutor.java    | 105 ++-
 .../iotdb/cluster/query/RemoteQueryContext.java    |   7 +-
 .../query/aggregate/ClusterAggregateExecutor.java  |   3 +-
 .../cluster/query/aggregate/ClusterAggregator.java |   3 +-
 .../cluster/query/fill/ClusterPreviousFill.java    |   3 +-
 .../cluster/query/filter/SlotTsFileFilter.java     |  12 +-
 .../groupby/ClusterGroupByVFilterDataSet.java      |   3 +-
 .../query/groupby/RemoteGroupByExecutor.java       |  18 +-
 .../query/last/ClusterLastQueryExecutor.java       |   3 +-
 .../cluster/query/reader/ClusterReaderFactory.java | 329 +++++++--
 .../cluster/query/reader/ClusterTimeGenerator.java |   7 +-
 .../iotdb/cluster/query/reader/DataSourceInfo.java |   3 +-
 ...er.java => ManagedDescPriorityMergeReader.java} |   8 +-
 ...Reader.java => ManagedPriorityMergeReader.java} |   4 +-
 .../cluster/query/reader/mult/MultBatchReader.java |   5 +-
 .../query/reader/mult/MultDataSourceInfo.java      |   3 +-
 .../query/reader/mult/RemoteMultSeriesReader.java  |   2 +-
 .../apache/iotdb/cluster/server/ClientServer.java  |   5 +-
 .../iotdb/cluster/server/DataClusterServer.java    | 518 ++++++++------
 .../iotdb/cluster/server/MetaClusterServer.java    |  26 +-
 .../cluster/server/PullSnapshotHintService.java    |  65 +-
 .../org/apache/iotdb/cluster/server/Response.java  |  13 +-
 .../iotdb/cluster/server/StoppedMemberManager.java |  37 +-
 .../server/clusterinfo/ClusterInfoServiceImpl.java |   2 +-
 .../server/handlers/caller/ElectionHandler.java    |   4 +
 .../server/handlers/caller/HeartbeatHandler.java   |  24 +-
 .../server/handlers/caller/NodeStatusHandler.java  |  10 +-
 .../server/heartbeat/DataHeartbeatThread.java      |   8 +-
 .../cluster/server/heartbeat/HeartbeatThread.java  |  28 +-
 .../server/heartbeat/MetaHeartbeatThread.java      |  17 +
 .../cluster/server/member/DataGroupMember.java     | 507 +++++++++-----
 .../cluster/server/member/MetaGroupMember.java     | 760 +++++++++++++--------
 .../iotdb/cluster/server/member/RaftMember.java    | 182 +++--
 .../iotdb/cluster/server/monitor/NodeReport.java   |   9 +-
 .../cluster/server/service/BaseAsyncService.java   |   6 +-
 .../cluster/server/service/BaseSyncService.java    |   6 +-
 .../cluster/server/service/DataAsyncService.java   |  64 +-
 .../cluster/server/service/DataSyncService.java    |  62 +-
 .../cluster/server/service/MetaAsyncService.java   |  42 +-
 .../cluster/server/service/MetaSyncService.java    |  37 +-
 .../apache/iotdb/cluster/utils/ClusterUtils.java   |  77 ++-
 .../iotdb/cluster/utils/NodeSerializeUtils.java    |   2 +
 .../apache/iotdb/cluster/utils/PartitionUtils.java |   4 +-
 .../apache/iotdb/cluster/utils/StatusUtils.java    |  10 +
 .../cluster/utils/nodetool/ClusterMonitor.java     | 106 ++-
 .../utils/nodetool/ClusterMonitorMBean.java        |  27 +-
 .../iotdb/cluster/utils/nodetool/NodeTool.java     |  13 +-
 .../cluster/utils/nodetool/function/Header.java    |  60 ++
 .../cluster/utils/nodetool/function/Migration.java |  59 ++
 .../utils/nodetool/function/NodeToolCmd.java       |  23 +-
 .../cluster/utils/nodetool/function/Partition.java |   6 +-
 .../cluster/utils/nodetool/function/Ring.java      |  15 +-
 .../nodetool/function/{Host.java => Slot.java}     |  26 +-
 .../cluster/utils/nodetool/function/Status.java    |  29 +-
 .../cluster/client/async/AsyncDataClientTest.java  |   2 +-
 .../cluster/client/async/AsyncMetaClientTest.java  |   2 +-
 .../cluster/client/sync/SyncClientAdaptorTest.java |  62 +-
 .../iotdb/cluster/common/TestAsyncDataClient.java  |  20 +-
 .../iotdb/cluster/common/TestDataGroupMember.java  |  14 +-
 .../iotdb/cluster/common/TestMetaGroupMember.java  |   5 +-
 .../apache/iotdb/cluster/common/TestSnapshot.java  |   4 +-
 .../org/apache/iotdb/cluster/common/TestUtils.java |  11 +
 .../iotdb/cluster/log/LogDispatcherTest.java       |   3 +-
 .../apache/iotdb/cluster/log/LogParserTest.java    |  21 +
 .../cluster/log/applier/DataLogApplierTest.java    |   7 +-
 .../cluster/log/applier/MetaLogApplierTest.java    |  14 +-
 .../iotdb/cluster/log/catchup/CatchUpTaskTest.java |  24 +-
 .../cluster/log/catchup/LogCatchUpTaskTest.java    |  17 +-
 .../log/catchup/SnapshotCatchUpTaskTest.java       |  15 +-
 .../cluster/log/logtypes/SerializeLogTest.java     |   2 +
 .../FilePartitionedSnapshotLogManagerTest.java     |   8 +-
 .../cluster/log/snapshot/DataSnapshotTest.java     |   6 +-
 .../cluster/log/snapshot/FileSnapshotTest.java     |  12 +-
 .../log/snapshot/MetaSimpleSnapshotTest.java       |   2 +-
 .../log/snapshot/PartitionedSnapshotTest.java      |   2 +-
 .../cluster/log/snapshot/PullSnapshotTaskTest.java |   5 +-
 .../iotdb/cluster/log/snapshot/SimpleSnapshot.java |   4 +-
 .../iotdb/cluster/partition/SlotManagerTest.java   |   6 +-
 .../cluster/partition/SlotPartitionTableTest.java  |  51 +-
 .../query/reader/ClusterReaderFactoryTest.java     |  10 +-
 .../reader/RemoteSeriesReaderByTimestampTest.java  |   3 +-
 .../query/reader/RemoteSimpleSeriesReaderTest.java |   3 +-
 .../mult/AssignPathManagedMergeReaderTest.java     |   3 +-
 .../reader/mult/RemoteMultSeriesReaderTest.java    |   5 +-
 .../server/heartbeat/DataHeartbeatThreadTest.java  |   8 +-
 .../server/heartbeat/MetaHeartbeatThreadTest.java  |  36 +-
 .../iotdb/cluster/server/member/BaseMember.java    |  28 +-
 .../cluster/server/member/DataGroupMemberTest.java | 153 +++--
 .../cluster/server/member/MetaGroupMemberTest.java |  94 ++-
 .../cluster/server/member/RaftMemberTest.java      |   5 +-
 .../iotdb/cluster/utils/SerializeUtilTest.java     |  18 +
 docs/UserGuide/Cluster/Cluster-Setup.md            |  39 ++
 docs/zh/UserGuide/Cluster/Cluster-Setup.md         |  35 +
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |   5 +-
 .../org/apache/iotdb/db/engine/StorageEngine.java  |   4 +-
 .../engine/storagegroup/StorageGroupProcessor.java |  78 ++-
 .../db/engine/storagegroup/TsFileResource.java     |   4 +-
 .../iotdb/db/metadata/logfile/MLogWriter.java      |  15 +-
 .../apache/iotdb/db/qp/physical/PhysicalPlan.java  |   5 +
 .../apache/iotdb/db/qp/physical/sys/LogPlan.java   |  79 +++
 .../org/apache/iotdb/db/service/TSServiceImpl.java |   1 +
 .../org/apache/iotdb/db/utils/CommonUtils.java     |   4 +-
 .../java/org/apache/iotdb/db/utils/IOUtils.java    |   9 +
 .../org/apache/iotdb/db/utils/SerializeUtils.java  |   5 +-
 .../java/org/apache/iotdb/rpc/TSStatusCode.java    |   3 +-
 thrift-cluster/src/main/thrift/cluster.thrift      |  92 +--
 .../apache/iotdb/tsfile/read/common/BatchData.java |  33 +
 .../tsfile/read/common/DescReadBatchData.java      |   5 +-
 .../tsfile/read/common/DescReadWriteBatchData.java |   1 +
 157 files changed, 4440 insertions(+), 1844 deletions(-)

diff --git a/cluster/src/assembly/resources/conf/iotdb-cluster.properties b/cluster/src/assembly/resources/conf/iotdb-cluster.properties
index ba626e3..2bdac88 100644
--- a/cluster/src/assembly/resources/conf/iotdb-cluster.properties
+++ b/cluster/src/assembly/resources/conf/iotdb-cluster.properties
@@ -72,6 +72,9 @@ seed_nodes=127.0.0.1:9003
 # number of replications for one partition
 default_replica_num=1
 
+# sub raft num for multi-raft
+multi_raft_factor=1
+
 # cluster name to identify different clusters
 # all node's cluster_name in one cluster are the same
 # cluster_name=default
@@ -169,6 +172,10 @@ default_replica_num=1
 # This default value is 1000
 # max_read_log_lag=1000
 
+# When a follower tries to sync log with the leader, sync will fail if the log Lag exceeds max_sync_log_lag.
+# This default value is 100000
+# max_sync_log_lag=100000
+
 # Max number of clients in a ClientPool of a member for one node.
 # max_client_pernode_permember_number=1000
 
diff --git a/cluster/src/assembly/resources/sbin/add-node.bat b/cluster/src/assembly/resources/sbin/add-node.bat
index 958f16f..34f90f1 100755
--- a/cluster/src/assembly/resources/sbin/add-node.bat
+++ b/cluster/src/assembly/resources/sbin/add-node.bat
@@ -19,7 +19,7 @@
 
 @echo off
 echo ````````````````````````
-echo Starting IoTDB
+echo Starting IoTDB (Cluster Mode)
 echo ````````````````````````
 
 PATH %PATH%;%JAVA_HOME%\bin\
@@ -54,15 +54,39 @@ pushd %~dp0..
 if NOT DEFINED IOTDB_HOME set IOTDB_HOME=%cd%
 popd
 
-set IOTDB_CONF=%IOTDB_HOME%\conf
-set IOTDB_LOGS=%IOTDB_HOME%\logs
+SET enable_printgc=false
+IF "%1" == "printgc" (
+  SET enable_printgc=true
+  SHIFT
+)
+
+SET IOTDB_CONF=%1
+IF "%IOTDB_CONF%" == "" (
+  SET IOTDB_CONF=%IOTDB_HOME%\conf
+) ELSE (
+  SET IOTDB_CONF="%IOTDB_CONF%"
+)
+
+SET IOTDB_LOGS=%IOTDB_HOME%\logs
 
 IF EXIST "%IOTDB_CONF%\iotdb-env.bat" (
+  IF  "%enable_printgc%" == "true" (
+    CALL "%IOTDB_CONF%\iotdb-env.bat" printgc
+  ) ELSE (
     CALL "%IOTDB_CONF%\iotdb-env.bat"
-    ) ELSE (
-    echo "can't find %IOTDB_CONF%\iotdb-env.bat"
-    )
+  )
+) ELSE IF EXIST "%IOTDB_HOME%/conf/iotdb-env.bat" (
+  IF  "%enable_printgc%" == "true" (
+    CALL "%IOTDB_HOME%/conf/iotdb-env.bat" printgc
+  ) ELSE (
+    CALL "%IOTDB_HOME%/conf/iotdb-env.bat"
+   )
+) ELSE (
+  echo "can't find iotdb-env.bat"
+)
 
+@setlocal ENABLEDELAYEDEXPANSION ENABLEEXTENSIONS
+set CONF_PARAMS=-a
 if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.cluster.ClusterMain
 if NOT DEFINED JAVA_HOME goto :err
 
@@ -72,6 +96,7 @@ set JAVA_OPTS=-ea^
  -Dlogback.configurationFile="%IOTDB_CONF%\logback.xml"^
  -DIOTDB_HOME="%IOTDB_HOME%"^
  -DTSFILE_HOME="%IOTDB_HOME%"^
+ -DTSFILE_CONF="%IOTDB_CONF%"^
  -DIOTDB_CONF="%IOTDB_CONF%"
 
 @REM ***** CLASSPATH library setting *****
@@ -92,7 +117,7 @@ goto :eof
 
 rem echo CLASSPATH: %CLASSPATH%
 
-"%JAVA_HOME%\bin\java" %JAVA_OPTS% %IOTDB_HEAP_OPTS% -cp %CLASSPATH% %IOTDB_JMX_OPTS% %MAIN_CLASS% -a
+"%JAVA_HOME%\bin\java" %JAVA_OPTS% %IOTDB_HEAP_OPTS% -cp %CLASSPATH% %IOTDB_JMX_OPTS% %MAIN_CLASS% %CONF_PARAMS%
 goto finally
 
 :err
diff --git a/cluster/src/assembly/resources/sbin/add-node.sh b/cluster/src/assembly/resources/sbin/add-node.sh
index 4304135..4deaee2 100755
--- a/cluster/src/assembly/resources/sbin/add-node.sh
+++ b/cluster/src/assembly/resources/sbin/add-node.sh
@@ -20,7 +20,7 @@
 
 
 echo ---------------------
-echo Starting IoTDB
+echo "Starting IoTDB (Cluster Mode)"
 echo ---------------------
 
 if [ -z "${IOTDB_HOME}" ]; then
diff --git a/cluster/src/assembly/resources/sbin/add-node.bat b/cluster/src/assembly/resources/sbin/remove-node.bat
similarity index 89%
copy from cluster/src/assembly/resources/sbin/add-node.bat
copy to cluster/src/assembly/resources/sbin/remove-node.bat
index 958f16f..0346771 100755
--- a/cluster/src/assembly/resources/sbin/add-node.bat
+++ b/cluster/src/assembly/resources/sbin/remove-node.bat
@@ -19,7 +19,7 @@
 
 @echo off
 echo ````````````````````````
-echo Starting IoTDB
+echo Starting to remove a node (Cluster Mode)
 echo ````````````````````````
 
 PATH %PATH%;%JAVA_HOME%\bin\
@@ -57,11 +57,12 @@ popd
 set IOTDB_CONF=%IOTDB_HOME%\conf
 set IOTDB_LOGS=%IOTDB_HOME%\logs
 
-IF EXIST "%IOTDB_CONF%\iotdb-env.bat" (
-    CALL "%IOTDB_CONF%\iotdb-env.bat"
-    ) ELSE (
-    echo "can't find %IOTDB_CONF%\iotdb-env.bat"
-    )
+@setlocal ENABLEDELAYEDEXPANSION ENABLEEXTENSIONS
+set CONF_PARAMS=-r
+set is_conf_path=false
+for %%i in (%*) do (
+	set CONF_PARAMS=!CONF_PARAMS! %%i
+)
 
 if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.cluster.ClusterMain
 if NOT DEFINED JAVA_HOME goto :err
@@ -72,6 +73,7 @@ set JAVA_OPTS=-ea^
  -Dlogback.configurationFile="%IOTDB_CONF%\logback.xml"^
  -DIOTDB_HOME="%IOTDB_HOME%"^
  -DTSFILE_HOME="%IOTDB_HOME%"^
+ -DCLUSTER_CONF="%IOTDB_CONF%"^
  -DIOTDB_CONF="%IOTDB_CONF%"
 
 @REM ***** CLASSPATH library setting *****
@@ -80,7 +82,7 @@ set CLASSPATH="%IOTDB_HOME%\lib"
 
 @REM For each jar in the IOTDB_HOME lib directory call append to build the CLASSPATH variable.
 set CLASSPATH=%CLASSPATH%;"%IOTDB_HOME%\lib\*"
-set CLASSPATH=%CLASSPATH%;iotdb.IoTDB
+set CLASSPATH=%CLASSPATH%;iotdb.ClusterMain
 goto okClasspath
 
 :append
@@ -92,7 +94,7 @@ goto :eof
 
 rem echo CLASSPATH: %CLASSPATH%
 
-"%JAVA_HOME%\bin\java" %JAVA_OPTS% %IOTDB_HEAP_OPTS% -cp %CLASSPATH% %IOTDB_JMX_OPTS% %MAIN_CLASS% -a
+"%JAVA_HOME%\bin\java" %JAVA_OPTS% %IOTDB_HEAP_OPTS% -cp %CLASSPATH% %IOTDB_JMX_OPTS% %MAIN_CLASS% %CONF_PARAMS%
 goto finally
 
 :err
diff --git a/cluster/src/assembly/resources/sbin/add-node.sh b/cluster/src/assembly/resources/sbin/remove-node.sh
similarity index 73%
copy from cluster/src/assembly/resources/sbin/add-node.sh
copy to cluster/src/assembly/resources/sbin/remove-node.sh
index 4304135..c6f89bb 100755
--- a/cluster/src/assembly/resources/sbin/add-node.sh
+++ b/cluster/src/assembly/resources/sbin/remove-node.sh
@@ -20,40 +20,16 @@
 
 
 echo ---------------------
-echo Starting IoTDB
+echo "Starting to remove a node(Cluster Mode)"
 echo ---------------------
 
 if [ -z "${IOTDB_HOME}" ]; then
   export IOTDB_HOME="`dirname "$0"`/.."
 fi
 
-enable_printgc=false
-if [ "$#" -ge "1" -a "$1" == "printgc" ]; then
-  enable_printgc=true;
-  shift
-fi
-
-IOTDB_CONF=$1
-if [ -z "${IOTDB_CONF}" ]; then
-  export IOTDB_CONF=${IOTDB_HOME}/conf
-fi
-
-if [ -f "$IOTDB_CONF/iotdb-env.sh" ]; then
-    if [ $enable_printgc == "true" ]; then
-      . "$IOTDB_CONF/iotdb-env.sh" "printgc"
-    else
-       . "$IOTDB_CONF/iotdb-env.sh"
-    fi
-elif [ -f "${IOTDB_HOME}/conf/iotdb-env.sh" ]; then
-    if [ $enable_printgc == "true" ]; then
-      . "${IOTDB_HOME}/conf/iotdb-env.sh" "printgc"
-    else
-      . "${IOTDB_HOME}/conf/iotdb-env.sh"
-    fi
-else
-    echo "can't find $IOTDB_CONF/iotdb-env.sh"
-fi
+IOTDB_CONF=${IOTDB_HOME}/conf
 
+CONF_PARAMS="-r "$*
 
 if [ -n "$JAVA_HOME" ]; then
     for java in "$JAVA_HOME"/bin/amd64/java "$JAVA_HOME"/bin/java; do
@@ -84,8 +60,9 @@ launch_service()
 	iotdb_parms="$iotdb_parms -DIOTDB_HOME=${IOTDB_HOME}"
 	iotdb_parms="$iotdb_parms -DTSFILE_HOME=${IOTDB_HOME}"
 	iotdb_parms="$iotdb_parms -DIOTDB_CONF=${IOTDB_CONF}"
+	iotdb_parms="$iotdb_parms -DCLUSTER_CONF=${IOTDB_CONF}"
 	iotdb_parms="$iotdb_parms -Dname=iotdb\.IoTDB"
-	exec "$JAVA" $iotdb_parms $IOTDB_JMX_OPTS -cp "$CLASSPATH" "$class" -a
+	exec "$JAVA" $iotdb_parms $IOTDB_JMX_OPTS -cp "$CLASSPATH" "$class" $CONF_PARAMS
 	return $?
 }
 
diff --git a/cluster/src/assembly/resources/sbin/start-node.bat b/cluster/src/assembly/resources/sbin/start-node.bat
index c324bb7..2874aae 100755
--- a/cluster/src/assembly/resources/sbin/start-node.bat
+++ b/cluster/src/assembly/resources/sbin/start-node.bat
@@ -19,7 +19,7 @@
 
 @echo off
 echo ````````````````````````
-echo Starting IoTDB
+echo Starting IoTDB (Cluster Mode)
 echo ````````````````````````
 
 PATH %PATH%;%JAVA_HOME%\bin\
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java
index a904838..adc4661 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java
@@ -51,6 +51,8 @@ import java.io.IOException;
 import java.util.HashSet;
 import java.util.Set;
 
+import static org.apache.iotdb.cluster.utils.ClusterUtils.UNKNOWN_CLIENT_IP;
+
 public class ClusterMain {
 
   private static final Logger logger = LoggerFactory.getLogger(ClusterMain.class);
@@ -99,8 +101,8 @@ public class ClusterMain {
     }
 
     String mode = args[0];
-
     logger.info("Running mode {}", mode);
+
     if (MODE_START.equals(mode)) {
       try {
         metaServer = new MetaClusterServer();
@@ -121,6 +123,7 @@ public class ClusterMain {
       }
     } else if (MODE_ADD.equals(mode)) {
       try {
+        long startTime = System.currentTimeMillis();
         metaServer = new MetaClusterServer();
         preStartCustomize();
         metaServer.start();
@@ -128,6 +131,11 @@ public class ClusterMain {
         // Currently, we do not register ClusterInfoService as a JMX Bean,
         // so we use startService() rather than start()
         ClusterInfoServer.getInstance().startService();
+
+        logger.info(
+            "Adding this node {} to cluster costs {} ms",
+            metaServer.getMember().getThisNode(),
+            (System.currentTimeMillis() - startTime));
       } catch (TTransportException
           | StartupException
           | QueryProcessException
@@ -221,7 +229,7 @@ public class ClusterMain {
     TProtocolFactory factory =
         config.isRpcThriftCompressionEnabled() ? new TCompactProtocol.Factory() : new Factory();
     Node nodeToRemove = new Node();
-    nodeToRemove.setInternalIp(ip).setMetaPort(metaPort);
+    nodeToRemove.setInternalIp(ip).setMetaPort(metaPort).setClientIp(UNKNOWN_CLIENT_IP);
     // try sending the request to each seed node
     for (String url : config.getSeedNodeUrls()) {
       Node node = ClusterUtils.parseNode(url);
@@ -230,6 +238,7 @@ public class ClusterMain {
       }
       AsyncMetaClient client = new AsyncMetaClient(factory, new TAsyncClientManager(), node, null);
       Long response = null;
+      long startTime = System.currentTimeMillis();
       try {
         logger.info("Start removing node {} with the help of node {}", nodeToRemove, node);
         response = SyncClientAdaptor.removeNode(client, nodeToRemove);
@@ -240,19 +249,25 @@ public class ClusterMain {
         logger.warn("Cannot send remove node request through {}, try next node", node);
       }
       if (response != null) {
-        handleNodeRemovalResp(response, nodeToRemove);
+        handleNodeRemovalResp(response, nodeToRemove, startTime);
         return;
       }
     }
   }
 
-  private static void handleNodeRemovalResp(Long response, Node nodeToRemove) {
+  private static void handleNodeRemovalResp(Long response, Node nodeToRemove, long startTime) {
     if (response == Response.RESPONSE_AGREE) {
-      logger.info("Node {} is successfully removed", nodeToRemove);
+      logger.info(
+          "Node {} is successfully removed, cost {}ms",
+          nodeToRemove,
+          (System.currentTimeMillis() - startTime));
     } else if (response == Response.RESPONSE_CLUSTER_TOO_SMALL) {
       logger.error("Cluster size is too small, cannot remove any node");
     } else if (response == Response.RESPONSE_REJECT) {
       logger.error("Node {} is not found in the cluster, please check", nodeToRemove);
+    } else if (response == Response.RESPONSE_DATA_MIGRATION_NOT_FINISH) {
+      logger.warn(
+          "The data migration of the previous membership change operation is not finished. Please try again later");
     } else {
       logger.error("Unexpected response {}", response);
     }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
index a1145ba..ee15d0d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
@@ -34,6 +34,7 @@ import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.PreviousFillRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSchemaRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotRequest;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.RaftService.AsyncClient;
 import org.apache.iotdb.cluster.rpc.thrift.SingleSeriesQueryRequest;
 import org.apache.iotdb.cluster.rpc.thrift.StartUpStatus;
@@ -106,7 +107,7 @@ public class SyncClientAdaptor {
   }
 
   public static Boolean matchTerm(
-      AsyncClient client, Node target, long prevLogIndex, long prevLogTerm, Node header)
+      AsyncClient client, Node target, long prevLogIndex, long prevLogTerm, RaftNode header)
       throws TException, InterruptedException {
     try {
       AtomicReference<Boolean> resultRef = new AtomicReference<>(null);
@@ -165,7 +166,7 @@ public class SyncClientAdaptor {
   }
 
   public static List<String> getNodeList(
-      AsyncDataClient client, Node header, String schemaPattern, int level)
+      AsyncDataClient client, RaftNode header, String schemaPattern, int level)
       throws TException, InterruptedException {
     GetNodesListHandler handler = new GetNodesListHandler();
     AtomicReference<List<String>> response = new AtomicReference<>(null);
@@ -182,7 +183,8 @@ public class SyncClientAdaptor {
   }
 
   public static Set<String> getChildNodeInNextLevel(
-      AsyncDataClient client, Node header, String path) throws TException, InterruptedException {
+      AsyncDataClient client, RaftNode header, String path)
+      throws TException, InterruptedException {
     GetChildNodeNextLevelHandler handler = new GetChildNodeNextLevelHandler();
     AtomicReference<Set<String>> response = new AtomicReference<>(null);
     handler.setResponse(response);
@@ -197,7 +199,7 @@ public class SyncClientAdaptor {
     return response.get();
   }
 
-  public static Set<String> getNextChildren(AsyncDataClient client, Node header, String path)
+  public static Set<String> getNextChildren(AsyncDataClient client, RaftNode header, String path)
       throws TException, InterruptedException {
     GetChildNodeNextLevelPathHandler handler = new GetChildNodeNextLevelPathHandler();
     AtomicReference<Set<String>> response = new AtomicReference<>(null);
@@ -214,7 +216,7 @@ public class SyncClientAdaptor {
   }
 
   public static ByteBuffer getAllMeasurementSchema(
-      AsyncDataClient client, Node header, ShowTimeSeriesPlan plan)
+      AsyncDataClient client, RaftNode header, ShowTimeSeriesPlan plan)
       throws IOException, InterruptedException, TException {
     GetTimeseriesSchemaHandler handler = new GetTimeseriesSchemaHandler();
     AtomicReference<ByteBuffer> response = new AtomicReference<>(null);
@@ -338,7 +340,7 @@ public class SyncClientAdaptor {
   }
 
   public static List<String> getUnregisteredMeasurements(
-      AsyncDataClient client, Node header, List<String> seriesPaths)
+      AsyncDataClient client, RaftNode header, List<String> seriesPaths)
       throws TException, InterruptedException {
     AtomicReference<List<String>> remoteResult = new AtomicReference<>();
     GenericHandler<List<String>> handler = new GenericHandler<>(client.getNode(), remoteResult);
@@ -348,7 +350,7 @@ public class SyncClientAdaptor {
   }
 
   public static GetAllPathsResult getAllPaths(
-      AsyncDataClient client, Node header, List<String> pathsToQuery, boolean withAlias)
+      AsyncDataClient client, RaftNode header, List<String> pathsToQuery, boolean withAlias)
       throws InterruptedException, TException {
     AtomicReference<GetAllPathsResult> remoteResult = new AtomicReference<>();
     GenericHandler<GetAllPathsResult> handler =
@@ -359,7 +361,7 @@ public class SyncClientAdaptor {
   }
 
   public static Integer getPathCount(
-      AsyncDataClient client, Node header, List<String> pathsToQuery, int level)
+      AsyncDataClient client, RaftNode header, List<String> pathsToQuery, int level)
       throws InterruptedException, TException {
     AtomicReference<Integer> remoteResult = new AtomicReference<>(null);
     GenericHandler<Integer> handler = new GenericHandler<>(client.getNode(), remoteResult);
@@ -369,7 +371,7 @@ public class SyncClientAdaptor {
   }
 
   public static Set<String> getAllDevices(
-      AsyncDataClient client, Node header, List<String> pathsToQuery)
+      AsyncDataClient client, RaftNode header, List<String> pathsToQuery)
       throws InterruptedException, TException {
     AtomicReference<Set<String>> remoteResult = new AtomicReference<>();
     GenericHandler<Set<String>> handler = new GenericHandler<>(client.getNode(), remoteResult);
@@ -378,7 +380,7 @@ public class SyncClientAdaptor {
     return handler.getResult(RaftServer.getReadOperationTimeoutMS());
   }
 
-  public static ByteBuffer getDevices(AsyncDataClient client, Node header, ShowDevicesPlan plan)
+  public static ByteBuffer getDevices(AsyncDataClient client, RaftNode header, ShowDevicesPlan plan)
       throws InterruptedException, TException, IOException {
     GetDevicesHandler handler = new GetDevicesHandler();
     AtomicReference<ByteBuffer> response = new AtomicReference<>(null);
@@ -416,7 +418,7 @@ public class SyncClientAdaptor {
   }
 
   public static TSStatus executeNonQuery(
-      AsyncClient client, PhysicalPlan plan, Node header, Node receiver)
+      AsyncClient client, PhysicalPlan plan, RaftNode header, Node receiver)
       throws IOException, TException, InterruptedException {
     AtomicReference<TSStatus> status = new AtomicReference<>();
     ExecutNonQueryReq req = new ExecutNonQueryReq();
@@ -445,7 +447,7 @@ public class SyncClientAdaptor {
   }
 
   public static List<ByteBuffer> getGroupByResult(
-      AsyncDataClient client, Node header, long executorId, long curStartTime, long curEndTime)
+      AsyncDataClient client, RaftNode header, long executorId, long curStartTime, long curEndTime)
       throws InterruptedException, TException {
     AtomicReference<List<ByteBuffer>> fetchResult = new AtomicReference<>();
     GenericHandler<List<ByteBuffer>> handler = new GenericHandler<>(client.getNode(), fetchResult);
@@ -455,7 +457,7 @@ public class SyncClientAdaptor {
   }
 
   public static ByteBuffer peekNextNotNullValue(
-      AsyncDataClient client, Node header, long executorId, long curStartTime, long curEndTime)
+      AsyncDataClient client, RaftNode header, long executorId, long curStartTime, long curEndTime)
       throws InterruptedException, TException {
     AtomicReference<ByteBuffer> fetchResult = new AtomicReference<>();
     GenericHandler<ByteBuffer> handler = new GenericHandler<>(client.getNode(), fetchResult);
@@ -488,7 +490,7 @@ public class SyncClientAdaptor {
       List<Integer> dataTypeOrdinals,
       QueryContext context,
       Map<String, Set<String>> deviceMeasurements,
-      Node header)
+      RaftNode header)
       throws TException, InterruptedException {
     AtomicReference<ByteBuffer> result = new AtomicReference<>();
     GenericHandler<ByteBuffer> handler = new GenericHandler<>(client.getNode(), result);
@@ -505,9 +507,10 @@ public class SyncClientAdaptor {
     return handler.getResult(RaftServer.getReadOperationTimeoutMS());
   }
 
-  public static boolean onSnapshotApplied(AsyncDataClient client, Node header, List<Integer> slots)
+  public static boolean onSnapshotApplied(
+      AsyncDataClient client, RaftNode header, List<Integer> slots)
       throws TException, InterruptedException {
-    AtomicReference<Boolean> result = new AtomicReference<>(false);
+    AtomicReference<Boolean> result = new AtomicReference<>();
     GenericHandler<Boolean> handler = new GenericHandler<>(client.getNode(), result);
 
     client.onSnapshotApplied(header, slots, handler);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
index b9b0db4..c7d5088 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
@@ -48,6 +48,8 @@ public class ClusterConfig {
 
   @ClusterConsistent private int replicationNum = 1;
 
+  @ClusterConsistent private int multiRaftFactor = 1;
+
   @ClusterConsistent private String clusterName = "default";
 
   @ClusterConsistent private boolean useAsyncServer = false;
@@ -167,6 +169,12 @@ public class ClusterConfig {
    */
   private long maxReadLogLag = 1000L;
 
+  /**
+   * When a follower tries to sync log with the leader, sync will fail if the log Lag exceeds
+   * maxSyncLogLag.
+   */
+  private long maxSyncLogLag = 100000L;
+
   private boolean openServerRpcPort = false;
 
   /**
@@ -244,6 +252,14 @@ public class ClusterConfig {
     this.replicationNum = replicationNum;
   }
 
+  public int getMultiRaftFactor() {
+    return multiRaftFactor;
+  }
+
+  public void setMultiRaftFactor(int multiRaftFactor) {
+    this.multiRaftFactor = multiRaftFactor;
+  }
+
   void setClusterName(String clusterName) {
     this.clusterName = clusterName;
   }
@@ -464,6 +480,14 @@ public class ClusterConfig {
     this.maxReadLogLag = maxReadLogLag;
   }
 
+  public long getMaxSyncLogLag() {
+    return maxSyncLogLag;
+  }
+
+  public void setMaxSyncLogLag(long maxSyncLogLag) {
+    this.maxSyncLogLag = maxSyncLogLag;
+  }
+
   public String getInternalIp() {
     return internalIp;
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java
index a61a41a..b377b1a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java
@@ -36,6 +36,10 @@ public class ClusterConstant {
 
   public static final int LOG_NUM_IN_BATCH = 100;
 
+  public static final int RETRY_WAIT_TIME_MS = 10;
+
+  public static final int THREAD_POLL_WAIT_TERMINATION_TIME_S = 10;
+
   public static final Node EMPTY_NODE = new Node();
 
   private ClusterConstant() {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java
index 801bbf1..9d983ac 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java
@@ -156,6 +156,11 @@ public class ClusterDescriptor {
             properties.getProperty(
                 "max_concurrent_client_num", String.valueOf(config.getMaxConcurrentClientNum()))));
 
+    config.setMultiRaftFactor(
+        Integer.parseInt(
+            properties.getProperty(
+                "multi_raft_factor", String.valueOf(config.getMultiRaftFactor()))));
+
     config.setReplicationNum(
         Integer.parseInt(
             properties.getProperty(
@@ -293,6 +298,10 @@ public class ClusterDescriptor {
         Long.parseLong(
             properties.getProperty("max_read_log_lag", String.valueOf(config.getMaxReadLogLag()))));
 
+    config.setMaxSyncLogLag(
+        Long.parseLong(
+            properties.getProperty("max_sync_log_lag", String.valueOf(config.getMaxReadLogLag()))));
+
     config.setMaxClientPerNodePerMember(
         Integer.parseInt(
             properties.getProperty(
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java b/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
index 1d2c5ad..24e746d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
@@ -21,13 +21,18 @@ package org.apache.iotdb.cluster.coordinator;
 
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.exception.ChangeMembershipException;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
+import org.apache.iotdb.cluster.exception.UnknownLogTypeException;
 import org.apache.iotdb.cluster.exception.UnsupportedPlanException;
+import org.apache.iotdb.cluster.log.Log;
 import org.apache.iotdb.cluster.metadata.CMManager;
 import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.query.ClusterPlanRouter;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.RaftService;
 import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
@@ -66,6 +71,8 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CountDownLatch;
 
 /** Coordinator of client non-query request */
 public class Coordinator {
@@ -306,6 +313,82 @@ public class Coordinator {
     return status;
   }
 
+  public void sendLogToAllDataGroups(Log log) throws ChangeMembershipException {
+    if (logger.isDebugEnabled()) {
+      logger.debug("Send log {} to all data groups: start", log);
+    }
+
+    Map<PhysicalPlan, PartitionGroup> planGroupMap = router.splitAndRouteChangeMembershipLog(log);
+    List<String> errorCodePartitionGroups = new CopyOnWriteArrayList<>();
+    CountDownLatch counter = new CountDownLatch(planGroupMap.size());
+    for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
+      metaGroupMember
+          .getAppendLogThreadPool()
+          .submit(() -> forwardChangeMembershipPlan(log, entry, errorCodePartitionGroups, counter));
+    }
+    try {
+      counter.await();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new ChangeMembershipException(
+          String.format("Can not wait all data groups to apply %s", log));
+    }
+    if (!errorCodePartitionGroups.isEmpty()) {
+      throw new ChangeMembershipException(
+          String.format("Apply %s failed with status {%s}", log, errorCodePartitionGroups));
+    }
+    if (logger.isDebugEnabled()) {
+      logger.debug("Send log {} to all data groups: end", log);
+    }
+  }
+
+  private void forwardChangeMembershipPlan(
+      Log log,
+      Map.Entry<PhysicalPlan, PartitionGroup> entry,
+      List<String> errorCodePartitionGroups,
+      CountDownLatch counter) {
+    int retryTime = 0;
+    long startTime = System.currentTimeMillis();
+    try {
+      while (true) {
+        if (logger.isDebugEnabled()) {
+          logger.debug(
+              "Send change membership log {} to data group {}, retry time: {}",
+              log,
+              entry.getValue(),
+              retryTime);
+        }
+        try {
+          TSStatus status = forwardToSingleGroup(entry);
+          if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+            if (logger.isDebugEnabled()) {
+              logger.debug(
+                  "Success to send change membership log {} to data group {}",
+                  log,
+                  entry.getValue());
+            }
+            return;
+          }
+          long cost = System.currentTimeMillis() - startTime;
+          if (cost > ClusterDescriptor.getInstance().getConfig().getWriteOperationTimeoutMS()) {
+            errorCodePartitionGroups.add(
+                String.format(
+                    "Forward change membership log %s to data group %s", log, entry.getValue()));
+            return;
+          }
+          Thread.sleep(ClusterConstant.RETRY_WAIT_TIME_MS);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          errorCodePartitionGroups.add(e.getMessage());
+          return;
+        }
+        retryTime++;
+      }
+    } finally {
+      counter.countDown();
+    }
+  }
+
   /** split a plan into several sub-plans, each belongs to only one data group. */
   private Map<PhysicalPlan, PartitionGroup> splitPlan(PhysicalPlan plan)
       throws UnsupportedPlanException, CheckConsistencyException {
@@ -317,10 +400,10 @@ public class Coordinator {
       metaGroupMember.syncLeaderWithConsistencyCheck(true);
       try {
         planGroupMap = router.splitAndRoutePlan(plan);
-      } catch (MetadataException ex) {
+      } catch (MetadataException | UnknownLogTypeException ex) {
         // ignore
       }
-    } catch (MetadataException e) {
+    } catch (MetadataException | UnknownLogTypeException e) {
       logger.error("Cannot route plan {}", plan, e);
     }
     logger.debug("route plan {} with partitionGroup {}", plan, planGroupMap);
@@ -444,9 +527,10 @@ public class Coordinator {
     }
     TSStatus status;
     if (errorCodePartitionGroups.isEmpty()) {
-      status = StatusUtils.OK;
       if (allRedirect) {
-        status = StatusUtils.getStatus(status, endPoint);
+        status = StatusUtils.getStatus(TSStatusCode.NEED_REDIRECTION, endPoint);
+      } else {
+        status = StatusUtils.OK;
       }
     } else {
       status =
@@ -487,7 +571,7 @@ public class Coordinator {
           // and the second dimension is the number of rows per InsertTabletPlan
           totalRowNum = ((InsertMultiTabletPlan) parentPlan).getTabletsSize();
         } else if (parentPlan instanceof CreateMultiTimeSeriesPlan) {
-          totalRowNum = ((CreateMultiTimeSeriesPlan) parentPlan).getPaths().size();
+          totalRowNum = parentPlan.getPaths().size();
         } else if (parentPlan instanceof InsertRowsPlan) {
           totalRowNum = ((InsertRowsPlan) parentPlan).getRowCount();
         }
@@ -669,7 +753,7 @@ public class Coordinator {
    * @param header to determine which DataGroupMember of "receiver" will process the request.
    * @return a TSStatus indicating if the forwarding is successful.
    */
-  private TSStatus forwardDataPlanAsync(PhysicalPlan plan, Node receiver, Node header)
+  private TSStatus forwardDataPlanAsync(PhysicalPlan plan, Node receiver, RaftNode header)
       throws IOException {
     RaftService.AsyncClient client =
         metaGroupMember
@@ -678,9 +762,9 @@ public class Coordinator {
     return this.metaGroupMember.forwardPlanAsync(plan, receiver, header, client);
   }
 
-  private TSStatus forwardDataPlanSync(PhysicalPlan plan, Node receiver, Node header)
+  private TSStatus forwardDataPlanSync(PhysicalPlan plan, Node receiver, RaftNode header)
       throws IOException {
-    RaftService.Client client = null;
+    RaftService.Client client;
     try {
       client =
           metaGroupMember
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/NodeAdditionResult.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/ChangeMembershipException.java
similarity index 72%
copy from cluster/src/main/java/org/apache/iotdb/cluster/partition/NodeAdditionResult.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/ChangeMembershipException.java
index 84dbcff..6ad64ea 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/NodeAdditionResult.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/ChangeMembershipException.java
@@ -17,18 +17,12 @@
  * under the License.
  */
 
-package org.apache.iotdb.cluster.partition;
+package org.apache.iotdb.cluster.exception;
 
-public class NodeAdditionResult {
+/** Raised when add/remove membership log can not be sent to all data groups */
+public class ChangeMembershipException extends Exception {
 
-  /** A new data group headed by the new node. */
-  private PartitionGroup newGroup;
-
-  public PartitionGroup getNewGroup() {
-    return newGroup;
-  }
-
-  public void setNewGroup(PartitionGroup newGroup) {
-    this.newGroup = newGroup;
+  public ChangeMembershipException(String errMsg) {
+    super(String.format("Change membership fails, error message=%s ", errMsg));
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/exception/CheckConsistencyException.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/CheckConsistencyException.java
index 5f3bca0..c872cee 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/exception/CheckConsistencyException.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/CheckConsistencyException.java
@@ -24,8 +24,8 @@ package org.apache.iotdb.cluster.exception;
  */
 public class CheckConsistencyException extends Exception {
 
-  public CheckConsistencyException(String errMag) {
-    super(String.format("check consistency failed, error message=%s ", errMag));
+  public CheckConsistencyException(String errMsg) {
+    super(String.format("check consistency failed, error message=%s ", errMsg));
   }
 
   public static final CheckConsistencyException CHECK_STRONG_CONSISTENCY_EXCEPTION =
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/LogParser.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/LogParser.java
index 82ecbf0..e35943b 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/LogParser.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/LogParser.java
@@ -64,6 +64,10 @@ public class LogParser {
       case ADD_NODE:
         AddNodeLog addNodeLog = new AddNodeLog();
         addNodeLog.deserialize(buffer);
+        if (logger.isDebugEnabled()) {
+          logger.debug(
+              "The last meta log index of log {} is {}", addNodeLog, addNodeLog.getMetaLogIndex());
+        }
         log = addNodeLog;
         break;
       case PHYSICAL_PLAN:
@@ -79,6 +83,12 @@ public class LogParser {
       case REMOVE_NODE:
         RemoveNodeLog removeNodeLog = new RemoveNodeLog();
         removeNodeLog.deserialize(buffer);
+        if (logger.isDebugEnabled()) {
+          logger.debug(
+              "The last meta log index of log {} is {}",
+              removeNodeLog,
+              removeNodeLog.getMetaLogIndex());
+        }
         log = removeNodeLog;
         break;
       case EMPTY_CONTENT:
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/BaseApplier.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/BaseApplier.java
index 52e8baa..ed98803 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/BaseApplier.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/BaseApplier.java
@@ -23,7 +23,7 @@ import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.log.LogApplier;
 import org.apache.iotdb.cluster.metadata.MetaPuller;
 import org.apache.iotdb.cluster.query.ClusterPlanExecutor;
-import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.db.exception.BatchProcessException;
@@ -169,7 +169,7 @@ abstract class BaseApplier implements LogApplier {
    * @param ignoredGroup do not pull schema from the group to avoid backward dependency
    * @throws QueryProcessException
    */
-  private void pullTimeseriesSchema(InsertPlan plan, Node ignoredGroup)
+  private void pullTimeseriesSchema(InsertPlan plan, RaftNode ignoredGroup)
       throws QueryProcessException {
     try {
       PartialPath path = plan.getPrefixPath();
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/DataLogApplier.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/DataLogApplier.java
index 542e422..d68d7c5 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/DataLogApplier.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/DataLogApplier.java
@@ -19,12 +19,12 @@
 
 package org.apache.iotdb.cluster.log.applier;
 
-import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.log.Log;
+import org.apache.iotdb.cluster.log.logtypes.AddNodeLog;
 import org.apache.iotdb.cluster.log.logtypes.CloseFileLog;
 import org.apache.iotdb.cluster.log.logtypes.PhysicalPlanLog;
-import org.apache.iotdb.cluster.partition.slot.SlotPartitionTable;
+import org.apache.iotdb.cluster.log.logtypes.RemoveNodeLog;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.cluster.utils.IOUtils;
@@ -64,7 +64,19 @@ public class DataLogApplier extends BaseApplier {
     logger.debug("DataMember [{}] start applying Log {}", dataGroupMember.getName(), log);
 
     try {
-      if (log instanceof PhysicalPlanLog) {
+      if (log instanceof AddNodeLog) {
+        metaGroupMember
+            .getDataClusterServer()
+            .preAddNodeForDataGroup((AddNodeLog) log, dataGroupMember);
+        dataGroupMember.setAndSaveLastAppliedPartitionTableVersion(
+            ((AddNodeLog) log).getMetaLogIndex());
+      } else if (log instanceof RemoveNodeLog) {
+        metaGroupMember
+            .getDataClusterServer()
+            .preRemoveNodeForDataGroup((RemoveNodeLog) log, dataGroupMember);
+        dataGroupMember.setAndSaveLastAppliedPartitionTableVersion(
+            ((RemoveNodeLog) log).getMetaLogIndex());
+      } else if (log instanceof PhysicalPlanLog) {
         PhysicalPlanLog physicalPlanLog = (PhysicalPlanLog) log;
         PhysicalPlan plan = physicalPlanLog.getPlan();
         if (plan instanceof InsertMultiTabletPlan) {
@@ -114,11 +126,8 @@ public class DataLogApplier extends BaseApplier {
 
   private void applyInsert(InsertPlan plan)
       throws StorageGroupNotSetException, QueryProcessException, StorageEngineException {
-    // check if the corresponding slot is being pulled
-    PartialPath sg;
-    long time = plan.getMinTime();
     try {
-      sg = IoTDB.metaManager.getStorageGroupPath(plan.getPrefixPath());
+      IoTDB.metaManager.getStorageGroupPath(plan.getPrefixPath());
     } catch (StorageGroupNotSetException e) {
       // the sg may not exist because the node does not catch up with the leader, retry after
       // synchronization
@@ -127,13 +136,7 @@ public class DataLogApplier extends BaseApplier {
       } catch (CheckConsistencyException ce) {
         throw new QueryProcessException(ce.getMessage());
       }
-      sg = IoTDB.metaManager.getStorageGroupPath(plan.getPrefixPath());
     }
-    int slotId =
-        SlotPartitionTable.getSlotStrategy()
-            .calculateSlotByTime(sg.getFullPath(), time, ClusterConstant.SLOT_NUM);
-    // the slot may not be writable because it is pulling file versions, wait until it is done
-    dataGroupMember.getSlotManager().waitSlotForWrite(slotId);
     applyPhysicalPlan(plan, dataGroupMember);
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/MetaLogApplier.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/MetaLogApplier.java
index 0556ed7..1c501f8 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/MetaLogApplier.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/MetaLogApplier.java
@@ -19,15 +19,14 @@
 
 package org.apache.iotdb.cluster.log.applier;
 
+import org.apache.iotdb.cluster.exception.ChangeMembershipException;
 import org.apache.iotdb.cluster.log.Log;
 import org.apache.iotdb.cluster.log.logtypes.AddNodeLog;
+import org.apache.iotdb.cluster.log.logtypes.EmptyContentLog;
 import org.apache.iotdb.cluster.log.logtypes.PhysicalPlanLog;
 import org.apache.iotdb.cluster.log.logtypes.RemoveNodeLog;
-import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.server.NodeCharacter;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,22 +47,44 @@ public class MetaLogApplier extends BaseApplier {
     try {
       logger.debug("MetaMember [{}] starts applying Log {}", metaGroupMember.getName(), log);
       if (log instanceof AddNodeLog) {
-        AddNodeLog addNodeLog = (AddNodeLog) log;
-        Node newNode = addNodeLog.getNewNode();
-        member.applyAddNode(newNode);
+        applyAddNodeLog((AddNodeLog) log);
       } else if (log instanceof PhysicalPlanLog) {
         applyPhysicalPlan(((PhysicalPlanLog) log).getPlan(), null);
       } else if (log instanceof RemoveNodeLog) {
-        RemoveNodeLog removeNodeLog = ((RemoveNodeLog) log);
-        member.applyRemoveNode(removeNodeLog.getRemovedNode());
+        applyRemoveNodeLog((RemoveNodeLog) log);
+      } else if (log instanceof EmptyContentLog) {
+        // Do nothing
       } else {
         logger.error("Unsupported log: {} {}", log.getClass().getName(), log);
       }
-    } catch (StorageEngineException | StorageGroupNotSetException | QueryProcessException e) {
+    } catch (Exception e) {
       logger.debug("Exception occurred when executing {}", log, e);
       log.setException(e);
     } finally {
       log.setApplied(true);
     }
   }
+
+  private void applyAddNodeLog(AddNodeLog log) throws ChangeMembershipException {
+    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
+      logger.info("Ignore previous change membership log");
+      // ignore previous change membership log
+      return;
+    }
+    if (metaGroupMember.getCharacter() == NodeCharacter.LEADER) {
+      metaGroupMember.getCoordinator().sendLogToAllDataGroups(log);
+    }
+    member.applyAddNode(log);
+  }
+
+  private void applyRemoveNodeLog(RemoveNodeLog log) throws ChangeMembershipException {
+    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
+      // ignore previous change membership log
+      return;
+    }
+    if (metaGroupMember.getCharacter() == NodeCharacter.LEADER) {
+      metaGroupMember.getCoordinator().sendLogToAllDataGroups(log);
+    }
+    member.applyRemoveNode(log);
+  }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/CatchUpTask.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/CatchUpTask.java
index 1472b12..8b868b6 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/CatchUpTask.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/CatchUpTask.java
@@ -57,9 +57,11 @@ public class CatchUpTask implements Runnable {
   private long lastLogIndex;
   private boolean abort;
   private String name;
+  private int raftId;
 
-  public CatchUpTask(Node node, Peer peer, RaftMember raftMember, long lastLogIdx) {
+  public CatchUpTask(Node node, int raftId, Peer peer, RaftMember raftMember, long lastLogIdx) {
     this.node = node;
+    this.raftId = raftId;
     this.peer = peer;
     this.raftMember = raftMember;
     this.logs = Collections.emptyList();
@@ -280,8 +282,8 @@ public class CatchUpTask implements Runnable {
       if (client == null) {
         return false;
       }
-      Node header = raftMember.getHeader();
-      matched = SyncClientAdaptor.matchTerm(client, node, logIndex, logTerm, header);
+      matched =
+          SyncClientAdaptor.matchTerm(client, node, logIndex, logTerm, raftMember.getHeader());
     } else {
       Client client = raftMember.getSyncClient(node);
       if (client == null) {
@@ -357,11 +359,12 @@ public class CatchUpTask implements Runnable {
         doSnapshot();
         // snapshot may overlap with logs
         removeSnapshotLogs();
-        SnapshotCatchUpTask task = new SnapshotCatchUpTask(logs, snapshot, node, raftMember);
+        SnapshotCatchUpTask task =
+            new SnapshotCatchUpTask(logs, snapshot, node, raftId, raftMember);
         catchUpSucceeded = task.call();
       } else {
         logger.info("{}: performing a log catch-up to {}", raftMember.getName(), node);
-        LogCatchUpTask task = new LogCatchUpTask(logs, node, raftMember);
+        LogCatchUpTask task = new LogCatchUpTask(logs, node, raftId, raftMember);
         catchUpSucceeded = task.call();
       }
       if (catchUpSucceeded) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/LogCatchUpTask.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/LogCatchUpTask.java
index dcb287d..99b3e4d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/LogCatchUpTask.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/LogCatchUpTask.java
@@ -60,17 +60,20 @@ public class LogCatchUpTask implements Callable<Boolean> {
   private List<Log> logs;
   private boolean useBatch = ClusterDescriptor.getInstance().getConfig().isUseBatchInLogCatchUp();
   boolean abort = false;
+  private int raftId;
 
-  LogCatchUpTask(List<Log> logs, Node node, RaftMember raftMember) {
+  LogCatchUpTask(List<Log> logs, Node node, int raftId, RaftMember raftMember) {
     this.logs = logs;
     this.node = node;
+    this.raftId = raftId;
     this.raftMember = raftMember;
   }
 
   @TestOnly
-  LogCatchUpTask(List<Log> logs, Node node, RaftMember raftMember, boolean useBatch) {
+  LogCatchUpTask(List<Log> logs, Node node, int raftId, RaftMember raftMember, boolean useBatch) {
     this.logs = logs;
     this.node = node;
+    this.raftId = raftId;
     this.raftMember = raftMember;
     this.useBatch = useBatch;
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/SnapshotCatchUpTask.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/SnapshotCatchUpTask.java
index 4ed48fe..b11f886 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/SnapshotCatchUpTask.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/SnapshotCatchUpTask.java
@@ -53,8 +53,9 @@ public class SnapshotCatchUpTask extends LogCatchUpTask implements Callable<Bool
       ClusterDescriptor.getInstance().getConfig().getCatchUpTimeoutMS();
   private Snapshot snapshot;
 
-  SnapshotCatchUpTask(List<Log> logs, Snapshot snapshot, Node node, RaftMember raftMember) {
-    super(logs, node, raftMember);
+  SnapshotCatchUpTask(
+      List<Log> logs, Snapshot snapshot, Node node, int raftId, RaftMember raftMember) {
+    super(logs, node, raftId, raftMember);
     this.snapshot = snapshot;
   }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/AddNodeLog.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/AddNodeLog.java
index ea41685..83c72a8 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/AddNodeLog.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/AddNodeLog.java
@@ -32,16 +32,44 @@ import java.util.Objects;
 /** AddNodeLog records the operation of adding a node into this cluster. */
 public class AddNodeLog extends Log {
 
+  private ByteBuffer partitionTable;
+
   private Node newNode;
 
-  public Node getNewNode() {
-    return newNode;
+  private long metaLogIndex;
+
+  public AddNodeLog(ByteBuffer partitionTable, Node newNode) {
+    this.partitionTable = partitionTable;
+    this.newNode = newNode;
+  }
+
+  public AddNodeLog() {}
+
+  public long getMetaLogIndex() {
+    return metaLogIndex;
+  }
+
+  public void setMetaLogIndex(long metaLogIndex) {
+    this.metaLogIndex = metaLogIndex;
+  }
+
+  public void setPartitionTable(ByteBuffer partitionTable) {
+    this.partitionTable = partitionTable;
   }
 
   public void setNewNode(Node newNode) {
     this.newNode = newNode;
   }
 
+  public Node getNewNode() {
+    return newNode;
+  }
+
+  public ByteBuffer getPartitionTable() {
+    partitionTable.rewind();
+    return partitionTable;
+  }
+
   @Override
   public ByteBuffer serialize() {
     ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
@@ -49,8 +77,12 @@ public class AddNodeLog extends Log {
       dataOutputStream.writeByte(Types.ADD_NODE.ordinal());
       dataOutputStream.writeLong(getCurrLogIndex());
       dataOutputStream.writeLong(getCurrLogTerm());
+      dataOutputStream.writeLong(getMetaLogIndex());
 
       NodeSerializeUtils.serialize(newNode, dataOutputStream);
+
+      dataOutputStream.writeInt(partitionTable.array().length);
+      dataOutputStream.write(partitionTable.array());
     } catch (IOException e) {
       // ignored
     }
@@ -65,9 +97,15 @@ public class AddNodeLog extends Log {
     // ipLength(int), inBytes(byte[]), port(int), identifier(int), dataPort(int)
     setCurrLogIndex(buffer.getLong());
     setCurrLogTerm(buffer.getLong());
+    setMetaLogIndex(buffer.getLong());
 
     newNode = new Node();
     NodeSerializeUtils.deserialize(newNode, buffer);
+
+    int len = buffer.getInt();
+    byte[] data = new byte[len];
+    System.arraycopy(buffer.array(), buffer.position(), data, 0, len);
+    partitionTable = ByteBuffer.wrap(data);
   }
 
   @Override
@@ -82,11 +120,17 @@ public class AddNodeLog extends Log {
       return false;
     }
     AddNodeLog that = (AddNodeLog) o;
-    return Objects.equals(newNode, that.newNode);
+    return Objects.equals(newNode, that.newNode)
+        && Objects.equals(partitionTable, that.partitionTable);
+  }
+
+  @Override
+  public String toString() {
+    return "AddNodeLog{" + "newNode=" + newNode.toString() + '}';
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(super.hashCode(), newNode);
+    return Objects.hash(super.hashCode(), newNode, partitionTable);
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/RemoveNodeLog.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/RemoveNodeLog.java
index 8a84023..ea06cfe 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/RemoveNodeLog.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/RemoveNodeLog.java
@@ -31,8 +31,36 @@ import java.util.Objects;
 
 public class RemoveNodeLog extends Log {
 
+  private ByteBuffer partitionTable;
+
   private Node removedNode;
 
+  private long metaLogIndex;
+
+  public RemoveNodeLog(ByteBuffer partitionTable, Node removedNode) {
+    this.partitionTable = partitionTable;
+    this.removedNode = removedNode;
+  }
+
+  public RemoveNodeLog() {}
+
+  public long getMetaLogIndex() {
+    return metaLogIndex;
+  }
+
+  public void setMetaLogIndex(long metaLogIndex) {
+    this.metaLogIndex = metaLogIndex;
+  }
+
+  public ByteBuffer getPartitionTable() {
+    partitionTable.rewind();
+    return partitionTable;
+  }
+
+  public void setPartitionTable(ByteBuffer partitionTable) {
+    this.partitionTable = partitionTable;
+  }
+
   @Override
   public ByteBuffer serialize() {
     ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
@@ -40,8 +68,12 @@ public class RemoveNodeLog extends Log {
       dataOutputStream.writeByte(Types.REMOVE_NODE.ordinal());
       dataOutputStream.writeLong(getCurrLogIndex());
       dataOutputStream.writeLong(getCurrLogTerm());
+      dataOutputStream.writeLong(getMetaLogIndex());
 
       NodeSerializeUtils.serialize(removedNode, dataOutputStream);
+
+      dataOutputStream.writeInt(partitionTable.array().length);
+      dataOutputStream.write(partitionTable.array());
     } catch (IOException e) {
       // ignored
     }
@@ -52,9 +84,15 @@ public class RemoveNodeLog extends Log {
   public void deserialize(ByteBuffer buffer) {
     setCurrLogIndex(buffer.getLong());
     setCurrLogTerm(buffer.getLong());
+    setMetaLogIndex(buffer.getLong());
 
     removedNode = new Node();
     NodeSerializeUtils.deserialize(removedNode, buffer);
+
+    int len = buffer.getInt();
+    byte[] data = new byte[len];
+    System.arraycopy(buffer.array(), buffer.position(), data, 0, len);
+    partitionTable = ByteBuffer.wrap(data);
   }
 
   public Node getRemovedNode() {
@@ -77,11 +115,17 @@ public class RemoveNodeLog extends Log {
       return false;
     }
     RemoveNodeLog that = (RemoveNodeLog) o;
-    return Objects.equals(removedNode, that.removedNode);
+    return Objects.equals(removedNode, that.removedNode)
+        && Objects.equals(partitionTable, that.partitionTable);
+  }
+
+  @Override
+  public String toString() {
+    return "RemoveNodeLog{" + "removedNode=" + removedNode.toString() + '}';
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(super.hashCode(), removedNode);
+    return Objects.hash(super.hashCode(), removedNode, partitionTable);
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/FilePartitionedSnapshotLogManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/FilePartitionedSnapshotLogManager.java
index 89fe8ad..39dbab8 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/FilePartitionedSnapshotLogManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/FilePartitionedSnapshotLogManager.java
@@ -19,8 +19,12 @@
 
 package org.apache.iotdb.cluster.log.manage;
 
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.exception.EntryCompactedException;
+import org.apache.iotdb.cluster.log.Log;
 import org.apache.iotdb.cluster.log.LogApplier;
+import org.apache.iotdb.cluster.log.logtypes.AddNodeLog;
+import org.apache.iotdb.cluster.log.logtypes.RemoveNodeLog;
 import org.apache.iotdb.cluster.log.snapshot.FileSnapshot;
 import org.apache.iotdb.cluster.log.snapshot.FileSnapshot.Factory;
 import org.apache.iotdb.cluster.partition.PartitionTable;
@@ -64,7 +68,7 @@ public class FilePartitionedSnapshotLogManager extends PartitionedSnapshotLogMan
   }
 
   /** send FlushPlan to all nodes in one dataGroup */
-  private void syncFlushAllProcessor() {
+  private void syncFlushAllProcessor(List<Integer> requiredSlots, boolean needLeader) {
     logger.info("{}: Start flush all storage group processor in one data group", getName());
     Map<String, List<Pair<Long, Boolean>>> storageGroupPartitions =
         StorageEngine.getInstance().getWorkingStorageGroupPartitions();
@@ -72,12 +76,19 @@ public class FilePartitionedSnapshotLogManager extends PartitionedSnapshotLogMan
       logger.info("{}: no need to flush processor", getName());
       return;
     }
-    dataGroupMember.flushFileWhenDoSnapshot(storageGroupPartitions);
+    dataGroupMember.flushFileWhenDoSnapshot(storageGroupPartitions, requiredSlots, needLeader);
   }
 
   @Override
   @SuppressWarnings("java:S1135") // ignore todos
   public void takeSnapshot() throws IOException {
+    takeSnapshotForSpecificSlots(
+        ((SlotPartitionTable) partitionTable).getNodeSlots(dataGroupMember.getHeader()), true);
+  }
+
+  @Override
+  public void takeSnapshotForSpecificSlots(List<Integer> requiredSlots, boolean needLeader)
+      throws IOException {
     try {
       logger.info("{}: Taking snapshots, flushing IoTDB", getName());
       // record current commit index and prevent further logs from being applied, so the
@@ -86,14 +97,14 @@ public class FilePartitionedSnapshotLogManager extends PartitionedSnapshotLogMan
       // wait until all logs before BlockAppliedCommitIndex are applied
       super.takeSnapshot();
       // flush data to disk so that the disk files will represent a complete state
-      syncFlushAllProcessor();
+      syncFlushAllProcessor(requiredSlots, needLeader);
       logger.info("{}: Taking snapshots, IoTDB is flushed", getName());
       // TODO-cluster https://issues.apache.org/jira/browse/IOTDB-820
       synchronized (this) {
-        collectTimeseriesSchemas();
+        collectTimeseriesSchemas(requiredSlots);
         snapshotLastLogIndex = getBlockAppliedCommitIndex();
         snapshotLastLogTerm = getTerm(snapshotLastLogIndex);
-        collectTsFilesAndFillTimeseriesSchemas();
+        collectTsFilesAndFillTimeseriesSchemas(requiredSlots);
         logger.info("{}: Snapshot is taken", getName());
       }
     } catch (EntryCompactedException e) {
@@ -112,9 +123,10 @@ public class FilePartitionedSnapshotLogManager extends PartitionedSnapshotLogMan
    *
    * @throws IOException
    */
-  private void collectTsFilesAndFillTimeseriesSchemas() throws IOException {
+  private void collectTsFilesAndFillTimeseriesSchemas(List<Integer> requiredSlots)
+      throws IOException {
     // 1.collect tsfile
-    collectTsFiles();
+    collectTsFiles(requiredSlots);
 
     // 2.register the measurement
     boolean slotExistsInPartition;
@@ -138,7 +150,7 @@ public class FilePartitionedSnapshotLogManager extends PartitionedSnapshotLogMan
     }
   }
 
-  private void collectTsFiles() throws IOException {
+  private void collectTsFiles(List<Integer> requiredSlots) throws IOException {
     slotSnapshots.clear();
     Map<PartialPath, Map<Long, List<TsFileResource>>> allClosedStorageGroupTsFile =
         StorageEngine.getInstance().getAllClosedStorageGroupTsFile();
@@ -151,13 +163,14 @@ public class FilePartitionedSnapshotLogManager extends PartitionedSnapshotLogMan
       for (Entry<Long, List<TsFileResource>> storageGroupFiles : storageGroupsFiles.entrySet()) {
         Long partitionNum = storageGroupFiles.getKey();
         List<TsFileResource> resourceList = storageGroupFiles.getValue();
-        if (!collectTsFiles(partitionNum, resourceList, storageGroupName, createdHardlinks)) {
+        if (!collectTsFiles(
+            partitionNum, resourceList, storageGroupName, createdHardlinks, requiredSlots)) {
           // some file is deleted during the collecting, clean created hardlinks and restart
           // from the beginning
           for (TsFileResource createdHardlink : createdHardlinks) {
             createdHardlink.remove();
           }
-          collectTsFiles();
+          collectTsFiles(requiredSlots);
           return;
         }
       }
@@ -179,14 +192,16 @@ public class FilePartitionedSnapshotLogManager extends PartitionedSnapshotLogMan
       Long partitionNum,
       List<TsFileResource> resourceList,
       PartialPath storageGroupName,
-      List<TsFileResource> createdHardlinks)
+      List<TsFileResource> createdHardlinks,
+      List<Integer> requiredSlots)
       throws IOException {
     int slotNum =
         SlotPartitionTable.getSlotStrategy()
             .calculateSlotByPartitionNum(
-                storageGroupName.getFullPath(),
-                partitionNum,
-                ((SlotPartitionTable) partitionTable).getTotalSlotNumbers());
+                storageGroupName.getFullPath(), partitionNum, ClusterConstant.SLOT_NUM);
+    if (!requiredSlots.contains(slotNum)) {
+      return true;
+    }
     FileSnapshot snapshot = slotSnapshots.computeIfAbsent(slotNum, s -> new FileSnapshot());
     for (TsFileResource tsFileResource : resourceList) {
       TsFileResource hardlink = tsFileResource.createHardlink();
@@ -223,4 +238,14 @@ public class FilePartitionedSnapshotLogManager extends PartitionedSnapshotLogMan
     }
     return true;
   }
+
+  @Override
+  public long append(Log entry) {
+    long res = super.append(entry);
+    // For data group, it's necessary to apply remove/add log immediately after append
+    if (entry instanceof AddNodeLog || entry instanceof RemoveNodeLog) {
+      applyEntry(entry);
+    }
+    return res;
+  }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/PartitionedSnapshotLogManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/PartitionedSnapshotLogManager.java
index 0123cc1..d2fd280 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/PartitionedSnapshotLogManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/PartitionedSnapshotLogManager.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.cluster.log.manage;
 
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.log.LogApplier;
 import org.apache.iotdb.cluster.log.Snapshot;
 import org.apache.iotdb.cluster.log.manage.serializable.SyncLogDequeSerializer;
@@ -36,6 +37,7 @@ import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -74,6 +76,9 @@ public abstract class PartitionedSnapshotLogManager<T extends Snapshot> extends
     this.dataGroupMember = dataGroupMember;
   }
 
+  public void takeSnapshotForSpecificSlots(List<Integer> requiredSlots, boolean needLeader)
+      throws IOException {}
+
   @Override
   public Snapshot getSnapshot(long minIndex) {
     // copy snapshots
@@ -89,30 +94,22 @@ public abstract class PartitionedSnapshotLogManager<T extends Snapshot> extends
     }
   }
 
-  void collectTimeseriesSchemas() {
+  void collectTimeseriesSchemas(List<Integer> requiredSlots) {
     slotTimeseries.clear();
     List<StorageGroupMNode> allSgNodes = IoTDB.metaManager.getAllStorageGroupNodes();
-    boolean slotExistsInPartition;
-    List<Integer> slots = null;
-    if (dataGroupMember.getMetaGroupMember() != null) {
-      slots =
-          ((SlotPartitionTable) dataGroupMember.getMetaGroupMember().getPartitionTable())
-              .getNodeSlots(dataGroupMember.getHeader());
-    }
     for (MNode sgNode : allSgNodes) {
       String storageGroupName = sgNode.getFullPath();
       int slot =
           SlotPartitionTable.getSlotStrategy()
-              .calculateSlotByTime(
-                  storageGroupName, 0, ((SlotPartitionTable) partitionTable).getTotalSlotNumbers());
-      slotExistsInPartition = slots == null || slots.contains(slot);
+              .calculateSlotByTime(storageGroupName, 0, ClusterConstant.SLOT_NUM);
 
-      if (slotExistsInPartition) {
-        Collection<TimeseriesSchema> schemas =
-            slotTimeseries.computeIfAbsent(slot, s -> new HashSet<>());
-        IoTDB.metaManager.collectTimeseriesSchema(sgNode, schemas);
-        logger.debug("{}: {} timeseries are snapshot in slot {}", getName(), schemas.size(), slot);
+      if (!requiredSlots.contains(slot)) {
+        continue;
       }
+      Collection<TimeseriesSchema> schemas =
+          slotTimeseries.computeIfAbsent(slot, s -> new HashSet<>());
+      IoTDB.metaManager.collectTimeseriesSchema(sgNode, schemas);
+      logger.debug("{}: {} timeseries are snapshot in slot {}", getName(), schemas.size(), slot);
     }
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
index 446eefc..322b541 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
@@ -80,9 +80,9 @@ public abstract class RaftLogManager {
    * The committed log whose index is larger than blockAppliedCommitIndex will be blocked. if
    * blockAppliedCommitIndex < 0(default is -1), will not block any operation.
    */
-  private volatile long blockAppliedCommitIndex;
+  protected volatile long blockAppliedCommitIndex;
 
-  private LogApplier logApplier;
+  protected LogApplier logApplier;
 
   /** to distinguish managers of different members */
   private String name;
@@ -110,7 +110,7 @@ public abstract class RaftLogManager {
    */
   private final Object[] logUpdateConditions = new Object[1024];
 
-  private List<Log> blockedUnappliedLogList;
+  protected List<Log> blockedUnappliedLogList;
 
   protected RaftLogManager(StableEntryManager stableEntryManager, LogApplier applier, String name) {
     this.logApplier = applier;
@@ -677,16 +677,26 @@ public abstract class RaftLogManager {
    */
   void applyEntries(List<Log> entries) {
     for (Log entry : entries) {
-      if (blockAppliedCommitIndex > 0 && entry.getCurrLogIndex() > blockAppliedCommitIndex) {
-        blockedUnappliedLogList.add(entry);
-        continue;
-      }
-      try {
-        logApplier.apply(entry);
-      } catch (Exception e) {
-        entry.setException(e);
-        entry.setApplied(true);
-      }
+      applyEntry(entry);
+    }
+  }
+
+  public void applyEntry(Log entry) {
+    // For add/remove logs in data groups, this log will be applied immediately when it is
+    // appended to the raft log.
+    // In this case, it will apply a log that has been applied.
+    if (entry.isApplied()) {
+      return;
+    }
+    if (blockAppliedCommitIndex > 0 && entry.getCurrLogIndex() > blockAppliedCommitIndex) {
+      blockedUnappliedLogList.add(entry);
+      return;
+    }
+    try {
+      logApplier.apply(entry);
+    } catch (Exception e) {
+      entry.setException(e);
+      entry.setApplied(true);
     }
   }
 
@@ -1014,4 +1024,8 @@ public abstract class RaftLogManager {
   public long getBlockAppliedCommitIndex() {
     return blockAppliedCommitIndex;
   }
+
+  public RaftLogManager(LogApplier logApplier) {
+    this.logApplier = logApplier;
+  }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
index 145c7c2..a67cd50 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
@@ -24,7 +24,6 @@ import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
-import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.exception.PullFileException;
 import org.apache.iotdb.cluster.exception.SnapshotInstallationException;
 import org.apache.iotdb.cluster.log.Snapshot;
@@ -195,64 +194,70 @@ public class FileSnapshot extends Snapshot implements TimeseriesSchemaSnapshot {
     }
 
     @Override
-    public void install(FileSnapshot snapshot, int slot) throws SnapshotInstallationException {
+    public void install(FileSnapshot snapshot, int slot, boolean isDataMigration)
+        throws SnapshotInstallationException {
       try {
         logger.info("Starting to install a snapshot {} into slot[{}]", snapshot, slot);
         installFileSnapshotSchema(snapshot);
         logger.info("Schemas in snapshot are registered");
-
-        SlotStatus status = slotManager.getStatus(slot);
-        if (status == SlotStatus.PULLING) {
-          // as the schemas are set, writes can proceed
-          slotManager.setToPullingWritable(slot);
-          logger.debug("{}: slot {} is now pulling writable", name, slot);
+        if (isDataMigration) {
+          SlotStatus status = slotManager.getStatus(slot);
+          if (status == SlotStatus.PULLING) {
+            // as the schemas are set, writes can proceed
+            slotManager.setToPullingWritable(slot);
+            logger.debug("{}: slot {} is now pulling writable", name, slot);
+          }
         }
-
-        installFileSnapshotFiles(snapshot, slot);
+        installFileSnapshotFiles(snapshot, slot, isDataMigration);
       } catch (PullFileException e) {
         throw new SnapshotInstallationException(e);
       }
     }
 
     @Override
-    public void install(Map<Integer, FileSnapshot> snapshotMap)
+    public void install(Map<Integer, FileSnapshot> snapshotMap, boolean isDataMigration)
         throws SnapshotInstallationException {
       logger.info("Starting to install snapshots {}", snapshotMap);
-      installSnapshot(snapshotMap);
+      installSnapshot(snapshotMap, isDataMigration);
     }
 
-    private void installSnapshot(Map<Integer, FileSnapshot> snapshotMap)
+    private void installSnapshot(Map<Integer, FileSnapshot> snapshotMap, boolean isDataMigration)
         throws SnapshotInstallationException {
-      // ensure StorageGroups are synchronized
-      try {
-        dataGroupMember.getMetaGroupMember().syncLeaderWithConsistencyCheck(true);
-      } catch (CheckConsistencyException e) {
-        throw new SnapshotInstallationException(e);
-      }
-
-      for (FileSnapshot value : snapshotMap.values()) {
-        installFileSnapshotSchema(value);
-      }
-
+      // In data migration, meta group member other than new node does not need to synchronize the
+      // leader, because data migration must be carried out after meta group applied add/remove node
+      // log.
+      dataGroupMember
+          .getMetaGroupMember()
+          .syncLocalApply(
+              dataGroupMember.getMetaGroupMember().getPartitionTable().getLastMetaLogIndex() - 1,
+              false);
       for (Entry<Integer, FileSnapshot> integerSnapshotEntry : snapshotMap.entrySet()) {
         Integer slot = integerSnapshotEntry.getKey();
-        SlotStatus status = slotManager.getStatus(slot);
-        if (status == SlotStatus.PULLING) {
-          // as schemas are set, writes can proceed
-          slotManager.setToPullingWritable(slot);
-          logger.debug("{}: slot {} is now pulling writable", name, slot);
+        FileSnapshot snapshot = integerSnapshotEntry.getValue();
+        installFileSnapshotSchema(snapshot);
+        if (isDataMigration) {
+          SlotStatus status = slotManager.getStatus(slot);
+          if (status == SlotStatus.PULLING) {
+            // as schemas are set, writes can proceed
+            slotManager.setToPullingWritable(slot, false);
+            logger.debug("{}: slot {} is now pulling writable", name, slot);
+          }
         }
       }
+      if (isDataMigration) {
+        slotManager.save();
+      }
 
       for (Entry<Integer, FileSnapshot> integerSnapshotEntry : snapshotMap.entrySet()) {
         Integer slot = integerSnapshotEntry.getKey();
         FileSnapshot snapshot = integerSnapshotEntry.getValue();
         try {
-          installFileSnapshotFiles(snapshot, slot);
+          installFileSnapshotFiles(snapshot, slot, isDataMigration);
         } catch (PullFileException e) {
           throw new SnapshotInstallationException(e);
         }
       }
+      slotManager.save();
     }
 
     private void installFileSnapshotSchema(FileSnapshot snapshot) {
@@ -263,7 +268,7 @@ public class FileSnapshot extends Snapshot implements TimeseriesSchemaSnapshot {
       }
     }
 
-    private void installFileSnapshotFiles(FileSnapshot snapshot, int slot)
+    private void installFileSnapshotFiles(FileSnapshot snapshot, int slot, boolean isDataMigration)
         throws PullFileException {
       List<RemoteTsFileResource> remoteTsFileResources = snapshot.getDataFiles();
       // pull file
@@ -274,18 +279,28 @@ public class FileSnapshot extends Snapshot implements TimeseriesSchemaSnapshot {
         logger.info(
             "Pulling {}/{} files, current: {}", i + 1, remoteTsFileResources.size(), resource);
         try {
-          if (!isFileAlreadyPulled(resource)) {
+          if (isDataMigration) {
+            // This means that the minimum plan index and maximum plan index of some files are the
+            // same,
+            // so the logic of judging index coincidence needs to remove the case of equal
+            resource.setMinPlanIndex(dataGroupMember.getLogManager().getLastLogIndex());
+            resource.setMaxPlanIndex(dataGroupMember.getLogManager().getLastLogIndex());
             loadRemoteFile(resource);
           } else {
-            // notify the snapshot provider to remove the hardlink
-            removeRemoteHardLink(resource);
+            if (!isFileAlreadyPulled(resource)) {
+              loadRemoteFile(resource);
+            } else {
+              // notify the snapshot provider to remove the hardlink
+              removeRemoteHardLink(resource);
+            }
           }
         } catch (IllegalPathException e) {
           throw new PullFileException(resource.getTsFilePath(), resource.getSource(), e);
         }
       }
+
       // all files are loaded, the slot can be queried without accessing the previous holder
-      slotManager.setToNull(slot);
+      slotManager.setToNull(slot, !isDataMigration);
       logger.info("{}: slot {} is ready", name, slot);
     }
 
@@ -393,8 +408,6 @@ public class FileSnapshot extends Snapshot implements TimeseriesSchemaSnapshot {
       // you can see FilePathUtils.splitTsFilePath() method for details.
       PartialPath storageGroupName =
           new PartialPath(FilePathUtils.getLogicalStorageGroupName(resource));
-      File remoteModFile =
-          new File(resource.getTsFile().getAbsoluteFile() + ModificationFile.FILE_SUFFIX);
       try {
         StorageEngine.getInstance().getProcessor(storageGroupName).loadNewTsFile(resource);
         if (resource.isPlanRangeUnique()) {
@@ -408,22 +421,6 @@ public class FileSnapshot extends Snapshot implements TimeseriesSchemaSnapshot {
         logger.error("{}: Cannot load remote file {} into storage group", name, resource, e);
         return;
       }
-      if (remoteModFile.exists()) {
-        // when successfully loaded, the filepath of the resource will be changed to the IoTDB data
-        // dir, so we can add a suffix to find the old modification file.
-        File localModFile =
-            new File(resource.getTsFile().getAbsoluteFile() + ModificationFile.FILE_SUFFIX);
-        try {
-          Files.deleteIfExists(localModFile.toPath());
-        } catch (IOException e) {
-          logger.warn("Cannot delete localModFile {}", localModFile, e);
-        }
-        if (!remoteModFile.renameTo(localModFile)) {
-          logger.warn("Cannot rename remoteModFile {}", remoteModFile);
-        }
-        // ModFile will be updated during the next call to `getModFile`
-        resource.setModFile(null);
-      }
       resource.setRemote(false);
     }
 
@@ -604,7 +601,7 @@ public class FileSnapshot extends Snapshot implements TimeseriesSchemaSnapshot {
   public void truncateBefore(long minIndex) {
     dataFiles.removeIf(
         res -> {
-          boolean toBeTruncated = res.getMaxPlanIndex() <= minIndex;
+          boolean toBeTruncated = res.getMaxPlanIndex() < minIndex;
           if (toBeTruncated) {
             // also remove the hardlink
             res.remove();
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/MetaSimpleSnapshot.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/MetaSimpleSnapshot.java
index 47720b3..1713426 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/MetaSimpleSnapshot.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/MetaSimpleSnapshot.java
@@ -178,12 +178,12 @@ public class MetaSimpleSnapshot extends Snapshot {
     }
 
     @Override
-    public void install(MetaSimpleSnapshot snapshot, int slot) {
+    public void install(MetaSimpleSnapshot snapshot, int slot, boolean isDataMigration) {
       installSnapshot(snapshot);
     }
 
     @Override
-    public void install(Map<Integer, MetaSimpleSnapshot> snapshotMap) {
+    public void install(Map<Integer, MetaSimpleSnapshot> snapshotMap, boolean isDataMigration) {
       throw new UnsupportedOperationException("Method unimplemented");
     }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PartitionedSnapshot.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PartitionedSnapshot.java
index 679e7a8..e346d9c 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PartitionedSnapshot.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PartitionedSnapshot.java
@@ -143,13 +143,13 @@ public class PartitionedSnapshot<T extends Snapshot> extends Snapshot {
     }
 
     @Override
-    public void install(PartitionedSnapshot snapshot, int slot)
+    public void install(PartitionedSnapshot snapshot, int slot, boolean isDataMigration)
         throws SnapshotInstallationException {
       installPartitionedSnapshot(snapshot);
     }
 
     @Override
-    public void install(Map<Integer, PartitionedSnapshot> snapshotMap) {
+    public void install(Map<Integer, PartitionedSnapshot> snapshotMap, boolean isDataMigration) {
       throw new IllegalStateException("Method unimplemented");
     }
 
@@ -203,7 +203,7 @@ public class PartitionedSnapshot<T extends Snapshot> extends Snapshot {
       }
       SnapshotInstaller<T> defaultInstaller =
           (SnapshotInstaller<T>) snapshot.getDefaultInstaller(dataGroupMember);
-      defaultInstaller.install(snapshot, slot);
+      defaultInstaller.install(snapshot, slot, false);
     }
   }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTask.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTask.java
index 4d231cc..daa813d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTask.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTask.java
@@ -25,6 +25,7 @@ import org.apache.iotdb.cluster.client.sync.SyncDataClient;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.exception.SnapshotInstallationException;
 import org.apache.iotdb.cluster.log.Snapshot;
+import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotResp;
@@ -85,6 +86,7 @@ public class PullSnapshotTask<T extends Snapshot> implements Callable<Void> {
     this.newMember = newMember;
     this.snapshotFactory = snapshotFactory;
     this.snapshotSave = snapshotSave;
+    persistTask();
   }
 
   @SuppressWarnings("java:S3740") // type cannot be known ahead
@@ -92,10 +94,12 @@ public class PullSnapshotTask<T extends Snapshot> implements Callable<Void> {
     Node node = descriptor.getPreviousHolders().get(nodeIndex);
     if (logger.isDebugEnabled()) {
       logger.debug(
-          "Pulling {} snapshots from {} of {}",
-          descriptor.getSlots().size(),
+          "Pulling slot {} and other {} snapshots from {} of {} for {}",
+          descriptor.getSlots().get(0),
+          descriptor.getSlots().size() - 1,
           node,
-          descriptor.getPreviousHolders().getHeader());
+          descriptor.getPreviousHolders().getHeader(),
+          newMember.getName());
     }
 
     Map<Integer, T> result = pullSnapshot(node);
@@ -105,10 +109,11 @@ public class PullSnapshotTask<T extends Snapshot> implements Callable<Void> {
       List<Integer> noSnapshotSlots = new ArrayList<>();
       for (Integer slot : descriptor.getSlots()) {
         if (!result.containsKey(slot)) {
-          newMember.getSlotManager().setToNull(slot);
+          newMember.getSlotManager().setToNull(slot, false);
           noSnapshotSlots.add(slot);
         }
       }
+      newMember.getSlotManager().save();
       if (!noSnapshotSlots.isEmpty() && logger.isInfoEnabled()) {
         logger.info(
             "{}: {} and other {} slots do not have snapshot",
@@ -119,14 +124,17 @@ public class PullSnapshotTask<T extends Snapshot> implements Callable<Void> {
 
       if (logger.isInfoEnabled()) {
         logger.info(
-            "Received a snapshot {} from {}",
+            "{}: Received a snapshot {} from {}",
+            newMember.getName(),
             result,
             descriptor.getPreviousHolders().get(nodeIndex));
       }
       try {
-        Snapshot snapshot = result.values().iterator().next();
-        SnapshotInstaller installer = snapshot.getDefaultInstaller(newMember);
-        installer.install(result);
+        if (result.size() > 0) {
+          Snapshot snapshot = result.values().iterator().next();
+          SnapshotInstaller installer = snapshot.getDefaultInstaller(newMember);
+          installer.install(result, true);
+        }
         // inform the previous holders that one member has successfully pulled snapshot
         newMember.registerPullSnapshotHint(descriptor);
         return true;
@@ -173,22 +181,36 @@ public class PullSnapshotTask<T extends Snapshot> implements Callable<Void> {
 
   @Override
   public Void call() {
-    persistTask();
     request = new PullSnapshotRequest();
     request.setHeader(descriptor.getPreviousHolders().getHeader());
     request.setRequiredSlots(descriptor.getSlots());
     request.setRequireReadOnly(descriptor.isRequireReadOnly());
 
+    logger.info("{}: data migration starts.", newMember.getName());
     boolean finished = false;
-    int nodeIndex = -1;
+    int nodeIndex = ((PartitionGroup) newMember.getAllNodes()).indexOf(newMember.getThisNode()) - 1;
     while (!finished) {
       try {
         // sequentially pick up a node that may have this slot
         nodeIndex = (nodeIndex + 1) % descriptor.getPreviousHolders().size();
+        long startTime = System.currentTimeMillis();
         finished = pullSnapshot(nodeIndex);
         if (!finished) {
+          if (logger.isDebugEnabled()) {
+            logger.debug(
+                "Cannot pull slot {} from {}, retry",
+                descriptor.getSlots(),
+                descriptor.getPreviousHolders().get(nodeIndex));
+          }
           Thread.sleep(
               ClusterDescriptor.getInstance().getConfig().getPullSnapshotRetryIntervalMs());
+        } else {
+          if (logger.isDebugEnabled()) {
+            logger.debug(
+                "{}: Data migration ends, cost {}ms",
+                newMember,
+                (System.currentTimeMillis() - startTime));
+          }
         }
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTaskDescriptor.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTaskDescriptor.java
index a2216fd..7587fa1 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTaskDescriptor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTaskDescriptor.java
@@ -75,6 +75,7 @@ public class PullSnapshotTaskDescriptor {
       dataOutputStream.writeInt(slot);
     }
 
+    dataOutputStream.writeInt(previousHolders.getId());
     dataOutputStream.writeInt(previousHolders.size());
     for (Node previousHolder : previousHolders) {
       NodeSerializeUtils.serialize(previousHolder, dataOutputStream);
@@ -90,8 +91,8 @@ public class PullSnapshotTaskDescriptor {
       slots.add(dataInputStream.readInt());
     }
 
+    previousHolders = new PartitionGroup(dataInputStream.readInt());
     int holderSize = dataInputStream.readInt();
-    previousHolders = new PartitionGroup();
     for (int i = 0; i < holderSize; i++) {
       Node node = new Node();
       NodeSerializeUtils.deserialize(node, dataInputStream);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/SnapshotInstaller.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/SnapshotInstaller.java
index 7b8a1ae..363d97f 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/SnapshotInstaller.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/SnapshotInstaller.java
@@ -26,7 +26,8 @@ import java.util.Map;
 
 public interface SnapshotInstaller<T extends Snapshot> {
 
-  void install(T snapshot, int slot) throws SnapshotInstallationException;
+  void install(T snapshot, int slot, boolean isDataMigration) throws SnapshotInstallationException;
 
-  void install(Map<Integer, T> snapshotMap) throws SnapshotInstallationException;
+  void install(Map<Integer, T> snapshotMap, boolean isDataMigration)
+      throws SnapshotInstallationException;
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
index 170ef49..6f74504 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
@@ -30,6 +30,7 @@ import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.query.manage.QueryCoordinator;
 import org.apache.iotdb.cluster.rpc.thrift.GetAllPathsResult;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
@@ -1075,7 +1076,9 @@ public class CMManager extends MManager {
         // this node is a member of the group, perform a local query after synchronizing with the
         // leader
         try {
-          metaGroupMember.getLocalDataMember(partitionGroup.getHeader()).syncLeader(null);
+          metaGroupMember
+              .getLocalDataMember(partitionGroup.getHeader(), partitionGroup.getId())
+              .syncLeader(null);
         } catch (CheckConsistencyException e) {
           logger.warn("Failed to check consistency.", e);
         }
@@ -1149,7 +1152,7 @@ public class CMManager extends MManager {
 
   @SuppressWarnings("java:S1168") // null and empty list are different
   private List<PartialPath> getMatchedPaths(
-      Node node, Node header, List<String> pathsToQuery, boolean withAlias)
+      Node node, RaftNode header, List<String> pathsToQuery, boolean withAlias)
       throws IOException, TException, InterruptedException {
     GetAllPathsResult result;
     if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
@@ -1212,7 +1215,9 @@ public class CMManager extends MManager {
         // this node is a member of the group, perform a local query after synchronizing with the
         // leader
         try {
-          metaGroupMember.getLocalDataMember(partitionGroup.getHeader()).syncLeader(null);
+          metaGroupMember
+              .getLocalDataMember(partitionGroup.getHeader(), partitionGroup.getId())
+              .syncLeader(null);
         } catch (CheckConsistencyException e) {
           logger.warn("Failed to check consistency.", e);
         }
@@ -1275,7 +1280,7 @@ public class CMManager extends MManager {
     return Collections.emptySet();
   }
 
-  private Set<String> getMatchedDevices(Node node, Node header, List<String> pathsToQuery)
+  private Set<String> getMatchedDevices(Node node, RaftNode header, List<String> pathsToQuery)
       throws IOException, TException, InterruptedException {
     Set<String> paths;
     if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
@@ -1627,8 +1632,8 @@ public class CMManager extends MManager {
   private void getLocalDevices(
       PartitionGroup group, ShowDevicesPlan plan, Set<ShowDevicesResult> resultSet)
       throws CheckConsistencyException, MetadataException {
-    Node header = group.getHeader();
-    DataGroupMember localDataMember = metaGroupMember.getLocalDataMember(header);
+    DataGroupMember localDataMember =
+        metaGroupMember.getLocalDataMember(group.getHeader(), group.getId());
     localDataMember.syncLeaderWithConsistencyCheck(false);
     try {
       List<ShowDevicesResult> localResult = super.getDevices(plan);
@@ -1646,8 +1651,8 @@ public class CMManager extends MManager {
       Set<ShowTimeSeriesResult> resultSet,
       QueryContext context)
       throws CheckConsistencyException, MetadataException {
-    Node header = group.getHeader();
-    DataGroupMember localDataMember = metaGroupMember.getLocalDataMember(header);
+    DataGroupMember localDataMember =
+        metaGroupMember.getLocalDataMember(group.getHeader(), group.getId());
     localDataMember.syncLeaderWithConsistencyCheck(false);
     try {
       List<ShowTimeSeriesResult> localResult = super.showTimeseries(plan, context);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/MetaPuller.java b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/MetaPuller.java
index 987686b..ba85bd1 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/MetaPuller.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/MetaPuller.java
@@ -29,6 +29,7 @@ import org.apache.iotdb.cluster.query.manage.QueryCoordinator;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.PullSchemaRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSchemaResp;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.cluster.utils.ClusterUtils;
@@ -130,7 +131,7 @@ public class MetaPuller {
    * @param prefixPaths
    * @param results
    */
-  private void pullMeasurementSchemas(
+  public void pullMeasurementSchemas(
       PartitionGroup partitionGroup,
       List<PartialPath> prefixPaths,
       List<IMeasurementSchema> results) {
@@ -262,7 +263,7 @@ public class MetaPuller {
    *     the same groups. If this method is called by an applier, it holds the lock of LogManager,
    *     while the pulling thread may want this lock too, resulting in a deadlock.
    */
-  public void pullTimeSeriesSchemas(List<PartialPath> prefixPaths, Node ignoredGroup)
+  public void pullTimeSeriesSchemas(List<PartialPath> prefixPaths, RaftNode ignoredGroup)
       throws MetadataException {
     logger.debug(
         "{}: Pulling timeseries schemas of {}, ignored group {}",
@@ -297,16 +298,20 @@ public class MetaPuller {
         partitionGroupPathMap.entrySet()) {
       PartitionGroup partitionGroup = partitionGroupListEntry.getKey();
       List<String> paths = partitionGroupListEntry.getValue();
-      pullTimeSeriesSchemas(partitionGroup, paths);
+      pullTimeSeriesSchemas(partitionGroup, paths, null);
     }
   }
 
   /**
    * Pull timeseries schemas of "prefixPaths" from "partitionGroup". If this node is a member of
    * "partitionGroup", synchronize with the group leader and collect local schemas. Otherwise pull
-   * schemas from one node in the group. The pulled schemas will be cached in CMManager.
+   * schemas from one node in the group. If "timeseriesSchemas" is null, the pulled schemas will be
+   * cached in CMManager.
    */
-  private void pullTimeSeriesSchemas(PartitionGroup partitionGroup, List<String> prefixPaths) {
+  public void pullTimeSeriesSchemas(
+      PartitionGroup partitionGroup,
+      List<String> prefixPaths,
+      List<TimeseriesSchema> timeseriesSchemas) {
     if (partitionGroup.contains(metaGroupMember.getThisNode())) {
       // the node is in the target group, synchronize with leader should be enough
       try {
@@ -327,19 +332,20 @@ public class MetaPuller {
     // decide the node access order with the help of QueryCoordinator
     List<Node> nodes = QueryCoordinator.getINSTANCE().reorderNodes(partitionGroup);
     for (Node node : nodes) {
-      if (tryPullTimeSeriesSchemas(node, pullSchemaRequest)) {
+      if (tryPullTimeSeriesSchemas(node, pullSchemaRequest, timeseriesSchemas)) {
         break;
       }
     }
   }
 
   /**
-   * send the PullSchemaRequest to "node" and cache the results in CMManager if they are
-   * successfully returned.
+   * send the PullSchemaRequest to "node" and cache the results in CMManager or add the results to
+   * "timeseriesSchemas" if they are successfully returned.
    *
    * @return true if the pull succeeded, false otherwise
    */
-  private boolean tryPullTimeSeriesSchemas(Node node, PullSchemaRequest request) {
+  private boolean tryPullTimeSeriesSchemas(
+      Node node, PullSchemaRequest request, List<TimeseriesSchema> timeseriesSchemas) {
     if (logger.isDebugEnabled()) {
       logger.debug(
           "{}: Pulling timeseries schemas of {} and other {} paths from {}",
@@ -382,8 +388,12 @@ public class MetaPuller {
             node,
             request.getHeader());
       }
-      for (TimeseriesSchema schema : schemas) {
-        SchemaUtils.cacheTimeseriesSchema(schema);
+      if (timeseriesSchemas == null) {
+        for (TimeseriesSchema schema : schemas) {
+          SchemaUtils.cacheTimeseriesSchema(schema);
+        }
+      } else {
+        timeseriesSchemas.addAll(schemas);
       }
       return true;
     }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/NodeAdditionResult.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/NodeAdditionResult.java
index 84dbcff..2d8dadd 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/NodeAdditionResult.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/NodeAdditionResult.java
@@ -19,16 +19,19 @@
 
 package org.apache.iotdb.cluster.partition;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public class NodeAdditionResult {
 
   /** A new data group headed by the new node. */
-  private PartitionGroup newGroup;
+  private List<PartitionGroup> newGroupList = new ArrayList<>();
 
-  public PartitionGroup getNewGroup() {
-    return newGroup;
+  public List<PartitionGroup> getNewGroupList() {
+    return newGroupList;
   }
 
-  public void setNewGroup(PartitionGroup newGroup) {
-    this.newGroup = newGroup;
+  public void addNewGroup(PartitionGroup newGroup) {
+    this.newGroupList.add(newGroup);
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/NodeRemovalResult.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/NodeRemovalResult.java
index 71d1cd3..ab4d413 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/NodeRemovalResult.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/NodeRemovalResult.java
@@ -19,26 +19,43 @@
 
 package org.apache.iotdb.cluster.partition;
 
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
 /** NodeRemovalResult stores the removed partition group. */
 public class NodeRemovalResult {
-  private PartitionGroup removedGroup;
-  // if the removed group contains the local node, the local node should join a new group to
-  // preserve the replication number
-  private PartitionGroup newGroup;
 
-  public PartitionGroup getRemovedGroup() {
-    return removedGroup;
+  private List<PartitionGroup> removedGroupList = new ArrayList<>();
+
+  public PartitionGroup getRemovedGroup(int raftId) {
+    for (PartitionGroup group : removedGroupList) {
+      if (group.getId() == raftId) {
+        return group;
+      }
+    }
+    return null;
   }
 
-  public void setRemovedGroup(PartitionGroup group) {
-    this.removedGroup = group;
+  public void addRemovedGroup(PartitionGroup group) {
+    this.removedGroupList.add(group);
   }
 
-  public PartitionGroup getNewGroup() {
-    return newGroup;
+  public void serialize(DataOutputStream dataOutputStream) throws IOException {
+    dataOutputStream.writeInt(removedGroupList.size());
+    for (PartitionGroup group : removedGroupList) {
+      group.serialize(dataOutputStream);
+    }
   }
 
-  public void setNewGroup(PartitionGroup newGroup) {
-    this.newGroup = newGroup;
+  public void deserialize(ByteBuffer buffer) {
+    int removedGroupListSize = buffer.getInt();
+    for (int i = 0; i < removedGroupListSize; i++) {
+      PartitionGroup group = new PartitionGroup();
+      group.deserialize(buffer);
+      removedGroupList.add(group);
+    }
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionGroup.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionGroup.java
index 039fd95..1ce653e 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionGroup.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionGroup.java
@@ -20,9 +20,16 @@
 package org.apache.iotdb.cluster.partition;
 
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
+import org.apache.iotdb.cluster.utils.NodeSerializeUtils;
 
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.Objects;
 
 /**
  * PartitionGroup contains all the nodes that will form a data group with a certain node, which are
@@ -31,34 +38,68 @@ import java.util.Arrays;
  */
 public class PartitionGroup extends ArrayList<Node> {
 
-  private Node thisNode;
+  private int id;
 
   public PartitionGroup() {}
 
-  public PartitionGroup(Node... nodes) {
+  public PartitionGroup(Collection<Node> nodes) {
+    this.addAll(nodes);
+  }
+
+  public PartitionGroup(int id, Node... nodes) {
     this.addAll(Arrays.asList(nodes));
+    this.id = id;
   }
 
   public PartitionGroup(PartitionGroup other) {
     super(other);
-    this.thisNode = other.thisNode;
+    this.id = other.getId();
   }
 
   @Override
   public boolean equals(Object o) {
-    return super.equals(o);
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    PartitionGroup group = (PartitionGroup) o;
+    return Objects.equals(id, group.getId()) && super.equals(group);
+  }
+
+  public void serialize(DataOutputStream dataOutputStream) throws IOException {
+    dataOutputStream.writeInt(getId());
+    dataOutputStream.writeInt(size());
+    for (Node node : this) {
+      NodeSerializeUtils.serialize(node, dataOutputStream);
+    }
+  }
+
+  public void deserialize(ByteBuffer buffer) {
+    id = buffer.getInt();
+    int nodeNum = buffer.getInt();
+    for (int i2 = 0; i2 < nodeNum; i2++) {
+      Node node = new Node();
+      NodeSerializeUtils.deserialize(node, buffer);
+      add(node);
+    }
   }
 
   @Override
   public int hashCode() {
-    return super.hashCode();
+    return Objects.hash(id, super.hashCode());
+  }
+
+  public RaftNode getHeader() {
+    return new RaftNode(get(0), getId());
   }
 
-  public Node getHeader() {
-    return get(0);
+  public int getId() {
+    return id;
   }
 
-  public void setThisNode(Node thisNode) {
-    this.thisNode = thisNode;
+  public void setId(int id) {
+    this.id = id;
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
index 2ecab71..98b8b84 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.cluster.partition;
 
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
@@ -48,29 +49,37 @@ public interface PartitionTable {
   PartitionGroup route(String storageGroupName, long timestamp);
 
   /**
-   * Given the storageGroupName and the timestamp, return the header node of the partitionGroup by
-   * which the storage group and the corresponding time interval is managed.
+   * Given the storageGroupName and the timestamp, return the header RaftNode of the partitionGroup
+   * by which the storage group and the corresponding time interval is managed.
    *
    * @param storageGroupName
    * @param timestamp
    * @return
    */
-  Node routeToHeaderByTime(String storageGroupName, long timestamp);
+  RaftNode routeToHeaderByTime(String storageGroupName, long timestamp);
 
   /**
    * Add a new node to update the partition table.
    *
    * @param node
-   * @return the new group generated by the node
    */
-  NodeAdditionResult addNode(Node node);
+  void addNode(Node node);
+
+  NodeAdditionResult getNodeAdditionResult(Node node);
 
   /**
    * Remove a node and update the partition table.
    *
    * @param node
    */
-  NodeRemovalResult removeNode(Node node);
+  void removeNode(Node node);
+
+  /**
+   * Get the result after remove node, include removedGroupList and newSlotOwners.
+   *
+   * @return result after remove node.
+   */
+  NodeRemovalResult getNodeRemovalResult();
 
   /**
    * @return All data groups where all VNodes of this node is the header. The first index indicates
@@ -79,19 +88,33 @@ public interface PartitionTable {
   List<PartitionGroup> getLocalGroups();
 
   /**
-   * @param header
+   * @param raftNode
    * @return the partition group starting from the header.
    */
-  PartitionGroup getHeaderGroup(Node header);
+  PartitionGroup getHeaderGroup(RaftNode raftNode);
 
   ByteBuffer serialize();
 
-  void deserialize(ByteBuffer buffer);
+  /**
+   * Deserialize partition table and check whether the partition table in byte buffer is valid
+   *
+   * @param buffer
+   * @return true if the partition table is valid
+   */
+  boolean deserialize(ByteBuffer buffer);
 
   List<Node> getAllNodes();
 
   List<PartitionGroup> getGlobalGroups();
 
+  List<PartitionGroup> calculateGlobalGroups(List<Node> nodeRing);
+
+  /** get the last meta log index that modifies the partition table */
+  long getLastMetaLogIndex();
+
+  /** set the last meta log index that modifies the partition table */
+  void setLastMetaLogIndex(long index);
+
   /**
    * @param path can be an incomplete path (but should contain a storage group name) e.g., if
    *     "root.sg" is a storage group, then path can not be "root".
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/balancer/DefaultSlotBalancer.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/balancer/DefaultSlotBalancer.java
new file mode 100644
index 0000000..f7bf377
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/balancer/DefaultSlotBalancer.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.cluster.partition.balancer;
+
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.partition.PartitionGroup;
+import org.apache.iotdb.cluster.partition.slot.SlotPartitionTable;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/** This balancer aims to avg slots to all raft groups. */
+public class DefaultSlotBalancer implements SlotBalancer {
+
+  private int multiRaftFactor = ClusterDescriptor.getInstance().getConfig().getMultiRaftFactor();
+  private SlotPartitionTable table;
+
+  public DefaultSlotBalancer(SlotPartitionTable partitionTable) {
+    this.table = partitionTable;
+  }
+
+  /**
+   * Move last slots from each group whose slot number is bigger than the new average to the new
+   * node.
+   */
+  @Override
+  public void moveSlotsToNew(Node newNode, List<Node> oldRing) {
+    Map<RaftNode, List<Integer>> nodeSlotMap = table.getAllNodeSlots();
+    Map<RaftNode, Map<Integer, PartitionGroup>> previousNodeMap = table.getPreviousNodeMap();
+    RaftNode[] slotNodes = table.getSlotNodes();
+
+    // as a node is added, the average slots for each node decrease
+    // move the slots to the new node if any previous node have more slots than the new average
+    int newAvg = table.getTotalSlotNumbers() / table.getAllNodes().size() / multiRaftFactor;
+    Map<RaftNode, List<Integer>> newNodeSlotMap = new HashMap<>();
+    int raftId = 0;
+    for (int i = 0; i < multiRaftFactor; i++) {
+      RaftNode raftNode = new RaftNode(newNode, i);
+      newNodeSlotMap.putIfAbsent(raftNode, new ArrayList<>());
+      previousNodeMap.putIfAbsent(raftNode, new HashMap<>());
+    }
+    for (Entry<RaftNode, List<Integer>> entry : nodeSlotMap.entrySet()) {
+      List<Integer> slots = entry.getValue();
+      int transferNum = slots.size() - newAvg;
+      if (transferNum > 0) {
+        RaftNode curNode = new RaftNode(newNode, raftId);
+        int numToMove = transferNum;
+        if (raftId != multiRaftFactor - 1) {
+          numToMove = Math.min(numToMove, newAvg - newNodeSlotMap.get(curNode).size());
+        }
+        List<Integer> slotsToMove =
+            slots.subList(slots.size() - transferNum, slots.size() - transferNum + numToMove);
+        newNodeSlotMap.get(curNode).addAll(slotsToMove);
+        for (Integer slot : slotsToMove) {
+          // record what node previously hold the integer
+          previousNodeMap.get(curNode).put(slot, table.getHeaderGroup(entry.getKey(), oldRing));
+          slotNodes[slot] = curNode;
+        }
+        slotsToMove.clear();
+        transferNum -= numToMove;
+        if (transferNum > 0) {
+          curNode = new RaftNode(newNode, ++raftId);
+          slotsToMove = slots.subList(slots.size() - transferNum, slots.size());
+          newNodeSlotMap.get(curNode).addAll(slotsToMove);
+          for (Integer slot : slotsToMove) {
+            // record what node previously hold the integer
+            previousNodeMap.get(curNode).put(slot, table.getHeaderGroup(entry.getKey(), oldRing));
+            slotNodes[slot] = curNode;
+          }
+          slotsToMove.clear();
+        }
+      }
+    }
+    nodeSlotMap.putAll(newNodeSlotMap);
+  }
+
+  @Override
+  public Map<RaftNode, List<Integer>> retrieveSlots(Node target) {
+    Map<RaftNode, List<Integer>> nodeSlotMap = table.getAllNodeSlots();
+    RaftNode[] slotNodes = table.getSlotNodes();
+    List<Node> nodeRing = table.getAllNodes();
+
+    Map<RaftNode, List<Integer>> newHolderSlotMap = new HashMap<>();
+    for (int raftId = 0; raftId < multiRaftFactor; raftId++) {
+      RaftNode raftNode = new RaftNode(target, raftId);
+      List<Integer> slots = nodeSlotMap.remove(raftNode);
+      for (int i = 0; i < slots.size(); i++) {
+        int slot = slots.get(i);
+        RaftNode newHolder = new RaftNode(nodeRing.get(i % nodeRing.size()), raftId);
+        slotNodes[slot] = newHolder;
+        nodeSlotMap.computeIfAbsent(newHolder, n -> new ArrayList<>()).add(slot);
+        newHolderSlotMap.computeIfAbsent(newHolder, n -> new ArrayList<>()).add(slot);
+      }
+    }
+    return newHolderSlotMap;
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotNodeRemovalResult.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/balancer/SlotBalancer.java
similarity index 60%
copy from cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotNodeRemovalResult.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/partition/balancer/SlotBalancer.java
index 218be7f..3bfe241 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotNodeRemovalResult.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/balancer/SlotBalancer.java
@@ -17,24 +17,27 @@
  * under the License.
  */
 
-package org.apache.iotdb.cluster.partition.slot;
+package org.apache.iotdb.cluster.partition.balancer;
 
-import org.apache.iotdb.cluster.partition.NodeRemovalResult;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 
 import java.util.List;
 import java.util.Map;
 
-/** SlotNodeRemovalResult stores the removed partition group and who will take over its slots. */
-public class SlotNodeRemovalResult extends NodeRemovalResult {
+/** When add/remove node, the slots need to be redistributed. */
+public interface SlotBalancer {
 
-  private Map<Node, List<Integer>> newSlotOwners;
+  /**
+   * When add a new node, new raft groups will take over some hash slots from another raft groups.
+   */
+  void moveSlotsToNew(Node newNode, List<Node> oldRing);
 
-  public Map<Node, List<Integer>> getNewSlotOwners() {
-    return newSlotOwners;
-  }
-
-  public void setNewSlotOwners(Map<Node, List<Integer>> newSlotOwners) {
-    this.newSlotOwners = newSlotOwners;
-  }
+  /**
+   * When remove a old node, all hash slots of the removed groups will assigned to other raft
+   * groups.
+   *
+   * @param target the node to be removed
+   */
+  Map<RaftNode, List<Integer>> retrieveSlots(Node target);
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java
index 528119f..4b9be78 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotManager.java
@@ -1,5 +1,20 @@
 /*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at      http://www.apache.org/licenses/LICENSE-2.0  Unless required by applicable law or ag [...]
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
  */
 
 package org.apache.iotdb.cluster.partition.slot;
@@ -32,6 +47,7 @@ public class SlotManager {
 
   private static final Logger logger = LoggerFactory.getLogger(SlotManager.class);
   private static final long SLOT_WAIT_INTERVAL_MS = 10;
+  private static final long SLOT_WAIT_THRESHOLD_MS = 2000;
   private static final String SLOT_FILE_NAME = "SLOT_STATUS";
 
   private String slotFilePath;
@@ -39,10 +55,13 @@ public class SlotManager {
   /** the serial number of a slot -> the status and source of a slot */
   private Map<Integer, SlotDescriptor> idSlotMap;
 
-  public SlotManager(long totalSlotNumber, String memberDir) {
+  private String memberName;
+
+  public SlotManager(long totalSlotNumber, String memberDir, String memberName) {
     if (memberDir != null) {
       this.slotFilePath = memberDir + File.separator + SLOT_FILE_NAME;
     }
+    this.memberName = memberName;
     if (!load()) {
       init(totalSlotNumber);
     }
@@ -62,6 +81,7 @@ public class SlotManager {
    */
   public void waitSlot(int slotId) {
     SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
+    long startTime = System.currentTimeMillis();
     while (true) {
       synchronized (slotDescriptor) {
         if (slotDescriptor.slotStatus == SlotStatus.PULLING
@@ -73,6 +93,10 @@ public class SlotManager {
             logger.error("Unexpected interruption when waiting for slot {}", slotId, e);
           }
         } else {
+          long cost = System.currentTimeMillis() - startTime;
+          if (cost > SLOT_WAIT_THRESHOLD_MS) {
+            logger.info("Wait slot {} cost {}ms", slotId, cost);
+          }
           return;
         }
       }
@@ -86,16 +110,15 @@ public class SlotManager {
    */
   public void waitSlotForWrite(int slotId) throws StorageEngineException {
     SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
+    long startTime = System.currentTimeMillis();
     while (true) {
       synchronized (slotDescriptor) {
-        if (slotDescriptor.slotStatus == SlotStatus.SENDING
-            || slotDescriptor.slotStatus == SlotStatus.SENT) {
-          throw new StorageEngineException(
-              String.format("Slot %d no longer belongs to the node", slotId));
-        }
-        if (slotDescriptor.slotStatus != SlotStatus.NULL
-            && slotDescriptor.slotStatus != SlotStatus.PULLING_WRITABLE) {
+        if (slotDescriptor.slotStatus == SlotStatus.PULLING) {
           try {
+            if ((System.currentTimeMillis() - startTime) >= SLOT_WAIT_THRESHOLD_MS) {
+              throw new StorageEngineException(
+                  String.format("The status of slot %d is still PULLING after 5s.", slotId));
+            }
             slotDescriptor.wait(SLOT_WAIT_INTERVAL_MS);
           } catch (InterruptedException e) {
             Thread.currentThread().interrupt();
@@ -108,6 +131,18 @@ public class SlotManager {
     }
   }
 
+  /** If a slot in the status of PULLING or PULLING_WRITABLE, reads of it should merge the source */
+  public boolean checkSlotInDataMigrationStatus(int slotId) {
+    SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
+    return slotDescriptor.slotStatus == SlotStatus.PULLING
+        || slotDescriptor.slotStatus == SlotStatus.PULLING_WRITABLE;
+  }
+
+  public boolean checkSlotInMetaMigrationStatus(int slotId) {
+    SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
+    return slotDescriptor.slotStatus == SlotStatus.PULLING;
+  }
+
   /**
    * @param slotId
    * @return the SlotStatus of a slot
@@ -131,11 +166,18 @@ public class SlotManager {
    * @param source
    */
   public void setToPulling(int slotId, Node source) {
+    setToPulling(slotId, source, true);
+  }
+
+  public void setToPulling(int slotId, Node source, boolean needSave) {
     SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
     synchronized (slotDescriptor) {
       slotDescriptor.slotStatus = SlotStatus.PULLING;
       slotDescriptor.source = source;
     }
+    if (needSave) {
+      save();
+    }
   }
 
   /**
@@ -144,12 +186,18 @@ public class SlotManager {
    * @param slotId
    */
   public void setToPullingWritable(int slotId) {
+    setToPullingWritable(slotId, true);
+  }
+
+  public void setToPullingWritable(int slotId, boolean needSave) {
     SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
     synchronized (slotDescriptor) {
       slotDescriptor.slotStatus = SlotStatus.PULLING_WRITABLE;
       slotDescriptor.notifyAll();
     }
-    save();
+    if (needSave) {
+      save();
+    }
   }
 
   /**
@@ -158,16 +206,26 @@ public class SlotManager {
    * @param slotId
    */
   public void setToNull(int slotId) {
+    setToNull(slotId, true);
+  }
+
+  public void setToNull(int slotId, boolean needSave) {
     SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
     synchronized (slotDescriptor) {
       slotDescriptor.slotStatus = SlotStatus.NULL;
       slotDescriptor.source = null;
       slotDescriptor.notifyAll();
     }
-    save();
+    if (needSave) {
+      save();
+    }
   }
 
   public void setToSending(int slotId) {
+    setToSending(slotId, true);
+  }
+
+  public void setToSending(int slotId, boolean needSave) {
     // only NULL slots can be set to SENDING
     waitSlot(slotId);
     SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
@@ -175,7 +233,9 @@ public class SlotManager {
       slotDescriptor.slotStatus = SlotStatus.SENDING;
       slotDescriptor.snapshotReceivedCount = 0;
     }
-    save();
+    if (needSave) {
+      save();
+    }
   }
 
   private void setToSent(int slotId) {
@@ -183,7 +243,6 @@ public class SlotManager {
     synchronized (slotDescriptor) {
       slotDescriptor.slotStatus = SlotStatus.SENT;
     }
-    save();
   }
 
   /**
@@ -195,13 +254,19 @@ public class SlotManager {
    *     invocation).
    */
   public int sentOneReplication(int slotId) {
+    return sentOneReplication(slotId, true);
+  }
+
+  public int sentOneReplication(int slotId, boolean needSave) {
     SlotDescriptor slotDescriptor = idSlotMap.get(slotId);
     synchronized (slotDescriptor) {
       int sentReplicaNum = ++slotDescriptor.snapshotReceivedCount;
       if (sentReplicaNum >= ClusterDescriptor.getInstance().getConfig().getReplicationNum()) {
         setToSent(slotId);
       }
-      save();
+      if (needSave) {
+        save();
+      }
       return sentReplicaNum;
     }
   }
@@ -234,7 +299,7 @@ public class SlotManager {
     }
   }
 
-  private synchronized void save() {
+  public synchronized void save() {
     if (slotFilePath == null) {
       return;
     }
@@ -251,6 +316,23 @@ public class SlotManager {
     }
   }
 
+  public int getSloNumInDataMigration() {
+    int res = 0;
+    for (Entry<Integer, SlotDescriptor> entry : idSlotMap.entrySet()) {
+      SlotDescriptor descriptor = entry.getValue();
+      if (descriptor.slotStatus == SlotStatus.PULLING
+          || descriptor.slotStatus == SlotStatus.PULLING_WRITABLE) {
+        logger.info(
+            "{}: slot {} is in data migration, status is {}",
+            memberName,
+            entry.getKey(),
+            descriptor.slotStatus);
+        res++;
+      }
+    }
+    return res;
+  }
+
   private void serialize(DataOutputStream outputStream) throws IOException {
     outputStream.writeInt(idSlotMap.size());
     for (Entry<Integer, SlotDescriptor> integerSlotDescriptorEntry : idSlotMap.entrySet()) {
@@ -284,7 +366,7 @@ public class SlotManager {
   }
 
   private static class SlotDescriptor {
-    private SlotStatus slotStatus;
+    private volatile SlotStatus slotStatus;
     private Node source;
     // in LOSING status, how many members in the new owner have pulled data
     private volatile int snapshotReceivedCount;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotNodeAdditionResult.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotNodeAdditionResult.java
index 0c8b99d..c6c702c 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotNodeAdditionResult.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotNodeAdditionResult.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.cluster.partition.slot;
 
 import org.apache.iotdb.cluster.partition.NodeAdditionResult;
-import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 
 import java.util.Map;
 import java.util.Set;
@@ -28,13 +28,13 @@ import java.util.Set;
 public class SlotNodeAdditionResult extends NodeAdditionResult {
 
   /** What slots will the old data groups transfer to the new one. */
-  private Map<Node, Set<Integer>> lostSlots;
+  private Map<RaftNode, Set<Integer>> lostSlots;
 
-  public Map<Node, Set<Integer>> getLostSlots() {
+  public Map<RaftNode, Set<Integer>> getLostSlots() {
     return lostSlots;
   }
 
-  public void setLostSlots(Map<Node, Set<Integer>> lostSlots) {
+  public void setLostSlots(Map<RaftNode, Set<Integer>> lostSlots) {
     this.lostSlots = lostSlots;
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotNodeRemovalResult.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotNodeRemovalResult.java
index 218be7f..783208d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotNodeRemovalResult.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotNodeRemovalResult.java
@@ -21,20 +21,59 @@ package org.apache.iotdb.cluster.partition.slot;
 
 import org.apache.iotdb.cluster.partition.NodeRemovalResult;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
+import org.apache.iotdb.cluster.utils.NodeSerializeUtils;
 
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 /** SlotNodeRemovalResult stores the removed partition group and who will take over its slots. */
 public class SlotNodeRemovalResult extends NodeRemovalResult {
 
-  private Map<Node, List<Integer>> newSlotOwners;
+  private Map<RaftNode, List<Integer>> newSlotOwners = new HashMap<>();
 
-  public Map<Node, List<Integer>> getNewSlotOwners() {
+  public Map<RaftNode, List<Integer>> getNewSlotOwners() {
     return newSlotOwners;
   }
 
-  public void setNewSlotOwners(Map<Node, List<Integer>> newSlotOwners) {
+  public void addNewSlotOwners(Map<RaftNode, List<Integer>> newSlotOwners) {
     this.newSlotOwners = newSlotOwners;
   }
+
+  @Override
+  public void serialize(DataOutputStream dataOutputStream) throws IOException {
+    super.serialize(dataOutputStream);
+    dataOutputStream.writeInt(newSlotOwners.size());
+    for (Map.Entry<RaftNode, List<Integer>> entry : newSlotOwners.entrySet()) {
+      RaftNode raftNode = entry.getKey();
+      NodeSerializeUtils.serialize(raftNode.getNode(), dataOutputStream);
+      dataOutputStream.writeInt(raftNode.getRaftId());
+      dataOutputStream.writeInt(entry.getValue().size());
+      for (Integer slot : entry.getValue()) {
+        dataOutputStream.writeInt(slot);
+      }
+    }
+  }
+
+  @Override
+  public void deserialize(ByteBuffer buffer) {
+    super.deserialize(buffer);
+    int size = buffer.getInt();
+    for (int i = 0; i < size; i++) {
+      Node node = new Node();
+      NodeSerializeUtils.deserialize(node, buffer);
+      RaftNode raftNode = new RaftNode(node, buffer.getInt());
+      List<Integer> slots = new ArrayList<>();
+      int slotSize = buffer.getInt();
+      for (int j = 0; j < slotSize; j++) {
+        slots.add(buffer.getInt());
+      }
+      newSlotOwners.put(raftNode, slots);
+    }
+  }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotPartitionTable.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotPartitionTable.java
index bdc1578..7a46baa 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotPartitionTable.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotPartitionTable.java
@@ -1,5 +1,20 @@
 /*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at      http://www.apache.org/licenses/LICENSE-2.0  Unless required by applicable law or ag [...]
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
  */
 
 package org.apache.iotdb.cluster.partition.slot;
@@ -10,8 +25,11 @@ import org.apache.iotdb.cluster.partition.NodeAdditionResult;
 import org.apache.iotdb.cluster.partition.NodeRemovalResult;
 import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.partition.PartitionTable;
+import org.apache.iotdb.cluster.partition.balancer.DefaultSlotBalancer;
+import org.apache.iotdb.cluster.partition.balancer.SlotBalancer;
 import org.apache.iotdb.cluster.partition.slot.SlotStrategy.DefaultStrategy;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.utils.NodeSerializeUtils;
 import org.apache.iotdb.db.utils.SerializeUtils;
 
@@ -49,6 +67,8 @@ public class SlotPartitionTable implements PartitionTable {
 
   private int replicationNum = ClusterDescriptor.getInstance().getConfig().getReplicationNum();
 
+  private int multiRaftFactor = ClusterDescriptor.getInstance().getConfig().getMultiRaftFactor();
+
   // all nodes
   private List<Node> nodeRing = new ArrayList<>();
   // normally, it is equal to ClusterConstant.SLOT_NUM.
@@ -56,12 +76,14 @@ public class SlotPartitionTable implements PartitionTable {
 
   // The following fields are used for determining which node a data item belongs to.
   // the slots held by each node
-  private Map<Node, List<Integer>> nodeSlotMap = new ConcurrentHashMap<>();
+  private Map<RaftNode, List<Integer>> nodeSlotMap = new ConcurrentHashMap<>();
   // each slot is managed by whom
-  private Node[] slotNodes = new Node[ClusterConstant.SLOT_NUM];
+  private RaftNode[] slotNodes = new RaftNode[ClusterConstant.SLOT_NUM];
   // the nodes that each slot belongs to before a new node is added, used for the new node to
   // find the data source
-  private Map<Node, Map<Integer, Node>> previousNodeMap = new ConcurrentHashMap<>();
+  private Map<RaftNode, Map<Integer, PartitionGroup>> previousNodeMap = new ConcurrentHashMap<>();
+
+  private NodeRemovalResult nodeRemovalResult = new SlotNodeRemovalResult();
 
   // the filed is used for determining which nodes need to be a group.
   // the data groups which this node belongs to.
@@ -71,8 +93,10 @@ public class SlotPartitionTable implements PartitionTable {
 
   private List<PartitionGroup> globalGroups;
 
-  // last log index that modifies the partition table
-  private long lastLogIndex = -1;
+  // the last meta log index that modifies the partition table
+  private volatile long lastMetaLogIndex = -1;
+
+  private SlotBalancer slotBalancer = new DefaultSlotBalancer(this);
 
   /**
    * only used for deserialize.
@@ -83,6 +107,19 @@ public class SlotPartitionTable implements PartitionTable {
     this.thisNode = thisNode;
   }
 
+  public SlotPartitionTable(SlotPartitionTable other) {
+    this.thisNode = other.thisNode;
+    this.totalSlotNumbers = other.totalSlotNumbers;
+    this.lastMetaLogIndex = other.lastMetaLogIndex;
+    this.nodeRing = new ArrayList<>(other.nodeRing);
+    this.nodeSlotMap = new HashMap<>(other.nodeSlotMap);
+    this.slotNodes = new RaftNode[totalSlotNumbers];
+    System.arraycopy(other.slotNodes, 0, this.slotNodes, 0, totalSlotNumbers);
+    this.previousNodeMap = new HashMap<>(previousNodeMap);
+
+    localGroups = getPartitionGroups(thisNode);
+  }
+
   public SlotPartitionTable(Collection<Node> nodes, Node thisNode) {
     this(nodes, thisNode, ClusterConstant.SLOT_NUM);
   }
@@ -101,6 +138,14 @@ public class SlotPartitionTable implements PartitionTable {
     SlotPartitionTable.slotStrategy = slotStrategy;
   }
 
+  public SlotBalancer getLoadBalancer() {
+    return slotBalancer;
+  }
+
+  public void setLoadBalancer(SlotBalancer slotBalancer) {
+    this.slotBalancer = slotBalancer;
+  }
+
   private void init(Collection<Node> nodes) {
     logger.info("Initializing a new partition table");
     nodeRing.addAll(nodes);
@@ -113,21 +158,28 @@ public class SlotPartitionTable implements PartitionTable {
     // evenly assign the slots to each node
     int nodeNum = nodeRing.size();
     int slotsPerNode = totalSlotNumbers / nodeNum;
+    int slotsPerRaftGroup = slotsPerNode / multiRaftFactor;
     for (Node node : nodeRing) {
-      nodeSlotMap.put(node, new ArrayList<>());
+      for (int i = 0; i < multiRaftFactor; i++) {
+        nodeSlotMap.put(new RaftNode(node, i), new ArrayList<>());
+      }
     }
 
     for (int i = 0; i < totalSlotNumbers; i++) {
       int nodeIdx = i / slotsPerNode;
+      int raftId = i % slotsPerNode / slotsPerRaftGroup;
       if (nodeIdx >= nodeNum) {
         // the last node may receive a little more if total slots cannot de divided by node number
         nodeIdx--;
       }
-      nodeSlotMap.get(nodeRing.get(nodeIdx)).add(i);
+      if (raftId >= multiRaftFactor) {
+        raftId--;
+      }
+      nodeSlotMap.get(new RaftNode(nodeRing.get(nodeIdx), raftId)).add(i);
     }
 
     // build the index to find a node by slot
-    for (Entry<Node, List<Integer>> entry : nodeSlotMap.entrySet()) {
+    for (Entry<RaftNode, List<Integer>> entry : nodeSlotMap.entrySet()) {
       for (Integer slot : entry.getValue()) {
         slotNodes[slot] = entry.getKey();
       }
@@ -139,6 +191,10 @@ public class SlotPartitionTable implements PartitionTable {
     List<PartitionGroup> ret = new ArrayList<>();
 
     int nodeIndex = nodeRing.indexOf(node);
+    if (nodeIndex == -1) {
+      logger.info("PartitionGroups is empty due to this node has been removed from the cluster!");
+      return ret;
+    }
     for (int i = 0; i < replicationNum; i++) {
       // the previous replicationNum nodes (including the node itself) are the headers of the
       // groups the node is in
@@ -146,21 +202,22 @@ public class SlotPartitionTable implements PartitionTable {
       if (startIndex < 0) {
         startIndex = startIndex + nodeRing.size();
       }
-      ret.add(getHeaderGroup(nodeRing.get(startIndex)));
+      for (int j = 0; j < multiRaftFactor; j++) {
+        ret.add(getHeaderGroup(new RaftNode(nodeRing.get(startIndex), j)));
+      }
     }
 
     logger.debug("The partition groups of {} are: {}", node, ret);
     return ret;
   }
 
-  @Override
-  public PartitionGroup getHeaderGroup(Node node) {
-    PartitionGroup ret = new PartitionGroup();
+  public PartitionGroup getHeaderGroup(RaftNode raftNode, List<Node> nodeRing) {
+    PartitionGroup ret = new PartitionGroup(raftNode.getRaftId());
 
     // assuming the nodes are [1,2,3,4,5]
-    int nodeIndex = nodeRing.indexOf(node);
+    int nodeIndex = nodeRing.indexOf(raftNode.getNode());
     if (nodeIndex == -1) {
-      logger.error("Node {} is not in the cluster", node);
+      logger.warn("Node {} is not in the cluster", raftNode.getNode());
       return null;
     }
     int endIndex = nodeIndex + replicationNum;
@@ -176,10 +233,15 @@ public class SlotPartitionTable implements PartitionTable {
   }
 
   @Override
+  public PartitionGroup getHeaderGroup(RaftNode raftNode) {
+    return getHeaderGroup(raftNode, this.nodeRing);
+  }
+
+  @Override
   public PartitionGroup route(String storageGroupName, long timestamp) {
     synchronized (nodeRing) {
-      Node node = routeToHeaderByTime(storageGroupName, timestamp);
-      return getHeaderGroup(node);
+      RaftNode raftNode = routeToHeaderByTime(storageGroupName, timestamp);
+      return getHeaderGroup(raftNode);
     }
   }
 
@@ -191,40 +253,43 @@ public class SlotPartitionTable implements PartitionTable {
           Thread.currentThread().getStackTrace());
       return null;
     }
-    Node node = slotNodes[slot];
-    logger.debug("The slot of {} is held by {}", slot, node);
-    if (node == null) {
+    RaftNode raftNode = slotNodes[slot];
+    logger.debug("The slot of {} is held by {}", slot, raftNode);
+    if (raftNode.getNode() == null) {
       logger.warn("The slot {} is incorrect", slot);
       return null;
     }
-    return getHeaderGroup(node);
+    return getHeaderGroup(raftNode);
   }
 
   @Override
-  public Node routeToHeaderByTime(String storageGroupName, long timestamp) {
+  public RaftNode routeToHeaderByTime(String storageGroupName, long timestamp) {
     synchronized (nodeRing) {
       int slot =
           getSlotStrategy().calculateSlotByTime(storageGroupName, timestamp, getTotalSlotNumbers());
-      Node node = slotNodes[slot];
-      logger.trace("The slot of {}@{} is {}, held by {}", storageGroupName, timestamp, slot, node);
-      return node;
+      RaftNode raftNode = slotNodes[slot];
+      logger.trace(
+          "The slot of {}@{} is {}, held by {}", storageGroupName, timestamp, slot, raftNode);
+      return raftNode;
     }
   }
 
   @Override
-  public NodeAdditionResult addNode(Node node) {
+  public void addNode(Node node) {
+    List<Node> oldRing;
     synchronized (nodeRing) {
       if (nodeRing.contains(node)) {
-        return null;
+        return;
       }
 
+      oldRing = new ArrayList<>(nodeRing);
       nodeRing.add(node);
       nodeRing.sort(Comparator.comparingInt(Node::getNodeIdentifier));
 
       List<PartitionGroup> retiredGroups = new ArrayList<>();
       for (int i = 0; i < localGroups.size(); i++) {
         PartitionGroup oldGroup = localGroups.get(i);
-        Node header = oldGroup.getHeader();
+        RaftNode header = oldGroup.getHeader();
         PartitionGroup newGrp = getHeaderGroup(header);
         if (newGrp.contains(node) && newGrp.contains(thisNode)) {
           // this group changes but still contains the local node
@@ -240,7 +305,8 @@ public class SlotPartitionTable implements PartitionTable {
       while (groupIterator.hasNext()) {
         PartitionGroup partitionGroup = groupIterator.next();
         for (PartitionGroup retiredGroup : retiredGroups) {
-          if (retiredGroup.getHeader().equals(partitionGroup.getHeader())) {
+          if (retiredGroup.getHeader().equals(partitionGroup.getHeader())
+              && retiredGroup.getId() == partitionGroup.getId()) {
             groupIterator.remove();
             break;
           }
@@ -248,53 +314,34 @@ public class SlotPartitionTable implements PartitionTable {
       }
     }
 
-    SlotNodeAdditionResult result = new SlotNodeAdditionResult();
-    PartitionGroup newGroup = getHeaderGroup(node);
-    if (newGroup.contains(thisNode)) {
-      localGroups.add(newGroup);
+    for (int raftId = 0; raftId < multiRaftFactor; raftId++) {
+      PartitionGroup newGroup = getHeaderGroup(new RaftNode(node, raftId));
+      if (newGroup.contains(thisNode)) {
+        localGroups.add(newGroup);
+      }
     }
-    result.setNewGroup(newGroup);
 
-    calculateGlobalGroups();
+    globalGroups = calculateGlobalGroups(nodeRing);
 
     // the slots movement is only done logically, the new node itself will pull data from the
     // old node
-    result.setLostSlots(moveSlotsToNew(node));
-
-    return result;
+    slotBalancer.moveSlotsToNew(node, oldRing);
+    this.nodeRemovalResult = new SlotNodeRemovalResult();
   }
 
-  /**
-   * Move last slots from each group whose slot number is bigger than the new average to the new
-   * node.
-   *
-   * @param newNode
-   * @return a map recording what slots each group lost.
-   */
-  private Map<Node, Set<Integer>> moveSlotsToNew(Node newNode) {
-    Map<Node, Set<Integer>> result = new HashMap<>();
-    // as a node is added, the average slots for each node decrease
-    // move the slots to the new node if any previous node have more slots than the new average
-    List<Integer> newSlots = new ArrayList<>();
-    Map<Integer, Node> previousHolders = new HashMap<>();
-    int newAvg = totalSlotNumbers / nodeRing.size();
-    for (Entry<Node, List<Integer>> entry : nodeSlotMap.entrySet()) {
-      List<Integer> slots = entry.getValue();
-      int transferNum = slots.size() - newAvg;
-      if (transferNum > 0) {
-        List<Integer> slotsToMove = slots.subList(slots.size() - transferNum, slots.size());
-        newSlots.addAll(slotsToMove);
-        for (Integer slot : slotsToMove) {
-          // record what node previously hold the integer
-          previousHolders.put(slot, entry.getKey());
-          slotNodes[slot] = newNode;
-        }
-        result.computeIfAbsent(entry.getKey(), n -> new HashSet<>()).addAll(slotsToMove);
-        slotsToMove.clear();
+  @Override
+  public NodeAdditionResult getNodeAdditionResult(Node node) {
+    SlotNodeAdditionResult result = new SlotNodeAdditionResult();
+    Map<RaftNode, Set<Integer>> lostSlotsMap = new HashMap<>();
+    for (int raftId = 0; raftId < multiRaftFactor; raftId++) {
+      RaftNode raftNode = new RaftNode(node, raftId);
+      result.addNewGroup(getHeaderGroup(raftNode));
+      for (Entry<Integer, PartitionGroup> entry : previousNodeMap.get(raftNode).entrySet()) {
+        RaftNode header = entry.getValue().getHeader();
+        lostSlotsMap.computeIfAbsent(header, k -> new HashSet<>()).add(entry.getKey());
       }
     }
-    nodeSlotMap.put(newNode, newSlots);
-    previousNodeMap.put(newNode, previousHolders);
+    result.setLostSlots(lostSlotsMap);
     return result;
   }
 
@@ -309,25 +356,29 @@ public class SlotPartitionTable implements PartitionTable {
     DataOutputStream dataOutputStream = new DataOutputStream(outputStream);
 
     try {
+      dataOutputStream.writeLong(lastMetaLogIndex);
       dataOutputStream.writeInt(totalSlotNumbers);
       dataOutputStream.writeInt(nodeSlotMap.size());
-      for (Entry<Node, List<Integer>> entry : nodeSlotMap.entrySet()) {
-        NodeSerializeUtils.serialize(entry.getKey(), dataOutputStream);
+      for (Entry<RaftNode, List<Integer>> entry : nodeSlotMap.entrySet()) {
+        NodeSerializeUtils.serialize(entry.getKey().getNode(), dataOutputStream);
+        dataOutputStream.writeInt(entry.getKey().getRaftId());
         SerializeUtils.serializeIntList(entry.getValue(), dataOutputStream);
       }
 
       dataOutputStream.writeInt(previousNodeMap.size());
-      for (Entry<Node, Map<Integer, Node>> nodeMapEntry : previousNodeMap.entrySet()) {
-        dataOutputStream.writeInt(nodeMapEntry.getKey().getNodeIdentifier());
-
-        Map<Integer, Node> prevHolders = nodeMapEntry.getValue();
+      for (Entry<RaftNode, Map<Integer, PartitionGroup>> nodeMapEntry :
+          previousNodeMap.entrySet()) {
+        NodeSerializeUtils.serialize(nodeMapEntry.getKey().getNode(), dataOutputStream);
+        dataOutputStream.writeInt(nodeMapEntry.getKey().getRaftId());
+        Map<Integer, PartitionGroup> prevHolders = nodeMapEntry.getValue();
         dataOutputStream.writeInt(prevHolders.size());
-        for (Entry<Integer, Node> integerNodeEntry : prevHolders.entrySet()) {
+        for (Entry<Integer, PartitionGroup> integerNodeEntry : prevHolders.entrySet()) {
+          integerNodeEntry.getValue().serialize(dataOutputStream);
           dataOutputStream.writeInt(integerNodeEntry.getKey());
-          dataOutputStream.writeInt(integerNodeEntry.getValue().getNodeIdentifier());
         }
       }
-      dataOutputStream.writeLong(lastLogIndex);
+
+      nodeRemovalResult.serialize(dataOutputStream);
     } catch (IOException ignored) {
       // not reachable
     }
@@ -335,45 +386,68 @@ public class SlotPartitionTable implements PartitionTable {
   }
 
   @Override
-  public void deserialize(ByteBuffer buffer) {
+  public synchronized boolean deserialize(ByteBuffer buffer) {
+    long newLastLogIndex = buffer.getLong();
+
+    if (logger.isDebugEnabled()) {
+      logger.debug(
+          "Partition table: lastMetaLogIndex {}, newLastLogIndex {}",
+          lastMetaLogIndex,
+          newLastLogIndex);
+    }
+    // judge whether the partition table of byte buffer is out of date
+    if (lastMetaLogIndex != -1 && lastMetaLogIndex >= newLastLogIndex) {
+      return lastMetaLogIndex == newLastLogIndex;
+    }
+    lastMetaLogIndex = newLastLogIndex;
     logger.info("Initializing the partition table from buffer");
     totalSlotNumbers = buffer.getInt();
     int size = buffer.getInt();
-    Map<Integer, Node> idNodeMap = new HashMap<>();
+    nodeSlotMap = new HashMap<>();
+    Node node;
     for (int i = 0; i < size; i++) {
-      Node node = new Node();
-      List<Integer> slots = new ArrayList<>();
+      node = new Node();
       NodeSerializeUtils.deserialize(node, buffer);
+      RaftNode raftNode = new RaftNode(node, buffer.getInt());
+      List<Integer> slots = new ArrayList<>();
       SerializeUtils.deserializeIntList(slots, buffer);
-      nodeSlotMap.put(node, slots);
-      idNodeMap.put(node.getNodeIdentifier(), node);
+      nodeSlotMap.put(raftNode, slots);
       for (Integer slot : slots) {
-        slotNodes[slot] = node;
+        slotNodes[slot] = raftNode;
       }
     }
 
     int prevNodeMapSize = buffer.getInt();
     previousNodeMap = new HashMap<>();
     for (int i = 0; i < prevNodeMapSize; i++) {
-      int nodeId = buffer.getInt();
-      Node node = idNodeMap.get(nodeId);
+      node = new Node();
+      NodeSerializeUtils.deserialize(node, buffer);
+      RaftNode raftNode = new RaftNode(node, buffer.getInt());
 
-      Map<Integer, Node> prevHolders = new HashMap<>();
+      Map<Integer, PartitionGroup> prevHolders = new HashMap<>();
       int holderNum = buffer.getInt();
       for (int i1 = 0; i1 < holderNum; i1++) {
-        int slot = buffer.getInt();
-        Node holder = idNodeMap.get(buffer.getInt());
-        prevHolders.put(slot, holder);
+        PartitionGroup group = new PartitionGroup();
+        group.deserialize(buffer);
+        prevHolders.put(buffer.getInt(), group);
       }
-      previousNodeMap.put(node, prevHolders);
+      previousNodeMap.put(raftNode, prevHolders);
     }
-    lastLogIndex = buffer.getLong();
 
-    nodeRing.addAll(nodeSlotMap.keySet());
+    nodeRemovalResult = new SlotNodeRemovalResult();
+    nodeRemovalResult.deserialize(buffer);
+
+    nodeRing.clear();
+    for (RaftNode raftNode : nodeSlotMap.keySet()) {
+      if (!nodeRing.contains(raftNode.getNode())) {
+        nodeRing.add(raftNode.getNode());
+      }
+    }
     Collections.sort(nodeRing);
     logger.info("All known nodes: {}", nodeRing);
 
     localGroups = getPartitionGroups(thisNode);
+    return true;
   }
 
   @Override
@@ -381,15 +455,19 @@ public class SlotPartitionTable implements PartitionTable {
     return nodeRing;
   }
 
-  public Map<Integer, Node> getPreviousNodeMap(Node node) {
-    return previousNodeMap.get(node);
+  public Map<RaftNode, Map<Integer, PartitionGroup>> getPreviousNodeMap() {
+    return previousNodeMap;
+  }
+
+  public Map<Integer, PartitionGroup> getPreviousNodeMap(RaftNode raftNode) {
+    return previousNodeMap.get(raftNode);
   }
 
-  public List<Integer> getNodeSlots(Node header) {
+  public List<Integer> getNodeSlots(RaftNode header) {
     return nodeSlotMap.get(header);
   }
 
-  public Map<Node, List<Integer>> getAllNodeSlots() {
+  public Map<RaftNode, List<Integer>> getAllNodeSlots() {
     return nodeSlotMap;
   }
 
@@ -410,7 +488,8 @@ public class SlotPartitionTable implements PartitionTable {
         && Objects.equals(nodeRing, that.nodeRing)
         && Objects.equals(nodeSlotMap, that.nodeSlotMap)
         && Arrays.equals(slotNodes, that.slotNodes)
-        && Objects.equals(previousNodeMap, that.previousNodeMap);
+        && Objects.equals(previousNodeMap, that.previousNodeMap)
+        && lastMetaLogIndex == that.lastMetaLogIndex;
   }
 
   @Override
@@ -419,64 +498,65 @@ public class SlotPartitionTable implements PartitionTable {
   }
 
   @Override
-  public NodeRemovalResult removeNode(Node target) {
+  public void removeNode(Node target) {
     synchronized (nodeRing) {
       if (!nodeRing.contains(target)) {
-        return null;
+        return;
       }
 
       SlotNodeRemovalResult result = new SlotNodeRemovalResult();
-      result.setRemovedGroup(getHeaderGroup(target));
+      for (int raftId = 0; raftId < multiRaftFactor; raftId++) {
+        result.addRemovedGroup(getHeaderGroup(new RaftNode(target, raftId)));
+      }
       nodeRing.remove(target);
 
       // if the node belongs to a group that headed by target, this group should be removed
       // and other groups containing target should be updated
-      int removedGroupIdx = -1;
+      List<Integer> removedGroupIdxs = new ArrayList<>();
       for (int i = 0; i < localGroups.size(); i++) {
         PartitionGroup oldGroup = localGroups.get(i);
-        Node header = oldGroup.getHeader();
-        if (header.equals(target)) {
-          removedGroupIdx = i;
+        RaftNode header = oldGroup.getHeader();
+        if (header.getNode().equals(target)) {
+          removedGroupIdxs.add(i);
         } else {
           PartitionGroup newGrp = getHeaderGroup(header);
           localGroups.set(i, newGrp);
         }
       }
-      if (removedGroupIdx != -1) {
+      for (int i = removedGroupIdxs.size() - 1; i >= 0; i--) {
+        int removedGroupIdx = removedGroupIdxs.get(i);
+        int raftId = localGroups.get(removedGroupIdx).getId();
         localGroups.remove(removedGroupIdx);
         // each node exactly joins replicationNum groups, so when a group is removed, the node
         // should join a new one
         int thisNodeIdx = nodeRing.indexOf(thisNode);
+
+        // check if this node is to be removed
+        if (thisNodeIdx == -1) {
+          continue;
+        }
+
         // this node must be the last node of the new group
         int headerNodeIdx = thisNodeIdx - (replicationNum - 1);
         headerNodeIdx = headerNodeIdx < 0 ? headerNodeIdx + nodeRing.size() : headerNodeIdx;
         Node header = nodeRing.get(headerNodeIdx);
-        PartitionGroup newGrp = getHeaderGroup(header);
+        PartitionGroup newGrp = getHeaderGroup(new RaftNode(header, raftId));
         localGroups.add(newGrp);
-        result.setNewGroup(newGrp);
       }
 
-      calculateGlobalGroups();
+      globalGroups = calculateGlobalGroups(nodeRing);
 
       // the slots movement is only done logically, the new node itself will pull data from the
       // old node
-      Map<Node, List<Integer>> nodeListMap = retrieveSlots(target);
-      result.setNewSlotOwners(nodeListMap);
-      return result;
+      Map<RaftNode, List<Integer>> raftNodeListMap = slotBalancer.retrieveSlots(target);
+      result.addNewSlotOwners(raftNodeListMap);
+      this.nodeRemovalResult = result;
     }
   }
 
-  private Map<Node, List<Integer>> retrieveSlots(Node target) {
-    Map<Node, List<Integer>> newHolderSlotMap = new HashMap<>();
-    List<Integer> slots = nodeSlotMap.remove(target);
-    for (int i = 0; i < slots.size(); i++) {
-      int slot = slots.get(i);
-      Node newHolder = nodeRing.get(i % nodeRing.size());
-      slotNodes[slot] = newHolder;
-      nodeSlotMap.get(newHolder).add(slot);
-      newHolderSlotMap.computeIfAbsent(newHolder, n -> new ArrayList<>()).add(slot);
-    }
-    return newHolderSlotMap;
+  @Override
+  public NodeRemovalResult getNodeRemovalResult() {
+    return nodeRemovalResult;
   }
 
   @Override
@@ -484,24 +564,46 @@ public class SlotPartitionTable implements PartitionTable {
     // preventing a thread from getting incomplete globalGroups
     synchronized (nodeRing) {
       if (globalGroups == null) {
-        calculateGlobalGroups();
+        globalGroups = calculateGlobalGroups(nodeRing);
       }
       return globalGroups;
     }
   }
 
-  private void calculateGlobalGroups() {
-    globalGroups = new ArrayList<>();
-    for (Node n : getAllNodes()) {
-      globalGroups.add(getHeaderGroup(n));
+  /**
+   * Judge whether the data of slot is held by node
+   *
+   * @param node target node
+   */
+  public boolean judgeHoldSlot(Node node, int slot) {
+    return getHeaderGroup(slotNodes[slot]).contains(node);
+  }
+
+  @Override
+  public List<PartitionGroup> calculateGlobalGroups(List<Node> nodeRing) {
+    List<PartitionGroup> result = new ArrayList<>();
+    for (Node node : nodeRing) {
+      for (int i = 0; i < multiRaftFactor; i++) {
+        result.add(getHeaderGroup(new RaftNode(node, i), nodeRing));
+      }
     }
+    return result;
+  }
+
+  @Override
+  public long getLastMetaLogIndex() {
+    return lastMetaLogIndex;
   }
 
-  public synchronized long getLastLogIndex() {
-    return lastLogIndex;
+  @Override
+  public void setLastMetaLogIndex(long lastMetaLogIndex) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("Set last meta log index of partition table to {}", lastMetaLogIndex);
+    }
+    this.lastMetaLogIndex = Math.max(this.lastMetaLogIndex, lastMetaLogIndex);
   }
 
-  public synchronized void setLastLogIndex(long lastLogIndex) {
-    this.lastLogIndex = Math.max(this.lastLogIndex, lastLogIndex);
+  public RaftNode[] getSlotNodes() {
+    return slotNodes;
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotStrategy.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotStrategy.java
index 765e77b..6d0032c 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotStrategy.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotStrategy.java
@@ -1,5 +1,20 @@
 /*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at      http://www.apache.org/licenses/LICENSE-2.0  Unless required by applicable law or ag [...]
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
  */
 
 package org.apache.iotdb.cluster.partition.slot;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterDataQueryExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterDataQueryExecutor.java
index 8c94a66..0176b72 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterDataQueryExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterDataQueryExecutor.java
@@ -196,7 +196,7 @@ public class ClusterDataQueryExecutor extends RawDataQueryExecutor {
       PartialPath path, Set<String> deviceMeasurements, TSDataType dataType, QueryContext context)
       throws StorageEngineException, QueryProcessException {
     return readerFactory.getReaderByTimestamp(
-        path, deviceMeasurements, dataType, context, queryPlan.isAscending());
+        path, deviceMeasurements, dataType, context, queryPlan.isAscending(), null);
   }
 
   @Override
@@ -286,7 +286,8 @@ public class ClusterDataQueryExecutor extends RawDataQueryExecutor {
                       dataType,
                       context,
                       dataGroupMember,
-                      queryPlan.isAscending());
+                      queryPlan.isAscending(),
+                      null);
 
               if (readerByTimestamp != null) {
                 this.hasLocalReader = true;
@@ -302,7 +303,8 @@ public class ClusterDataQueryExecutor extends RawDataQueryExecutor {
                       null,
                       context,
                       dataGroupMember,
-                      queryPlan.isAscending());
+                      queryPlan.isAscending(),
+                      null);
 
               if (pointReader.hasNextTimeValuePair()) {
                 this.hasLocalReader = true;
@@ -315,8 +317,8 @@ public class ClusterDataQueryExecutor extends RawDataQueryExecutor {
           } else if (endPoint == null) {
             endPoint =
                 new QueryDataSet.EndPoint(
-                    partitionGroup.getHeader().getClientIp(),
-                    partitionGroup.getHeader().getClientPort());
+                    partitionGroup.getHeader().getNode().getClientIp(),
+                    partitionGroup.getHeader().getNode().getClientPort());
           }
         }
       } catch (Exception e) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java
index ca16cfb..bebe5af 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java
@@ -164,7 +164,6 @@ public class ClusterPlanExecutor extends PlanExecutor {
    *     storage group added
    * @param level the max depth to match the pattern, -1 means matching the whole pattern
    * @return the number of paths that match the pattern at given level
-   * @throws MetadataException
    */
   private int getPathCount(Map<String, String> sgPathMap, int level)
       throws MetadataException, CheckConsistencyException {
@@ -181,7 +180,7 @@ public class ClusterPlanExecutor extends PlanExecutor {
         // this node is a member of the group, perform a local query after synchronizing with the
         // leader
         metaGroupMember
-            .getLocalDataMember(partitionGroup.getHeader())
+            .getLocalDataMember(partitionGroup.getHeader(), partitionGroup.getId())
             .syncLeaderWithConsistencyCheck(false);
         int localResult = getLocalPathCount(pathUnderSG, level);
         logger.debug(
@@ -329,15 +328,15 @@ public class ClusterPlanExecutor extends PlanExecutor {
   private List<PartialPath> getLocalNodesList(
       PartitionGroup group, PartialPath schemaPattern, int level)
       throws CheckConsistencyException, MetadataException {
-    Node header = group.getHeader();
-    DataGroupMember localDataMember = metaGroupMember.getLocalDataMember(header);
+    DataGroupMember localDataMember = metaGroupMember.getLocalDataMember(group.getHeader());
     localDataMember.syncLeaderWithConsistencyCheck(false);
     try {
       return IoTDB.metaManager.getNodesList(
           schemaPattern,
           level,
           new SlotSgFilter(
-              ((SlotPartitionTable) metaGroupMember.getPartitionTable()).getNodeSlots(header)));
+              ((SlotPartitionTable) metaGroupMember.getPartitionTable())
+                  .getNodeSlots(group.getHeader())));
     } catch (MetadataException e) {
       logger.error(
           "Cannot not get node list of {}@{} from {} locally", schemaPattern, level, group);
@@ -421,8 +420,8 @@ public class ClusterPlanExecutor extends PlanExecutor {
 
   private Set<String> getLocalChildNodeInNextLevel(PartitionGroup group, PartialPath path)
       throws CheckConsistencyException {
-    Node header = group.getHeader();
-    DataGroupMember localDataMember = metaGroupMember.getLocalDataMember(header);
+    DataGroupMember localDataMember =
+        metaGroupMember.getLocalDataMember(group.getHeader(), group.getId());
     localDataMember.syncLeaderWithConsistencyCheck(false);
     try {
       return IoTDB.metaManager.getChildNodeInNextLevel(path);
@@ -531,8 +530,7 @@ public class ClusterPlanExecutor extends PlanExecutor {
 
   private Set<String> getLocalNextChildren(PartitionGroup group, PartialPath path)
       throws CheckConsistencyException {
-    Node header = group.getHeader();
-    DataGroupMember localDataMember = metaGroupMember.getLocalDataMember(header);
+    DataGroupMember localDataMember = metaGroupMember.getLocalDataMember(group.getHeader());
     localDataMember.syncLeaderWithConsistencyCheck(false);
     try {
       return IoTDB.metaManager.getChildNodePathInNextLevel(path);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java
index fd10f77..cc06568 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java
@@ -19,9 +19,14 @@
 
 package org.apache.iotdb.cluster.query;
 
+import org.apache.iotdb.cluster.exception.UnknownLogTypeException;
 import org.apache.iotdb.cluster.exception.UnsupportedPlanException;
+import org.apache.iotdb.cluster.log.Log;
+import org.apache.iotdb.cluster.log.logtypes.AddNodeLog;
+import org.apache.iotdb.cluster.log.logtypes.RemoveNodeLog;
 import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.partition.PartitionTable;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.utils.PartitionUtils;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
@@ -39,6 +44,7 @@ import org.apache.iotdb.db.qp.physical.sys.CountPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateMultiTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.LogPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowChildPathsPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowPlan.ShowContentType;
 import org.apache.iotdb.db.service.IoTDB;
@@ -51,6 +57,7 @@ import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -112,7 +119,7 @@ public class ClusterPlanRouter {
   }
 
   public Map<PhysicalPlan, PartitionGroup> splitAndRoutePlan(PhysicalPlan plan)
-      throws UnsupportedPlanException, MetadataException {
+      throws UnsupportedPlanException, MetadataException, UnknownLogTypeException {
     if (plan instanceof InsertRowsPlan) {
       return splitAndRoutePlan((InsertRowsPlan) plan);
     } else if (plan instanceof InsertTabletPlan) {
@@ -144,6 +151,29 @@ public class ClusterPlanRouter {
     throw new UnsupportedPlanException(plan);
   }
 
+  public Map<PhysicalPlan, PartitionGroup> splitAndRouteChangeMembershipLog(Log log) {
+    Map<PhysicalPlan, PartitionGroup> result = new HashMap<>();
+    LogPlan plan = new LogPlan(log.serialize());
+    List<Node> oldRing = new ArrayList<>(partitionTable.getAllNodes());
+    if (log instanceof AddNodeLog) {
+      oldRing.remove(((AddNodeLog) log).getNewNode());
+    } else if (log instanceof RemoveNodeLog) {
+      if (!oldRing.contains(((RemoveNodeLog) log).getRemovedNode())) {
+        oldRing.add(((RemoveNodeLog) log).getRemovedNode());
+        oldRing.sort(Comparator.comparingInt(Node::getNodeIdentifier));
+      }
+    }
+    for (PartitionGroup partitionGroup : partitionTable.calculateGlobalGroups(oldRing)) {
+      // It doesn't need to notify the data group which will be removed from cluster.
+      if (log instanceof RemoveNodeLog
+          && partitionGroup.getHeader().getNode().equals(((RemoveNodeLog) log).getRemovedNode())) {
+        continue;
+      }
+      result.put(new LogPlan(plan), partitionGroup);
+    }
+    return result;
+  }
+
   private Map<PhysicalPlan, PartitionGroup> splitAndRoutePlan(InsertRowPlan plan)
       throws MetadataException {
     PartitionGroup partitionGroup =
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/LocalQueryExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/LocalQueryExecutor.java
index 1392f12..d35e200 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/LocalQueryExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/LocalQueryExecutor.java
@@ -22,6 +22,8 @@ package org.apache.iotdb.cluster.query;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.exception.ReaderNotFoundException;
 import org.apache.iotdb.cluster.metadata.CMManager;
+import org.apache.iotdb.cluster.metadata.MetaPuller;
+import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.partition.slot.SlotPartitionTable;
 import org.apache.iotdb.cluster.query.filter.SlotTsFileFilter;
 import org.apache.iotdb.cluster.query.manage.ClusterQueryManager;
@@ -35,9 +37,11 @@ import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.PreviousFillRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSchemaRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSchemaResp;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.SingleSeriesQueryRequest;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
 import org.apache.iotdb.cluster.utils.ClusterQueryUtils;
+import org.apache.iotdb.cluster.utils.ClusterUtils;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
@@ -83,6 +87,7 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -252,7 +257,8 @@ public class LocalQueryExecutor {
             valueFilter,
             queryContext,
             dataGroupMember,
-            request.ascending);
+            request.ascending,
+            request.requiredSlots);
 
     // if the reader contains no data, send a special id of -1 to prevent the requester from
     // meaninglessly fetching data
@@ -402,9 +408,7 @@ public class LocalQueryExecutor {
     // the measurements in them are the full paths.
     List<String> prefixPaths = request.getPrefixPaths();
     List<TimeseriesSchema> timeseriesSchemas = new ArrayList<>();
-    for (String prefixPath : prefixPaths) {
-      getCMManager().collectTimeseriesSchema(prefixPath, timeseriesSchemas);
-    }
+    collectTimeseriesSchema(prefixPaths, timeseriesSchemas);
     if (logger.isDebugEnabled()) {
       logger.debug(
           "{}: Collected {} schemas for {} and other {} paths",
@@ -437,8 +441,8 @@ public class LocalQueryExecutor {
    *
    * @param request
    */
-  public PullSchemaResp queryMeasurementSchema(PullSchemaRequest request)
-      throws CheckConsistencyException, IllegalPathException {
+  public PullSchemaResp queryMeasurementSchema(PullSchemaRequest request) // pullMeasurementSchemas
+      throws CheckConsistencyException, MetadataException {
     // try to synchronize with the leader first in case that some schema logs are accepted but
     // not committed yet
     dataGroupMember.syncLeaderWithConsistencyCheck(false);
@@ -447,9 +451,8 @@ public class LocalQueryExecutor {
     // the measurements in them are the full paths.
     List<String> prefixPaths = request.getPrefixPaths();
     List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
-    for (String prefixPath : prefixPaths) {
-      getCMManager().collectSeries(new PartialPath(prefixPath), measurementSchemas);
-    }
+
+    collectSeries(prefixPaths, measurementSchemas);
     if (logger.isDebugEnabled()) {
       logger.debug(
           "{}: Collected {} schemas for {} and other {} paths",
@@ -475,6 +478,82 @@ public class LocalQueryExecutor {
     return resp;
   }
 
+  private void collectSeries(List<String> prefixPaths, List<IMeasurementSchema> measurementSchemas)
+      throws MetadataException {
+    // Due to add/remove node, some slots may in the state of PULLING, which will not contains the
+    // corresponding schemas.
+    // In this case, we need to pull series from previous holder.
+    Map<PartitionGroup, List<PartialPath>> prePartitionGroupPathMap = new HashMap<>();
+
+    RaftNode header = dataGroupMember.getHeader();
+    Map<Integer, PartitionGroup> slotPreviousHolderMap =
+        ((SlotPartitionTable) dataGroupMember.getMetaGroupMember().getPartitionTable())
+            .getPreviousNodeMap()
+            .get(header);
+
+    for (String prefixPath : prefixPaths) {
+      int slot =
+          ClusterUtils.getSlotByPathTimeWithSync(
+              new PartialPath(prefixPath), dataGroupMember.getMetaGroupMember());
+      if (dataGroupMember.getSlotManager().checkSlotInMetaMigrationStatus(slot)
+          && slotPreviousHolderMap.containsKey(slot)) {
+        prePartitionGroupPathMap
+            .computeIfAbsent(slotPreviousHolderMap.get(slot), s -> new ArrayList<>())
+            .add(new PartialPath(prefixPath));
+      } else {
+        getCMManager().collectSeries(new PartialPath(prefixPath), measurementSchemas);
+      }
+    }
+
+    if (prePartitionGroupPathMap.isEmpty()) {
+      return;
+    }
+    for (Map.Entry<PartitionGroup, List<PartialPath>> partitionGroupListEntry :
+        prePartitionGroupPathMap.entrySet()) {
+      PartitionGroup partitionGroup = partitionGroupListEntry.getKey();
+      List<PartialPath> paths = partitionGroupListEntry.getValue();
+      MetaPuller.getInstance().pullMeasurementSchemas(partitionGroup, paths, measurementSchemas);
+    }
+  }
+
+  private void collectTimeseriesSchema(
+      List<String> prefixPaths, List<TimeseriesSchema> timeseriesSchemas) throws MetadataException {
+    // Due to add/remove node, some slots may in the state of PULLING, which will not contains the
+    // corresponding schemas.
+    // In this case, we need to pull series from previous holder.
+    Map<PartitionGroup, List<String>> prePartitionGroupPathMap = new HashMap<>();
+
+    RaftNode header = dataGroupMember.getHeader();
+    Map<Integer, PartitionGroup> slotPreviousHolderMap =
+        ((SlotPartitionTable) dataGroupMember.getMetaGroupMember().getPartitionTable())
+            .getPreviousNodeMap()
+            .get(header);
+
+    for (String prefixPath : prefixPaths) {
+      int slot =
+          ClusterUtils.getSlotByPathTimeWithSync(
+              new PartialPath(prefixPath), dataGroupMember.getMetaGroupMember());
+      if (dataGroupMember.getSlotManager().checkSlotInMetaMigrationStatus(slot)
+          && slotPreviousHolderMap.containsKey(slot)) {
+        prePartitionGroupPathMap
+            .computeIfAbsent(slotPreviousHolderMap.get(slot), s -> new ArrayList<>())
+            .add(prefixPath);
+      } else {
+        getCMManager().collectTimeseriesSchema(prefixPath, timeseriesSchemas);
+      }
+    }
+
+    if (prePartitionGroupPathMap.isEmpty()) {
+      return;
+    }
+    for (Map.Entry<PartitionGroup, List<String>> partitionGroupListEntry :
+        prePartitionGroupPathMap.entrySet()) {
+      PartitionGroup partitionGroup = partitionGroupListEntry.getKey();
+      List<String> paths = partitionGroupListEntry.getValue();
+      MetaPuller.getInstance().pullTimeSeriesSchemas(partitionGroup, paths, timeseriesSchemas);
+    }
+  }
+
   /**
    * Create an IReaderByTime of a path, register it in the query manager to get a reader id for it
    * and send the id back to the requester. If the reader does not have any data, an id of -1 will
@@ -515,7 +594,13 @@ public class LocalQueryExecutor {
         queryContext.getQueryId());
     IReaderByTimestamp readerByTimestamp =
         readerFactory.getReaderByTimestamp(
-            path, deviceMeasurements, dataType, queryContext, dataGroupMember, request.ascending);
+            path,
+            deviceMeasurements,
+            dataType,
+            queryContext,
+            dataGroupMember,
+            request.ascending,
+            request.requiredSlots);
     if (readerByTimestamp != null) {
       long readerId = queryManager.registerReaderByTime(readerByTimestamp);
       queryContext.registerLocalReader(readerId);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/RemoteQueryContext.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/RemoteQueryContext.java
index 1ecf758..dee731a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/RemoteQueryContext.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/RemoteQueryContext.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.cluster.query;
 
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.db.query.context.QueryContext;
 
 import java.util.HashMap;
@@ -30,7 +31,7 @@ import java.util.concurrent.ConcurrentSkipListSet;
 
 public class RemoteQueryContext extends QueryContext {
   /** The remote nodes that are queried in this query, grouped by the header nodes. */
-  private Map<Node, Set<Node>> queriedNodesMap = new HashMap<>();
+  private Map<RaftNode, Set<Node>> queriedNodesMap = new HashMap<>();
   /** The readers constructed locally to respond a remote query. */
   private Set<Long> localReaderIds = new ConcurrentSkipListSet<>();
 
@@ -45,7 +46,7 @@ public class RemoteQueryContext extends QueryContext {
     super(jobId, debug);
   }
 
-  public void registerRemoteNode(Node node, Node header) {
+  public void registerRemoteNode(Node node, RaftNode header) {
     queriedNodesMap.computeIfAbsent(header, n -> new HashSet<>()).add(node);
   }
 
@@ -65,7 +66,7 @@ public class RemoteQueryContext extends QueryContext {
     return localGroupByExecutorIds;
   }
 
-  public Map<Node, Set<Node>> getQueriedNodesMap() {
+  public Map<RaftNode, Set<Node>> getQueriedNodesMap() {
     return queriedNodesMap;
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/aggregate/ClusterAggregateExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/aggregate/ClusterAggregateExecutor.java
index 1104cae..a007ee9 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/aggregate/ClusterAggregateExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/aggregate/ClusterAggregateExecutor.java
@@ -97,6 +97,7 @@ public class ClusterAggregateExecutor extends AggregationExecutor {
         dataQueryPlan.getAllMeasurementsInDevice(path.getDevice()),
         dataType,
         context,
-        dataQueryPlan.isAscending());
+        dataQueryPlan.isAscending(),
+        null);
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/aggregate/ClusterAggregator.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/aggregate/ClusterAggregator.java
index a14b0cf..80aa81e 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/aggregate/ClusterAggregator.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/aggregate/ClusterAggregator.java
@@ -154,7 +154,8 @@ public class ClusterAggregator {
           ascending);
     } else {
       // perform the aggregations locally
-      DataGroupMember dataMember = metaGroupMember.getLocalDataMember(partitionGroup.getHeader());
+      DataGroupMember dataMember =
+          metaGroupMember.getLocalDataMember(partitionGroup.getHeader(), partitionGroup.getId());
       LocalQueryExecutor localQueryExecutor = new LocalQueryExecutor(dataMember);
       try {
         logger.debug(
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/fill/ClusterPreviousFill.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/fill/ClusterPreviousFill.java
index 33274e3..08689e7 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/fill/ClusterPreviousFill.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/fill/ClusterPreviousFill.java
@@ -154,7 +154,8 @@ public class ClusterPreviousFill extends PreviousFill {
       QueryContext context,
       PartitionGroup group,
       PreviousFillHandler fillHandler) {
-    DataGroupMember localDataMember = metaGroupMember.getLocalDataMember(group.getHeader());
+    DataGroupMember localDataMember =
+        metaGroupMember.getLocalDataMember(group.getHeader(), group.getId());
     try {
       fillHandler.onComplete(
           localDataMember
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/filter/SlotTsFileFilter.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/filter/SlotTsFileFilter.java
index 1fef19e..1da0a92 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/filter/SlotTsFileFilter.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/filter/SlotTsFileFilter.java
@@ -29,23 +29,29 @@ import org.apache.iotdb.tsfile.utils.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 public class SlotTsFileFilter implements TsFileFilter {
 
   private static final Logger logger = LoggerFactory.getLogger(SlotTsFileFilter.class);
-  private List<Integer> slots;
+  private Set<Integer> slots;
 
-  public SlotTsFileFilter(List<Integer> slots) {
+  public SlotTsFileFilter(Set<Integer> slots) {
     this.slots = slots;
   }
 
+  public SlotTsFileFilter(List<Integer> slots) {
+    this.slots = new HashSet<>(slots);
+  }
+
   @Override
   public boolean fileNotSatisfy(TsFileResource resource) {
     return fileNotInSlots(resource, slots);
   }
 
-  private static boolean fileNotInSlots(TsFileResource resource, List<Integer> nodeSlots) {
+  private static boolean fileNotInSlots(TsFileResource resource, Set<Integer> nodeSlots) {
     Pair<String, Long> sgNameAndPartitionIdPair =
         FilePathUtils.getLogicalSgNameAndTimePartitionIdPair(resource);
     int slot =
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/ClusterGroupByVFilterDataSet.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/ClusterGroupByVFilterDataSet.java
index 02e9f3f..b789d6f 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/ClusterGroupByVFilterDataSet.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/ClusterGroupByVFilterDataSet.java
@@ -77,6 +77,7 @@ public class ClusterGroupByVFilterDataSet extends GroupByWithValueFilterDataSet
         dataQueryPlan.getAllMeasurementsInDevice(path.getDevice()),
         dataType,
         context,
-        dataQueryPlan.isAscending());
+        dataQueryPlan.isAscending(),
+        null);
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/RemoteGroupByExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/RemoteGroupByExecutor.java
index 02df747..b2d52ac 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/RemoteGroupByExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/RemoteGroupByExecutor.java
@@ -24,8 +24,10 @@ import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
+import org.apache.iotdb.cluster.utils.ClientUtils;
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
 import org.apache.iotdb.db.query.dataset.groupby.GroupByExecutor;
 import org.apache.iotdb.db.utils.SerializeUtils;
@@ -47,12 +49,12 @@ public class RemoteGroupByExecutor implements GroupByExecutor {
   private long executorId;
   private MetaGroupMember metaGroupMember;
   private Node source;
-  private Node header;
+  private RaftNode header;
 
   private List<AggregateResult> results = new ArrayList<>();
 
   public RemoteGroupByExecutor(
-      long executorId, MetaGroupMember metaGroupMember, Node source, Node header) {
+      long executorId, MetaGroupMember metaGroupMember, Node source, RaftNode header) {
     this.executorId = executorId;
     this.metaGroupMember = metaGroupMember;
     this.source = source;
@@ -88,7 +90,6 @@ public class RemoteGroupByExecutor implements GroupByExecutor {
             metaGroupMember
                 .getClientProvider()
                 .getSyncDataClient(source, RaftServer.getReadOperationTimeoutMS())) {
-
           aggrBuffers =
               syncDataClient.getGroupByResult(header, executorId, curStartTime, curEndTime);
         }
@@ -129,14 +130,13 @@ public class RemoteGroupByExecutor implements GroupByExecutor {
             SyncClientAdaptor.peekNextNotNullValue(
                 client, header, executorId, nextStartTime, nextEndTime);
       } else {
-        try (SyncDataClient syncDataClient =
+        SyncDataClient syncDataClient =
             metaGroupMember
                 .getClientProvider()
-                .getSyncDataClient(source, RaftServer.getReadOperationTimeoutMS())) {
-
-          aggrBuffer =
-              syncDataClient.peekNextNotNullValue(header, executorId, nextStartTime, nextEndTime);
-        }
+                .getSyncDataClient(source, RaftServer.getReadOperationTimeoutMS());
+        aggrBuffer =
+            syncDataClient.peekNextNotNullValue(header, executorId, nextStartTime, nextEndTime);
+        ClientUtils.putBackSyncClient(syncDataClient);
       }
     } catch (TException e) {
       throw new IOException(e);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/last/ClusterLastQueryExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/last/ClusterLastQueryExecutor.java
index d5ec324..5ef7327 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/last/ClusterLastQueryExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/last/ClusterLastQueryExecutor.java
@@ -188,7 +188,8 @@ public class ClusterLastQueryExecutor extends LastQueryExecutor {
     private List<Pair<Boolean, TimeValuePair>> calculateSeriesLastLocally(
         PartitionGroup group, List<PartialPath> seriesPaths, QueryContext context)
         throws StorageEngineException, QueryProcessException, IOException {
-      DataGroupMember localDataMember = metaGroupMember.getLocalDataMember(group.getHeader());
+      DataGroupMember localDataMember =
+          metaGroupMember.getLocalDataMember(group.getHeader(), group.getId());
       try {
         localDataMember.syncLeaderWithConsistencyCheck(false);
       } catch (CheckConsistencyException e) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
index 0f83359..f379da5 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
@@ -42,6 +42,7 @@ import org.apache.iotdb.cluster.query.reader.mult.RemoteMultSeriesReader;
 import org.apache.iotdb.cluster.rpc.thrift.GroupByRequest;
 import org.apache.iotdb.cluster.rpc.thrift.MultSeriesQueryRequest;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.SingleSeriesQueryRequest;
 import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
@@ -56,6 +57,7 @@ import org.apache.iotdb.db.query.aggregation.AggregationType;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.query.dataset.groupby.GroupByExecutor;
+import org.apache.iotdb.db.query.externalsort.adapter.ByTimestampReaderAdapter;
 import org.apache.iotdb.db.query.factory.AggregateResultFactory;
 import org.apache.iotdb.db.query.reader.series.IReaderByTimestamp;
 import org.apache.iotdb.db.query.reader.series.ManagedSeriesReader;
@@ -63,6 +65,7 @@ import org.apache.iotdb.db.query.reader.series.SeriesRawDataBatchReader;
 import org.apache.iotdb.db.query.reader.series.SeriesRawDataPointReader;
 import org.apache.iotdb.db.query.reader.series.SeriesReader;
 import org.apache.iotdb.db.query.reader.series.SeriesReaderByTimestamp;
+import org.apache.iotdb.db.query.reader.universal.PriorityMergeReader;
 import org.apache.iotdb.db.utils.SerializeUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -82,8 +85,10 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 
 @SuppressWarnings("java:S107")
@@ -109,7 +114,8 @@ public class ClusterReaderFactory {
       Set<String> deviceMeasurements,
       TSDataType dataType,
       QueryContext context,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     // get all data groups
     List<PartitionGroup> partitionGroups;
@@ -129,9 +135,16 @@ public class ClusterReaderFactory {
       // query each group to get a reader in that group
       IReaderByTimestamp readerByTimestamp =
           getSeriesReaderByTime(
-              partitionGroup, path, deviceMeasurements, context, dataType, ascending);
+              partitionGroup,
+              path,
+              deviceMeasurements,
+              context,
+              dataType,
+              ascending,
+              requiredSlots);
       readers.add(readerByTimestamp);
     }
+    // merge the readers
     return new MergedReaderByTime(readers);
   }
 
@@ -146,12 +159,13 @@ public class ClusterReaderFactory {
       Set<String> deviceMeasurements,
       QueryContext context,
       TSDataType dataType,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     if (partitionGroup.contains(metaGroupMember.getThisNode())) {
       // the target storage group contains this node, perform a local query
       DataGroupMember dataGroupMember =
-          metaGroupMember.getLocalDataMember(partitionGroup.getHeader());
+          metaGroupMember.getLocalDataMember(partitionGroup.getHeader(), partitionGroup.getId());
       if (logger.isDebugEnabled()) {
         logger.debug(
             "{}: creating a local reader for {}#{}",
@@ -160,10 +174,10 @@ public class ClusterReaderFactory {
             context.getQueryId());
       }
       return getReaderByTimestamp(
-          path, deviceMeasurements, dataType, context, dataGroupMember, ascending);
+          path, deviceMeasurements, dataType, context, dataGroupMember, ascending, requiredSlots);
     } else {
       return getRemoteReaderByTimestamp(
-          path, deviceMeasurements, dataType, partitionGroup, context, ascending);
+          path, deviceMeasurements, dataType, partitionGroup, context, ascending, requiredSlots);
     }
   }
 
@@ -178,11 +192,20 @@ public class ClusterReaderFactory {
       TSDataType dataType,
       PartitionGroup partitionGroup,
       QueryContext context,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException {
     SingleSeriesQueryRequest request =
         constructSingleQueryRequest(
-            null, null, dataType, path, deviceMeasurements, partitionGroup, context, ascending);
+            null,
+            null,
+            dataType,
+            path,
+            deviceMeasurements,
+            partitionGroup,
+            context,
+            ascending,
+            requiredSlots);
 
     // reorder the nodes by their communication delays
     List<Node> reorderedNodes = QueryCoordinator.getINSTANCE().reorderNodes(partitionGroup);
@@ -314,7 +337,8 @@ public class ClusterReaderFactory {
                 valueFilter,
                 context,
                 dataGroupMember,
-                ascending);
+                ascending,
+                null);
         partialPathPointReaderMap.put(PartialPath.getExactFullPath(partialPath), seriesPointReader);
       }
 
@@ -365,7 +389,12 @@ public class ClusterReaderFactory {
         metaGroupMember.getName(),
         path,
         partitionGroups.size());
-    ManagedMergeReader mergeReader = new ManagedMergeReader(dataType);
+    PriorityMergeReader mergeReader;
+    if (ascending) {
+      mergeReader = new ManagedPriorityMergeReader(dataType);
+    } else {
+      mergeReader = new ManagedDescPriorityMergeReader(dataType);
+    }
     try {
       // build a reader for each group and merge them
       for (PartitionGroup partitionGroup : partitionGroups) {
@@ -378,13 +407,16 @@ public class ClusterReaderFactory {
                 valueFilter,
                 context,
                 dataType,
-                ascending);
+                ascending,
+                null);
         mergeReader.addReader(seriesReader, 0);
       }
     } catch (IOException | QueryProcessException e) {
       throw new StorageEngineException(e);
     }
-    return mergeReader;
+    // The instance of merge reader is either ManagedPriorityMergeReader or
+    // ManagedDescPriorityMergeReader, which is safe to cast type.
+    return (ManagedSeriesReader) mergeReader;
   }
 
   /**
@@ -403,7 +435,8 @@ public class ClusterReaderFactory {
       Filter valueFilter,
       QueryContext context,
       TSDataType dataType,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws IOException, StorageEngineException, QueryProcessException {
     if (partitionGroup.contains(metaGroupMember.getThisNode())) {
       // the target storage group contains this node, perform a local query
@@ -422,7 +455,8 @@ public class ClusterReaderFactory {
               valueFilter,
               context,
               dataGroupMember,
-              ascending);
+              ascending,
+              requiredSlots);
       if (logger.isDebugEnabled()) {
         logger.debug(
             "{}: creating a local reader for {}#{} of {}, empty: {}",
@@ -442,7 +476,8 @@ public class ClusterReaderFactory {
           deviceMeasurements,
           partitionGroup,
           context,
-          ascending);
+          ascending,
+          requiredSlots);
     }
   }
 
@@ -466,7 +501,8 @@ public class ClusterReaderFactory {
       Filter valueFilter,
       QueryContext context,
       DataGroupMember dataGroupMember,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     // pull the newest data
     try {
@@ -483,7 +519,8 @@ public class ClusterReaderFactory {
             valueFilter,
             context,
             dataGroupMember.getHeader(),
-            ascending));
+            ascending,
+            requiredSlots));
   }
 
   /**
@@ -505,12 +542,20 @@ public class ClusterReaderFactory {
       Filter timeFilter,
       Filter valueFilter,
       QueryContext context,
-      Node header,
-      boolean ascending)
+      RaftNode header,
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     ClusterQueryUtils.checkPathExistence(path);
-    List<Integer> nodeSlots =
-        ((SlotPartitionTable) metaGroupMember.getPartitionTable()).getNodeSlots(header);
+    // If requiredSlots is null, it means that this node should provide data of all slots about
+    // required paths.
+    if (requiredSlots == null) {
+      List<Integer> nodeSlots =
+          ((SlotPartitionTable) metaGroupMember.getPartitionTable()).getNodeSlots(header);
+      requiredSlots = new HashSet<>(nodeSlots);
+    }
+    // If requiredSlots is not null, it means that this node should provide partial data as previous
+    // holder, in order to assist the new holder to read the complete data.
     QueryDataSource queryDataSource =
         QueryResourceManager.getInstance().getQueryDataSource(path, context, timeFilter);
     valueFilter = queryDataSource.updateFilterUsingTTL(valueFilter);
@@ -522,7 +567,7 @@ public class ClusterReaderFactory {
         queryDataSource,
         timeFilter,
         valueFilter,
-        new SlotTsFileFilter(nodeSlots),
+        new SlotTsFileFilter(requiredSlots),
         ascending);
   }
 
@@ -556,7 +601,7 @@ public class ClusterReaderFactory {
             context,
             ascending);
 
-    // reorder the nodes such that the nodes that suit the query best (have lowest latenct or
+    // reorder the nodes such that the nodes that suit the query best (have lowest latency or
     // highest throughput) will be put to the front
     List<Node> orderedNodes = QueryCoordinator.getINSTANCE().reorderNodes(partitionGroup);
 
@@ -602,7 +647,8 @@ public class ClusterReaderFactory {
       Set<String> deviceMeasurements,
       PartitionGroup partitionGroup,
       QueryContext context,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException {
     SingleSeriesQueryRequest request =
         constructSingleQueryRequest(
@@ -613,7 +659,8 @@ public class ClusterReaderFactory {
             deviceMeasurements,
             partitionGroup,
             context,
-            ascending);
+            ascending,
+            requiredSlots);
 
     // reorder the nodes such that the nodes that suit the query best (have lowest latenct or
     // highest throughput) will be put to the front
@@ -694,7 +741,8 @@ public class ClusterReaderFactory {
       Set<String> deviceMeasurements,
       PartitionGroup partitionGroup,
       QueryContext context,
-      boolean ascending) {
+      boolean ascending,
+      Set<Integer> requiredSlots) {
     SingleSeriesQueryRequest request = new SingleSeriesQueryRequest();
     if (timeFilter != null) {
       request.setTimeFilterBytes(SerializeUtils.serializeFilter(timeFilter));
@@ -709,6 +757,7 @@ public class ClusterReaderFactory {
     request.setDataTypeOrdinal(dataType.ordinal());
     request.setDeviceMeasurements(deviceMeasurements);
     request.setAscending(ascending);
+    request.setRequiredSlots(requiredSlots);
     return request;
   }
 
@@ -787,7 +836,7 @@ public class ClusterReaderFactory {
     if (partitionGroup.contains(metaGroupMember.getThisNode())) {
       // the target storage group contains this node, perform a local query
       DataGroupMember dataGroupMember =
-          metaGroupMember.getLocalDataMember(partitionGroup.getHeader());
+          metaGroupMember.getLocalDataMember(partitionGroup.getHeader(), partitionGroup.getId());
       LocalQueryExecutor localQueryExecutor = new LocalQueryExecutor(dataGroupMember);
       logger.debug(
           "{}: creating a local group by executor for {}#{}",
@@ -910,6 +959,30 @@ public class ClusterReaderFactory {
     return executorId;
   }
 
+  public IBatchReader getSeriesBatchReader(
+      PartialPath path,
+      Set<String> allSensors,
+      TSDataType dataType,
+      Filter timeFilter,
+      Filter valueFilter,
+      QueryContext context,
+      DataGroupMember dataGroupMember,
+      boolean ascending,
+      Set<Integer> requiredSlots)
+      throws StorageEngineException, QueryProcessException, IOException {
+    return getSeriesBatchReader(
+        path,
+        allSensors,
+        dataType,
+        timeFilter,
+        valueFilter,
+        context,
+        dataGroupMember,
+        ascending,
+        requiredSlots,
+        true);
+  }
+
   /**
    * Create an IBatchReader of "path" with “timeFilter” and "valueFilter". A synchronization with
    * the leader will be performed according to consistency level
@@ -930,29 +1003,89 @@ public class ClusterReaderFactory {
       Filter valueFilter,
       QueryContext context,
       DataGroupMember dataGroupMember,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots,
+      boolean syncLeader)
       throws StorageEngineException, QueryProcessException, IOException {
-    // pull the newest data
-    try {
-      dataGroupMember.syncLeaderWithConsistencyCheck(false);
-    } catch (CheckConsistencyException e) {
-      throw new StorageEngineException(e);
+    if (syncLeader) {
+      // pull the newest data
+      try {
+        dataGroupMember.syncLeaderWithConsistencyCheck(false);
+      } catch (CheckConsistencyException e) {
+        throw new StorageEngineException(e);
+      }
     }
 
-    SeriesReader seriesReader =
-        getSeriesReader(
-            path,
-            allSensors,
-            dataType,
-            timeFilter,
-            valueFilter,
-            context,
-            dataGroupMember.getHeader(),
-            ascending);
-    if (seriesReader.isEmpty()) {
-      return null;
+    // find the groups that should be queried due to data migration.
+    Map<PartitionGroup, Set<Integer>> holderSlotMap = dataGroupMember.getPreviousHolderSlotMap();
+
+    // If requiredSlots is not null, it means that this data group is the previous holder of
+    // required slots, which is no need to merge other resource,
+    if (requiredSlots == null && !holderSlotMap.isEmpty()) {
+      // merge remote reader and local reader
+      PriorityMergeReader mergeReader;
+      if (ascending) {
+        mergeReader = new ManagedPriorityMergeReader(dataType);
+      } else {
+        mergeReader = new ManagedDescPriorityMergeReader(dataType);
+      }
+
+      // add local reader
+      IPointReader seriesPointReader =
+          getSeriesPointReader(
+              path,
+              allSensors,
+              dataType,
+              timeFilter,
+              valueFilter,
+              context,
+              dataGroupMember,
+              ascending,
+              null);
+      mergeReader.addReader(seriesPointReader, 1);
+
+      // add previous holder reader due to in the stage of data migration
+      logger.debug(
+          "{}: Sending data query of {} to {} groups due to data is in the state of data migration",
+          metaGroupMember.getName(),
+          path,
+          holderSlotMap.size());
+      for (Entry<PartitionGroup, Set<Integer>> entry : holderSlotMap.entrySet()) {
+        IPointReader seriesReader =
+            getSeriesReader(
+                entry.getKey(),
+                path,
+                allSensors,
+                timeFilter,
+                valueFilter,
+                context,
+                dataType,
+                ascending,
+                entry.getValue());
+        mergeReader.addReader(seriesReader, 0);
+      }
+
+      // The instance of merge reader is either ManagedPriorityMergeReader or
+      // ManagedDescPriorityMergeReader, which is safe to cast type.
+      return (IBatchReader) mergeReader;
+    } else {
+      // just local reader is enough
+      SeriesReader seriesReader =
+          getSeriesReader(
+              path,
+              allSensors,
+              dataType,
+              timeFilter,
+              valueFilter,
+              context,
+              dataGroupMember.getHeader(),
+              ascending,
+              requiredSlots);
+      if (seriesReader.isEmpty()) {
+        return null;
+      }
+      return new SeriesRawDataBatchReader(seriesReader);
     }
-    return new SeriesRawDataBatchReader(seriesReader);
   }
 
   /**
@@ -976,7 +1109,7 @@ public class ClusterReaderFactory {
       QueryContext context,
       DataGroupMember dataGroupMember,
       boolean ascending)
-      throws StorageEngineException, QueryProcessException {
+      throws StorageEngineException, QueryProcessException, IOException {
     // pull the newest data
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
@@ -988,18 +1121,19 @@ public class ClusterReaderFactory {
 
     for (int i = 0; i < paths.size(); i++) {
       PartialPath partialPath = paths.get(i);
-      SeriesReader seriesReader =
-          getSeriesReader(
+      IBatchReader batchReader =
+          getSeriesBatchReader(
               partialPath,
               allSensors.get(partialPath.getFullPath()),
               dataTypes.get(i),
               timeFilter,
               valueFilter,
               context,
-              dataGroupMember.getHeader(),
-              ascending);
-      partialPathBatchReaderMap.put(
-          PartialPath.getExactFullPath(partialPath), new SeriesRawDataBatchReader(seriesReader));
+              dataGroupMember,
+              ascending,
+              null,
+              false);
+      partialPathBatchReaderMap.put(PartialPath.getExactFullPath(partialPath), batchReader);
     }
     return new MultBatchReader(partialPathBatchReaderMap);
   }
@@ -1020,30 +1154,91 @@ public class ClusterReaderFactory {
       TSDataType dataType,
       QueryContext context,
       DataGroupMember dataGroupMember,
-      boolean ascending)
+      boolean ascending,
+      Set<Integer> requiredSlots)
       throws StorageEngineException, QueryProcessException {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
     } catch (CheckConsistencyException e) {
       throw new StorageEngineException(e);
     }
-    SeriesReader seriesReader =
-        getSeriesReader(
-            path,
-            allSensors,
-            dataType,
-            TimeFilter.gtEq(Long.MIN_VALUE),
-            null,
-            context,
-            dataGroupMember.getHeader(),
-            ascending);
+
+    // find the groups that should be queried due to data migration.
+    // when a slot is in the status of PULLING or PULLING_WRITABLE, the read of it should merge
+    // result to guarantee integrity.
+    Map<PartitionGroup, Set<Integer>> holderSlotMap = dataGroupMember.getPreviousHolderSlotMap();
     try {
-      if (seriesReader.isEmpty()) {
-        return null;
+      // If requiredSlots is not null, it means that this data group is the previous holder of
+      // required slots, which is no need to merge other resource,
+      if (requiredSlots == null && !holderSlotMap.isEmpty()) {
+        // merge remote reader and local reader
+        PriorityMergeReader mergeReader;
+        if (ascending) {
+          mergeReader = new ManagedPriorityMergeReader(dataType);
+        } else {
+          mergeReader = new ManagedDescPriorityMergeReader(dataType);
+        }
+
+        // add local reader
+        IPointReader seriesPointReader =
+            getSeriesPointReader(
+                path,
+                allSensors,
+                dataType,
+                TimeFilter.gtEq(Long.MIN_VALUE),
+                null,
+                context,
+                dataGroupMember,
+                ascending,
+                null);
+        mergeReader.addReader(seriesPointReader, 1);
+
+        // add previous holder reader due to in the stage of data migration
+        logger.debug(
+            "{}: Sending data query of {} to {} groups due to data is in the state of data migration",
+            metaGroupMember.getName(),
+            path,
+            holderSlotMap.size());
+        for (Entry<PartitionGroup, Set<Integer>> entry : holderSlotMap.entrySet()) {
+          IPointReader seriesReader =
+              getSeriesReader(
+                  entry.getKey(),
+                  path,
+                  allSensors,
+                  TimeFilter.gtEq(Long.MIN_VALUE),
+                  null,
+                  context,
+                  dataType,
+                  ascending,
+                  entry.getValue());
+          mergeReader.addReader(seriesReader, 0);
+        }
+
+        // The instance of merge reader is either ManagedPriorityMergeReader or
+        // ManagedDescPriorityMergeReader, which is safe to cast type.
+        return new ByTimestampReaderAdapter(mergeReader);
+      } else {
+        // just local reader is enough
+        SeriesReader seriesReader =
+            getSeriesReader(
+                path,
+                allSensors,
+                dataType,
+                TimeFilter.gtEq(Long.MIN_VALUE),
+                null,
+                context,
+                dataGroupMember.getHeader(),
+                ascending,
+                requiredSlots);
+
+        if (seriesReader.isEmpty()) {
+          return null;
+        }
+
+        return new SeriesReaderByTimestamp(seriesReader, ascending);
       }
     } catch (IOException e) {
       throw new QueryProcessException(e, TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode());
     }
-    return new SeriesReaderByTimestamp(seriesReader, ascending);
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterTimeGenerator.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterTimeGenerator.java
index 4b13e98..e0ea6ab 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterTimeGenerator.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterTimeGenerator.java
@@ -203,7 +203,8 @@ public class ClusterTimeGenerator extends ServerTimeGenerator {
                   filter,
                   context,
                   dataGroupMember,
-                  queryPlan.isAscending());
+                  queryPlan.isAscending(),
+                  null);
 
           if (pointReader.hasNextTimeValuePair()) {
             this.hasLocalReader = true;
@@ -215,8 +216,8 @@ public class ClusterTimeGenerator extends ServerTimeGenerator {
         } else if (endPoint == null) {
           endPoint =
               new QueryDataSet.EndPoint(
-                  partitionGroup.getHeader().getClientIp(),
-                  partitionGroup.getHeader().getClientPort());
+                  partitionGroup.getHeader().getNode().getClientIp(),
+                  partitionGroup.getHeader().getNode().getClientPort());
         }
       }
     } catch (Exception e) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/DataSourceInfo.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/DataSourceInfo.java
index 8889535..b9c4f00 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/DataSourceInfo.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/DataSourceInfo.java
@@ -26,6 +26,7 @@ import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.query.RemoteQueryContext;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.SingleSeriesQueryRequest;
 import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
@@ -187,7 +188,7 @@ public class DataSourceInfo {
     return this.dataType;
   }
 
-  public Node getHeader() {
+  public RaftNode getHeader() {
     return partitionGroup.getHeader();
   }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ManagedMergeReader.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ManagedDescPriorityMergeReader.java
similarity index 88%
copy from cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ManagedMergeReader.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ManagedDescPriorityMergeReader.java
index e54dede..653dea4 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ManagedMergeReader.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ManagedDescPriorityMergeReader.java
@@ -20,16 +20,18 @@
 package org.apache.iotdb.cluster.query.reader;
 
 import org.apache.iotdb.db.query.reader.series.ManagedSeriesReader;
-import org.apache.iotdb.db.query.reader.universal.PriorityMergeReader;
+import org.apache.iotdb.db.query.reader.universal.DescPriorityMergeReader;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
 import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.reader.IPointReader;
 
 import java.io.IOException;
 import java.util.NoSuchElementException;
 
 @SuppressWarnings("common-java:DuplicatedBlocks")
-public class ManagedMergeReader extends PriorityMergeReader implements ManagedSeriesReader {
+public class ManagedDescPriorityMergeReader extends DescPriorityMergeReader
+    implements ManagedSeriesReader, IPointReader {
 
   private static final int BATCH_SIZE = 4096;
 
@@ -39,7 +41,7 @@ public class ManagedMergeReader extends PriorityMergeReader implements ManagedSe
   private BatchData batchData;
   private TSDataType dataType;
 
-  public ManagedMergeReader(TSDataType dataType) {
+  public ManagedDescPriorityMergeReader(TSDataType dataType) {
     this.dataType = dataType;
   }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ManagedMergeReader.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ManagedPriorityMergeReader.java
similarity index 94%
rename from cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ManagedMergeReader.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ManagedPriorityMergeReader.java
index e54dede..e57f4d9 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ManagedMergeReader.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ManagedPriorityMergeReader.java
@@ -29,7 +29,7 @@ import java.io.IOException;
 import java.util.NoSuchElementException;
 
 @SuppressWarnings("common-java:DuplicatedBlocks")
-public class ManagedMergeReader extends PriorityMergeReader implements ManagedSeriesReader {
+public class ManagedPriorityMergeReader extends PriorityMergeReader implements ManagedSeriesReader {
 
   private static final int BATCH_SIZE = 4096;
 
@@ -39,7 +39,7 @@ public class ManagedMergeReader extends PriorityMergeReader implements ManagedSe
   private BatchData batchData;
   private TSDataType dataType;
 
-  public ManagedMergeReader(TSDataType dataType) {
+  public ManagedPriorityMergeReader(TSDataType dataType) {
     this.dataType = dataType;
   }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/MultBatchReader.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/MultBatchReader.java
index dd7f21b..dc47deb 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/MultBatchReader.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/MultBatchReader.java
@@ -41,7 +41,7 @@ public class MultBatchReader implements IMultBatchReader {
   @Override
   public boolean hasNextBatch() throws IOException {
     for (IBatchReader reader : pathBatchReaders.values()) {
-      if (reader.hasNextBatch()) {
+      if (reader != null && reader.hasNextBatch()) {
         return true;
       }
     }
@@ -50,7 +50,8 @@ public class MultBatchReader implements IMultBatchReader {
 
   @Override
   public boolean hasNextBatch(String fullPath) throws IOException {
-    return pathBatchReaders.get(fullPath).hasNextBatch();
+    IBatchReader reader = pathBatchReaders.get(fullPath);
+    return reader != null && reader.hasNextBatch();
   }
 
   @Override
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/MultDataSourceInfo.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/MultDataSourceInfo.java
index 27c9f59..14ca954 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/MultDataSourceInfo.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/MultDataSourceInfo.java
@@ -26,6 +26,7 @@ import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.query.RemoteQueryContext;
 import org.apache.iotdb.cluster.rpc.thrift.MultSeriesQueryRequest;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.handlers.caller.GenericHandler;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
@@ -202,7 +203,7 @@ public class MultDataSourceInfo {
     return this.dataTypes;
   }
 
-  public Node getHeader() {
+  public RaftNode getHeader() {
     return partitionGroup.getHeader();
   }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/RemoteMultSeriesReader.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/RemoteMultSeriesReader.java
index d608d7e..5e6bfba 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/RemoteMultSeriesReader.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/RemoteMultSeriesReader.java
@@ -46,7 +46,7 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicReference;
 
-/** mult reader without value filter that reads points from a remote side. */
+/** multi reader without value filter that reads points from a remote side. */
 public class RemoteMultSeriesReader extends AbstractMultPointReader {
 
   private static final Logger logger = LoggerFactory.getLogger(RemoteMultSeriesReader.class);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/ClientServer.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/ClientServer.java
index a8c588a..1200c2d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/ClientServer.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/ClientServer.java
@@ -29,6 +29,7 @@ import org.apache.iotdb.cluster.query.ClusterPlanExecutor;
 import org.apache.iotdb.cluster.query.ClusterPlanner;
 import org.apache.iotdb.cluster.query.RemoteQueryContext;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.server.handlers.caller.GenericHandler;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -302,8 +303,8 @@ public class ClientServer extends TSServiceImpl {
     RemoteQueryContext context = queryContextMap.remove(queryId);
     if (context != null) {
       // release the resources in every queried node
-      for (Entry<Node, Set<Node>> headerEntry : context.getQueriedNodesMap().entrySet()) {
-        Node header = headerEntry.getKey();
+      for (Entry<RaftNode, Set<Node>> headerEntry : context.getQueriedNodesMap().entrySet()) {
+        RaftNode header = headerEntry.getKey();
         Set<Node> queriedNodes = headerEntry.getValue();
 
         for (Node queriedNode : queriedNodes) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/DataClusterServer.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/DataClusterServer.java
index e4c81f8..e6be01a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/DataClusterServer.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/DataClusterServer.java
@@ -24,6 +24,8 @@ import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.exception.NoHeaderNodeException;
 import org.apache.iotdb.cluster.exception.NotInSameGroupException;
 import org.apache.iotdb.cluster.exception.PartitionTableUnavailableException;
+import org.apache.iotdb.cluster.log.logtypes.AddNodeLog;
+import org.apache.iotdb.cluster.log.logtypes.RemoveNodeLog;
 import org.apache.iotdb.cluster.partition.NodeAdditionResult;
 import org.apache.iotdb.cluster.partition.NodeRemovalResult;
 import org.apache.iotdb.cluster.partition.PartitionGroup;
@@ -46,6 +48,7 @@ import org.apache.iotdb.cluster.rpc.thrift.PullSchemaRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSchemaResp;
 import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotResp;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.RequestCommitIndexResponse;
 import org.apache.iotdb.cluster.rpc.thrift.SendSnapshotRequest;
 import org.apache.iotdb.cluster.rpc.thrift.SingleSeriesQueryRequest;
@@ -57,6 +60,7 @@ import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.cluster.server.monitor.NodeReport.DataMemberReport;
 import org.apache.iotdb.cluster.server.service.DataAsyncService;
 import org.apache.iotdb.cluster.server.service.DataSyncService;
+import org.apache.iotdb.cluster.utils.IOUtils;
 import org.apache.iotdb.service.rpc.thrift.TSStatus;
 
 import org.apache.thrift.TException;
@@ -69,8 +73,11 @@ import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
+import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -86,9 +93,9 @@ public class DataClusterServer extends RaftServer
 
   // key: the header of a data group, value: the member representing this node in this group and
   // it is currently at service
-  private Map<Node, DataGroupMember> headerGroupMap = new ConcurrentHashMap<>();
-  private Map<Node, DataAsyncService> asyncServiceMap = new ConcurrentHashMap<>();
-  private Map<Node, DataSyncService> syncServiceMap = new ConcurrentHashMap<>();
+  private Map<RaftNode, DataGroupMember> headerGroupMap = new ConcurrentHashMap<>();
+  private Map<RaftNode, DataAsyncService> asyncServiceMap = new ConcurrentHashMap<>();
+  private Map<RaftNode, DataSyncService> syncServiceMap = new ConcurrentHashMap<>();
   // key: the header of a data group, value: the member representing this node in this group but
   // it is out of service because another node has joined the group and expelled this node, or
   // the node itself is removed, but it is still stored to provide snapshot for other nodes
@@ -120,33 +127,43 @@ public class DataClusterServer extends RaftServer
    *
    * @param dataGroupMember
    */
-  public void addDataGroupMember(DataGroupMember dataGroupMember) {
-    DataGroupMember removedMember = headerGroupMap.remove(dataGroupMember.getHeader());
-    if (removedMember != null) {
-      removedMember.stop();
-      asyncServiceMap.remove(dataGroupMember.getHeader());
-      syncServiceMap.remove(dataGroupMember.getHeader());
+  public DataGroupMember addDataGroupMember(DataGroupMember dataGroupMember, RaftNode header) {
+    synchronized (headerGroupMap) {
+      if (headerGroupMap.containsKey(header)) {
+        logger.debug("Group {} already exist.", dataGroupMember.getAllNodes());
+        return headerGroupMap.get(header);
+      }
+      stoppedMemberManager.remove(header);
+      headerGroupMap.put(header, dataGroupMember);
+
+      dataGroupMember.start();
     }
-    stoppedMemberManager.remove(dataGroupMember.getHeader());
+    logger.info("Add group {} successfully.", dataGroupMember.getName());
+    resetServiceCache(header); // avoid dead-lock
+
+    return dataGroupMember;
+  }
 
-    headerGroupMap.put(dataGroupMember.getHeader(), dataGroupMember);
+  private void resetServiceCache(RaftNode header) {
+    asyncServiceMap.remove(header);
+    syncServiceMap.remove(header);
   }
 
   private <T> DataAsyncService getDataAsyncService(
-      Node header, AsyncMethodCallback<T> resultHandler, Object request) {
+      RaftNode header, AsyncMethodCallback<T> resultHandler, Object request) {
     return asyncServiceMap.computeIfAbsent(
         header,
         h -> {
-          DataGroupMember dataMember = getDataMember(h, resultHandler, request);
+          DataGroupMember dataMember = getDataMember(header, resultHandler, request);
           return dataMember != null ? new DataAsyncService(dataMember) : null;
         });
   }
 
-  private DataSyncService getDataSyncService(Node header) {
+  private DataSyncService getDataSyncService(RaftNode header) {
     return syncServiceMap.computeIfAbsent(
         header,
         h -> {
-          DataGroupMember dataMember = getDataMember(h, null, null);
+          DataGroupMember dataMember = getDataMember(header, null, null);
           return dataMember != null ? new DataSyncService(dataMember) : null;
         });
   }
@@ -159,10 +176,10 @@ public class DataClusterServer extends RaftServer
    * @return
    */
   public <T> DataGroupMember getDataMember(
-      Node header, AsyncMethodCallback<T> resultHandler, Object request) {
+      RaftNode header, AsyncMethodCallback<T> resultHandler, Object request) {
     // if the resultHandler is not null, then the request is a external one and must be with a
     // header
-    if (header == null) {
+    if (header.getNode() == null) {
       if (resultHandler != null) {
         resultHandler.onError(new NoHeaderNodeException());
       }
@@ -175,27 +192,25 @@ public class DataClusterServer extends RaftServer
 
     // avoid creating two members for a header
     Exception ex = null;
-    synchronized (headerGroupMap) {
-      member = headerGroupMap.get(header);
-      if (member != null) {
-        return member;
-      }
-      logger.info("Received a request \"{}\" from unregistered header {}", request, header);
-      if (partitionTable != null) {
-        try {
-          member = createNewMember(header);
-        } catch (NotInSameGroupException | CheckConsistencyException e) {
-          ex = e;
-        }
-      } else {
-        logger.info("Partition is not ready, cannot create member");
-        ex = new PartitionTableUnavailableException(thisNode);
-      }
-      if (ex != null && resultHandler != null) {
-        resultHandler.onError(ex);
-      }
+    member = headerGroupMap.get(header);
+    if (member != null) {
       return member;
     }
+    logger.info("Received a request \"{}\" from unregistered header {}", request, header);
+    if (partitionTable != null) {
+      try {
+        member = createNewMember(header);
+      } catch (NotInSameGroupException | CheckConsistencyException e) {
+        ex = e;
+      }
+    } else {
+      logger.info("Partition is not ready, cannot create member");
+      ex = new PartitionTableUnavailableException(thisNode);
+    }
+    if (ex != null && resultHandler != null) {
+      resultHandler.onError(ex);
+    }
+    return member;
   }
 
   /**
@@ -203,9 +218,8 @@ public class DataClusterServer extends RaftServer
    * @return A DataGroupMember representing this node in the data group of the header.
    * @throws NotInSameGroupException If this node is not in the group of the header.
    */
-  private DataGroupMember createNewMember(Node header)
+  private DataGroupMember createNewMember(RaftNode header)
       throws NotInSameGroupException, CheckConsistencyException {
-    DataGroupMember member;
     PartitionGroup partitionGroup;
     partitionGroup = partitionTable.getHeaderGroup(header);
     if (partitionGroup == null || !partitionGroup.contains(thisNode)) {
@@ -213,27 +227,32 @@ public class DataClusterServer extends RaftServer
       metaGroupMember.syncLeaderWithConsistencyCheck(true);
       partitionGroup = partitionTable.getHeaderGroup(header);
     }
-    if (partitionGroup != null && partitionGroup.contains(thisNode)) {
-      // the two nodes are in the same group, create a new data member
-      member = dataMemberFactory.create(partitionGroup, thisNode);
-      DataGroupMember prevMember = headerGroupMap.put(header, member);
-      if (prevMember != null) {
-        prevMember.stop();
-      }
-      logger.info("Created a member for header {}", header);
-      member.start();
-    } else {
-      // the member may have been stopped after syncLeader
-      member = stoppedMemberManager.get(header);
+    DataGroupMember member;
+    synchronized (headerGroupMap) {
+      member = headerGroupMap.get(header);
       if (member != null) {
         return member;
       }
-      logger.info(
-          "This node {} does not belong to the group {}, header {}",
-          thisNode,
-          partitionGroup,
-          header);
-      throw new NotInSameGroupException(partitionGroup, thisNode);
+      if (partitionGroup != null && partitionGroup.contains(thisNode)) {
+        // the two nodes are in the same group, create a new data member
+        member = dataMemberFactory.create(partitionGroup, thisNode);
+        headerGroupMap.put(header, member);
+        stoppedMemberManager.remove(header);
+        logger.info("Created a member for header {}, group is {}", header, partitionGroup);
+        member.start();
+      } else {
+        // the member may have been stopped after syncLeader
+        member = stoppedMemberManager.get(header);
+        if (member != null) {
+          return member;
+        }
+        logger.info(
+            "This node {} does not belong to the group {}, header {}",
+            thisNode,
+            partitionGroup,
+            header);
+        throw new NotInSameGroupException(partitionGroup, thisNode);
+      }
     }
     return member;
   }
@@ -244,8 +263,7 @@ public class DataClusterServer extends RaftServer
   @Override
   public void sendHeartbeat(
       HeartBeatRequest request, AsyncMethodCallback<HeartBeatResponse> resultHandler) {
-    Node header = request.getHeader();
-    DataAsyncService service = getDataAsyncService(header, resultHandler, request);
+    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
     if (service != null) {
       service.sendHeartbeat(request, resultHandler);
     }
@@ -253,8 +271,7 @@ public class DataClusterServer extends RaftServer
 
   @Override
   public void startElection(ElectionRequest request, AsyncMethodCallback<Long> resultHandler) {
-    Node header = request.getHeader();
-    DataAsyncService service = getDataAsyncService(header, resultHandler, request);
+    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
     if (service != null) {
       service.startElection(request, resultHandler);
     }
@@ -262,8 +279,7 @@ public class DataClusterServer extends RaftServer
 
   @Override
   public void appendEntries(AppendEntriesRequest request, AsyncMethodCallback<Long> resultHandler) {
-    Node header = request.getHeader();
-    DataAsyncService service = getDataAsyncService(header, resultHandler, request);
+    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
     if (service != null) {
       service.appendEntries(request, resultHandler);
     }
@@ -271,8 +287,7 @@ public class DataClusterServer extends RaftServer
 
   @Override
   public void appendEntry(AppendEntryRequest request, AsyncMethodCallback<Long> resultHandler) {
-    Node header = request.getHeader();
-    DataAsyncService service = getDataAsyncService(header, resultHandler, request);
+    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
     if (service != null) {
       service.appendEntry(request, resultHandler);
     }
@@ -280,8 +295,7 @@ public class DataClusterServer extends RaftServer
 
   @Override
   public void sendSnapshot(SendSnapshotRequest request, AsyncMethodCallback<Void> resultHandler) {
-    Node header = request.getHeader();
-    DataAsyncService service = getDataAsyncService(header, resultHandler, request);
+    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
     if (service != null) {
       service.sendSnapshot(request, resultHandler);
     }
@@ -290,8 +304,7 @@ public class DataClusterServer extends RaftServer
   @Override
   public void pullSnapshot(
       PullSnapshotRequest request, AsyncMethodCallback<PullSnapshotResp> resultHandler) {
-    Node header = request.getHeader();
-    DataAsyncService service = getDataAsyncService(header, resultHandler, request);
+    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
     if (service != null) {
       service.pullSnapshot(request, resultHandler);
     }
@@ -300,8 +313,7 @@ public class DataClusterServer extends RaftServer
   @Override
   public void executeNonQueryPlan(
       ExecutNonQueryReq request, AsyncMethodCallback<TSStatus> resultHandler) {
-    Node header = request.getHeader();
-    DataAsyncService service = getDataAsyncService(header, resultHandler, request);
+    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
     if (service != null) {
       service.executeNonQueryPlan(request, resultHandler);
     }
@@ -309,7 +321,7 @@ public class DataClusterServer extends RaftServer
 
   @Override
   public void requestCommitIndex(
-      Node header, AsyncMethodCallback<RequestCommitIndexResponse> resultHandler) {
+      RaftNode header, AsyncMethodCallback<RequestCommitIndexResponse> resultHandler) {
     DataAsyncService service = getDataAsyncService(header, resultHandler, "Request commit index");
     if (service != null) {
       service.requestCommitIndex(header, resultHandler);
@@ -319,10 +331,10 @@ public class DataClusterServer extends RaftServer
   @Override
   public void readFile(
       String filePath, long offset, int length, AsyncMethodCallback<ByteBuffer> resultHandler) {
-    DataAsyncService service =
-        getDataAsyncService(thisNode, resultHandler, "Read file:" + filePath);
-    if (service != null) {
-      service.readFile(filePath, offset, length, resultHandler);
+    try {
+      resultHandler.onComplete(IOUtils.readFile(filePath, offset, length));
+    } catch (IOException e) {
+      resultHandler.onError(e);
     }
   }
 
@@ -350,7 +362,7 @@ public class DataClusterServer extends RaftServer
 
   @Override
   public void fetchSingleSeries(
-      Node header, long readerId, AsyncMethodCallback<ByteBuffer> resultHandler) {
+      RaftNode header, long readerId, AsyncMethodCallback<ByteBuffer> resultHandler) {
     DataAsyncService service =
         getDataAsyncService(header, resultHandler, "Fetch reader:" + readerId);
     if (service != null) {
@@ -360,7 +372,7 @@ public class DataClusterServer extends RaftServer
 
   @Override
   public void fetchMultSeries(
-      Node header,
+      RaftNode header,
       long readerId,
       List<String> paths,
       AsyncMethodCallback<Map<String, ByteBuffer>> resultHandler)
@@ -374,7 +386,7 @@ public class DataClusterServer extends RaftServer
 
   @Override
   public void getAllPaths(
-      Node header,
+      RaftNode header,
       List<String> paths,
       boolean withAlias,
       AsyncMethodCallback<GetAllPathsResult> resultHandler) {
@@ -386,7 +398,7 @@ public class DataClusterServer extends RaftServer
 
   @Override
   public void endQuery(
-      Node header, Node thisNode, long queryId, AsyncMethodCallback<Void> resultHandler) {
+      RaftNode header, Node thisNode, long queryId, AsyncMethodCallback<Void> resultHandler) {
     DataAsyncService service = getDataAsyncService(header, resultHandler, "End query");
     if (service != null) {
       service.endQuery(header, thisNode, queryId, resultHandler);
@@ -413,7 +425,7 @@ public class DataClusterServer extends RaftServer
 
   @Override
   public void fetchSingleSeriesByTimestamps(
-      Node header,
+      RaftNode header,
       long readerId,
       List<Long> timestamps,
       AsyncMethodCallback<ByteBuffer> resultHandler) {
@@ -427,8 +439,7 @@ public class DataClusterServer extends RaftServer
   @Override
   public void pullTimeSeriesSchema(
       PullSchemaRequest request, AsyncMethodCallback<PullSchemaResp> resultHandler) {
-    Node header = request.getHeader();
-    DataAsyncService service = getDataAsyncService(header, resultHandler, request);
+    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
     if (service != null) {
       service.pullTimeSeriesSchema(request, resultHandler);
     }
@@ -439,85 +450,113 @@ public class DataClusterServer extends RaftServer
       PullSchemaRequest request, AsyncMethodCallback<PullSchemaResp> resultHandler) {
     DataAsyncService service =
         getDataAsyncService(request.getHeader(), resultHandler, "Pull measurement schema");
-    service.pullMeasurementSchema(request, resultHandler);
+    if (service != null) {
+      service.pullMeasurementSchema(request, resultHandler);
+    }
   }
 
   @Override
   public void getAllDevices(
-      Node header, List<String> paths, AsyncMethodCallback<Set<String>> resultHandler) {
+      RaftNode header, List<String> paths, AsyncMethodCallback<Set<String>> resultHandler) {
     DataAsyncService service = getDataAsyncService(header, resultHandler, "Get all devices");
-    service.getAllDevices(header, paths, resultHandler);
+    if (service != null) {
+      service.getAllDevices(header, paths, resultHandler);
+    }
   }
 
   @Override
   public void getDevices(
-      Node header, ByteBuffer planBytes, AsyncMethodCallback<ByteBuffer> resultHandler)
+      RaftNode header, ByteBuffer planBinary, AsyncMethodCallback<ByteBuffer> resultHandler)
       throws TException {
-    DataAsyncService service = getDataAsyncService(header, resultHandler, "Get devices");
-    service.getDevices(header, planBytes, resultHandler);
+    DataAsyncService service = getDataAsyncService(header, resultHandler, "get devices");
+    if (service != null) {
+      service.getDevices(header, planBinary, resultHandler);
+    }
   }
 
   @Override
   public void getNodeList(
-      Node header, String path, int nodeLevel, AsyncMethodCallback<List<String>> resultHandler) {
+      RaftNode header,
+      String path,
+      int nodeLevel,
+      AsyncMethodCallback<List<String>> resultHandler) {
     DataAsyncService service = getDataAsyncService(header, resultHandler, "Get node list");
-    service.getNodeList(header, path, nodeLevel, resultHandler);
+    if (service != null) {
+      service.getNodeList(header, path, nodeLevel, resultHandler);
+    }
   }
 
   @Override
   public void getChildNodeInNextLevel(
-      Node header, String path, AsyncMethodCallback<Set<String>> resultHandler) {
+      RaftNode header, String path, AsyncMethodCallback<Set<String>> resultHandler)
+      throws TException {
     DataAsyncService service =
         getDataAsyncService(header, resultHandler, "Get child node in next level");
-    service.getChildNodeInNextLevel(header, path, resultHandler);
+    if (service != null) {
+      service.getChildNodeInNextLevel(header, path, resultHandler);
+    }
   }
 
   @Override
   public void getChildNodePathInNextLevel(
-      Node header, String path, AsyncMethodCallback<Set<String>> resultHandler) {
+      RaftNode header, String path, AsyncMethodCallback<Set<String>> resultHandler) {
     DataAsyncService service =
         getDataAsyncService(header, resultHandler, "Get child node path in next level");
-    service.getChildNodePathInNextLevel(header, path, resultHandler);
+    if (service != null) {
+      service.getChildNodePathInNextLevel(header, path, resultHandler);
+    }
   }
 
   @Override
   public void getAllMeasurementSchema(
-      Node header, ByteBuffer planBytes, AsyncMethodCallback<ByteBuffer> resultHandler) {
+      RaftNode header, ByteBuffer planBytes, AsyncMethodCallback<ByteBuffer> resultHandler) {
     DataAsyncService service =
         getDataAsyncService(header, resultHandler, "Get all measurement schema");
-    service.getAllMeasurementSchema(header, planBytes, resultHandler);
+    if (service != null) {
+      service.getAllMeasurementSchema(header, planBytes, resultHandler);
+    }
   }
 
   @Override
   public void getAggrResult(
       GetAggrResultRequest request, AsyncMethodCallback<List<ByteBuffer>> resultHandler) {
     DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    service.getAggrResult(request, resultHandler);
+    if (service != null) {
+      service.getAggrResult(request, resultHandler);
+    }
   }
 
   @Override
   public void getUnregisteredTimeseries(
-      Node header, List<String> timeseriesList, AsyncMethodCallback<List<String>> resultHandler) {
+      RaftNode header,
+      List<String> timeseriesList,
+      AsyncMethodCallback<List<String>> resultHandler) {
     DataAsyncService service =
         getDataAsyncService(header, resultHandler, "Check if measurements are registered");
-    service.getUnregisteredTimeseries(header, timeseriesList, resultHandler);
+    if (service != null) {
+      service.getUnregisteredTimeseries(header, timeseriesList, resultHandler);
+    }
   }
 
   @Override
   public void getGroupByExecutor(GroupByRequest request, AsyncMethodCallback<Long> resultHandler) {
     DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    service.getGroupByExecutor(request, resultHandler);
+    if (service != null) {
+      service.getGroupByExecutor(request, resultHandler);
+    }
   }
 
   @Override
   public void getGroupByResult(
-      Node header,
+      RaftNode header,
       long executorId,
       long startTime,
       long endTime,
       AsyncMethodCallback<List<ByteBuffer>> resultHandler) {
     DataAsyncService service = getDataAsyncService(header, resultHandler, "Fetch group by");
-    service.getGroupByResult(header, executorId, startTime, endTime, resultHandler);
+    if (service != null) {
+      service.getGroupByResult(header, executorId, startTime, endTime, resultHandler);
+    }
   }
 
   @Override
@@ -556,6 +595,19 @@ public class DataClusterServer extends RaftServer
     return "DataServerThread-";
   }
 
+  public void preAddNodeForDataGroup(AddNodeLog log, DataGroupMember targetDataGroupMember) {
+
+    // Make sure the previous add/remove node log has applied
+    metaGroupMember.syncLocalApply(log.getMetaLogIndex() - 1, false);
+
+    // Check the validity of the partition table
+    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
+      return;
+    }
+
+    targetDataGroupMember.preAddNode(log.getNewNode());
+  }
+
   /**
    * Try adding the node into the group of each DataGroupMember, and if the DataGroupMember no
    * longer stays in that group, also remove and stop it. If the new group contains this node, also
@@ -565,40 +617,87 @@ public class DataClusterServer extends RaftServer
    * @param result
    */
   public void addNode(Node node, NodeAdditionResult result) {
-    Iterator<Entry<Node, DataGroupMember>> entryIterator = headerGroupMap.entrySet().iterator();
+    // If the node executed adding itself to the cluster, it's unnecessary to add new groups because
+    // they already exist.
+    if (node.equals(thisNode)) {
+      return;
+    }
+    Iterator<Entry<RaftNode, DataGroupMember>> entryIterator = headerGroupMap.entrySet().iterator();
     synchronized (headerGroupMap) {
       while (entryIterator.hasNext()) {
-        Entry<Node, DataGroupMember> entry = entryIterator.next();
+        Entry<RaftNode, DataGroupMember> entry = entryIterator.next();
         DataGroupMember dataGroupMember = entry.getValue();
         // the member may be extruded from the group, remove and stop it if so
         boolean shouldLeave = dataGroupMember.addNode(node, result);
         if (shouldLeave) {
           logger.info("This node does not belong to {} any more", dataGroupMember.getAllNodes());
+          removeMember(entry.getKey(), entry.getValue(), false);
           entryIterator.remove();
-          removeMember(entry.getKey(), entry.getValue());
         }
       }
 
-      if (result.getNewGroup().contains(thisNode)) {
-        logger.info("Adding this node into a new group {}", result.getNewGroup());
-        DataGroupMember dataGroupMember = dataMemberFactory.create(result.getNewGroup(), thisNode);
-        addDataGroupMember(dataGroupMember);
-        dataGroupMember.start();
-        dataGroupMember.pullNodeAdditionSnapshots(
-            ((SlotPartitionTable) partitionTable).getNodeSlots(node), node);
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "Data cluster server: start to handle new groups when adding new node {}", node);
+      }
+      for (PartitionGroup newGroup : result.getNewGroupList()) {
+        if (newGroup.contains(thisNode)) {
+          RaftNode header = newGroup.getHeader();
+          logger.info("Adding this node into a new group {}", newGroup);
+          DataGroupMember dataGroupMember = dataMemberFactory.create(newGroup, thisNode);
+          dataGroupMember = addDataGroupMember(dataGroupMember, header);
+          dataGroupMember.pullNodeAdditionSnapshots(
+              ((SlotPartitionTable) partitionTable).getNodeSlots(header), node);
+        }
       }
     }
   }
 
-  private void removeMember(Node header, DataGroupMember dataGroupMember) {
-    try {
-      dataGroupMember.syncLeader(null);
-    } catch (CheckConsistencyException e) {
-      logger.warn("Failed to check consistency.", e);
+  /**
+   * When the node joins a cluster, it also creates a new data group and a corresponding member
+   * which has no data. This is to make that member pull data from other nodes.
+   */
+  public void pullSnapshots() {
+    for (int raftId = 0;
+        raftId < ClusterDescriptor.getInstance().getConfig().getMultiRaftFactor();
+        raftId++) {
+      RaftNode raftNode = new RaftNode(thisNode, raftId);
+      List<Integer> slots = ((SlotPartitionTable) partitionTable).getNodeSlots(raftNode);
+      DataGroupMember dataGroupMember = headerGroupMap.get(raftNode);
+      dataGroupMember.pullNodeAdditionSnapshots(slots, thisNode);
     }
+  }
+
+  /**
+   * Make sure the group will not receive new raft logs
+   *
+   * @param header
+   * @param dataGroupMember
+   */
+  private void removeMember(
+      RaftNode header, DataGroupMember dataGroupMember, boolean removedGroup) {
     dataGroupMember.setReadOnly();
-    dataGroupMember.stop();
+    if (!removedGroup) {
+      dataGroupMember.stop();
+    } else {
+      if (dataGroupMember.getCharacter() != NodeCharacter.LEADER) {
+        new Thread(
+                () -> {
+                  try {
+                    dataGroupMember.syncLeader(null);
+                    dataGroupMember.stop();
+                  } catch (CheckConsistencyException e) {
+                    logger.warn("Failed to check consistency.", e);
+                  }
+                })
+            .start();
+      }
+    }
     stoppedMemberManager.put(header, dataGroupMember);
+    logger.info(
+        "Data group member has removed, header {}, group is {}.",
+        header,
+        dataGroupMember.getAllNodes());
   }
 
   /**
@@ -623,14 +722,14 @@ public class DataClusterServer extends RaftServer
 
     List<PartitionGroup> partitionGroups = partitionTable.getLocalGroups();
     for (PartitionGroup partitionGroup : partitionGroups) {
-      DataGroupMember prevMember = headerGroupMap.get(partitionGroup.getHeader());
+      RaftNode header = partitionGroup.getHeader();
+      DataGroupMember prevMember = headerGroupMap.get(header);
       if (prevMember == null || !prevMember.getAllNodes().equals(partitionGroup)) {
         logger.info("Building member of data group: {}", partitionGroup);
         // no previous member or member changed
         DataGroupMember dataGroupMember = dataMemberFactory.create(partitionGroup, thisNode);
-        dataGroupMember.start();
         // the previous member will be replaced here
-        addDataGroupMember(dataGroupMember);
+        addDataGroupMember(dataGroupMember, header);
         dataGroupMember.setUnchanged(true);
       } else {
         prevMember.setUnchanged(true);
@@ -643,52 +742,64 @@ public class DataClusterServer extends RaftServer
     logger.info("Data group members are ready");
   }
 
+  public void preRemoveNodeForDataGroup(RemoveNodeLog log, DataGroupMember targetDataGroupMember) {
+
+    // Make sure the previous add/remove node log has applied
+    metaGroupMember.syncLocalApply(log.getMetaLogIndex() - 1, false);
+
+    // Check the validity of the partition table
+    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
+      return;
+    }
+
+    logger.debug(
+        "Pre removing a node {} from {}",
+        log.getRemovedNode(),
+        targetDataGroupMember.getAllNodes());
+    targetDataGroupMember.preRemoveNode(log.getRemovedNode());
+  }
+
   /**
    * Try removing a node from the groups of each DataGroupMember. If the node is the header of some
    * group, set the member to read only so that it can still provide data for other nodes that has
-   * not yet pulled its data. If the node is the local node, remove all members whose group is not
-   * headed by this node. Otherwise, just change the node list of the member and pull new data. And
-   * create a new DataGroupMember if this node should join a new group because of this removal.
+   * not yet pulled its data. Otherwise, just change the node list of the member and pull new data.
+   * And create a new DataGroupMember if this node should join a new group because of this removal.
    *
    * @param node
    * @param removalResult cluster changes due to the node removal
    */
   public void removeNode(Node node, NodeRemovalResult removalResult) {
-    Iterator<Entry<Node, DataGroupMember>> entryIterator = headerGroupMap.entrySet().iterator();
+    Iterator<Entry<RaftNode, DataGroupMember>> entryIterator = headerGroupMap.entrySet().iterator();
     synchronized (headerGroupMap) {
       while (entryIterator.hasNext()) {
-        Entry<Node, DataGroupMember> entry = entryIterator.next();
+        Entry<RaftNode, DataGroupMember> entry = entryIterator.next();
         DataGroupMember dataGroupMember = entry.getValue();
-        if (dataGroupMember.getHeader().equals(node)) {
-          // the group is removed as the node is removed, so new writes should be rejected as
-          // they belong to the new holder, but the member is kept alive for other nodes to pull
-          // snapshots
+        if (dataGroupMember.getHeader().getNode().equals(node) || node.equals(thisNode)) {
           entryIterator.remove();
-          removeMember(entry.getKey(), entry.getValue());
+          removeMember(
+              entry.getKey(), dataGroupMember, dataGroupMember.getHeader().getNode().equals(node));
         } else {
-          if (node.equals(thisNode)) {
-            // this node is removed, it is no more replica of other groups
-            List<Integer> nodeSlots =
-                ((SlotPartitionTable) partitionTable).getNodeSlots(dataGroupMember.getHeader());
-            dataGroupMember.removeLocalData(nodeSlots);
-            entryIterator.remove();
-            dataGroupMember.stop();
-          } else {
-            // the group should be updated and pull new slots from the removed node
-            dataGroupMember.removeNode(node, removalResult);
-          }
+          // the group should be updated
+          dataGroupMember.removeNode(node);
         }
       }
-      PartitionGroup newGroup = removalResult.getNewGroup();
-      if (newGroup != null) {
-        logger.info("{} should join a new group {}", thisNode, newGroup);
-        try {
-          createNewMember(newGroup.getHeader());
-        } catch (NotInSameGroupException e) {
-          // ignored
-        } catch (CheckConsistencyException ce) {
-          logger.error("remove node failed, error={}", ce.getMessage());
+
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "Data cluster server: start to handle new groups and pulling data when removing node {}",
+            node);
+      }
+      // if the removed group contains the local node, the local node should join a new group to
+      // preserve the replication number
+      for (PartitionGroup group : partitionTable.getLocalGroups()) {
+        RaftNode header = group.getHeader();
+        if (!headerGroupMap.containsKey(header)) {
+          logger.info("{} should join a new group {}", thisNode, group);
+          DataGroupMember dataGroupMember = dataMemberFactory.create(group, thisNode);
+          addDataGroupMember(dataGroupMember, header);
         }
+        // pull new slots from the removed node
+        headerGroupMap.get(header).pullSlots(removalResult);
       }
     }
   }
@@ -697,16 +808,6 @@ public class DataClusterServer extends RaftServer
     this.partitionTable = partitionTable;
   }
 
-  /**
-   * When the node joins a cluster, it also creates a new data group and a corresponding member
-   * which has no data. This is to make that member pull data from other nodes.
-   */
-  public void pullSnapshots() {
-    List<Integer> slots = ((SlotPartitionTable) partitionTable).getNodeSlots(thisNode);
-    DataGroupMember dataGroupMember = headerGroupMap.get(thisNode);
-    dataGroupMember.pullNodeAdditionSnapshots(slots, thisNode);
-  }
-
   /** @return The reports of every DataGroupMember in this node. */
   public List<DataMemberReport> genMemberReports() {
     List<DataMemberReport> dataMemberReports = new ArrayList<>();
@@ -721,7 +822,9 @@ public class DataClusterServer extends RaftServer
   public void previousFill(
       PreviousFillRequest request, AsyncMethodCallback<ByteBuffer> resultHandler) {
     DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    service.previousFill(request, resultHandler);
+    if (service != null) {
+      service.previousFill(request, resultHandler);
+    }
   }
 
   public void closeLogManagers() {
@@ -730,34 +833,46 @@ public class DataClusterServer extends RaftServer
     }
   }
 
+  public Map<RaftNode, DataGroupMember> getHeaderGroupMap() {
+    return headerGroupMap;
+  }
+
   @Override
   public void matchTerm(
-      long index, long term, Node header, AsyncMethodCallback<Boolean> resultHandler) {
+      long index, long term, RaftNode header, AsyncMethodCallback<Boolean> resultHandler) {
     DataAsyncService service = getDataAsyncService(header, resultHandler, "Match term");
-    service.matchTerm(index, term, header, resultHandler);
+    if (service != null) {
+      service.matchTerm(index, term, header, resultHandler);
+    }
   }
 
   @Override
   public void last(LastQueryRequest request, AsyncMethodCallback<ByteBuffer> resultHandler) {
     DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, "last");
-    service.last(request, resultHandler);
+    if (service != null) {
+      service.last(request, resultHandler);
+    }
   }
 
   @Override
   public void getPathCount(
-      Node header,
+      RaftNode header,
       List<String> pathsToQuery,
       int level,
       AsyncMethodCallback<Integer> resultHandler) {
     DataAsyncService service = getDataAsyncService(header, resultHandler, "count path");
-    service.getPathCount(header, pathsToQuery, level, resultHandler);
+    if (service != null) {
+      service.getPathCount(header, pathsToQuery, level, resultHandler);
+    }
   }
 
   @Override
   public void onSnapshotApplied(
-      Node header, List<Integer> slots, AsyncMethodCallback<Boolean> resultHandler) {
+      RaftNode header, List<Integer> slots, AsyncMethodCallback<Boolean> resultHandler) {
     DataAsyncService service = getDataAsyncService(header, resultHandler, "Snapshot applied");
-    service.onSnapshotApplied(header, slots, resultHandler);
+    if (service != null) {
+      service.onSnapshotApplied(header, slots, resultHandler);
+    }
   }
 
   @Override
@@ -771,12 +886,12 @@ public class DataClusterServer extends RaftServer
   }
 
   @Override
-  public ByteBuffer fetchSingleSeries(Node header, long readerId) throws TException {
+  public ByteBuffer fetchSingleSeries(RaftNode header, long readerId) throws TException {
     return getDataSyncService(header).fetchSingleSeries(header, readerId);
   }
 
   @Override
-  public Map<String, ByteBuffer> fetchMultSeries(Node header, long readerId, List<String> paths)
+  public Map<String, ByteBuffer> fetchMultSeries(RaftNode header, long readerId, List<String> paths)
       throws TException {
     return getDataSyncService(header).fetchMultSeries(header, readerId, paths);
   }
@@ -787,49 +902,50 @@ public class DataClusterServer extends RaftServer
   }
 
   @Override
-  public ByteBuffer fetchSingleSeriesByTimestamps(Node header, long readerId, List<Long> timestamps)
-      throws TException {
+  public ByteBuffer fetchSingleSeriesByTimestamps(
+      RaftNode header, long readerId, List<Long> timestamps) throws TException {
     return getDataSyncService(header).fetchSingleSeriesByTimestamps(header, readerId, timestamps);
   }
 
   @Override
-  public void endQuery(Node header, Node thisNode, long queryId) throws TException {
+  public void endQuery(RaftNode header, Node thisNode, long queryId) throws TException {
     getDataSyncService(header).endQuery(header, thisNode, queryId);
   }
 
   @Override
-  public GetAllPathsResult getAllPaths(Node header, List<String> path, boolean withAlias)
+  public GetAllPathsResult getAllPaths(RaftNode header, List<String> path, boolean withAlias)
       throws TException {
     return getDataSyncService(header).getAllPaths(header, path, withAlias);
   }
 
   @Override
-  public Set<String> getAllDevices(Node header, List<String> path) throws TException {
+  public Set<String> getAllDevices(RaftNode header, List<String> path) throws TException {
     return getDataSyncService(header).getAllDevices(header, path);
   }
 
   @Override
-  public List<String> getNodeList(Node header, String path, int nodeLevel) throws TException {
+  public List<String> getNodeList(RaftNode header, String path, int nodeLevel) throws TException {
     return getDataSyncService(header).getNodeList(header, path, nodeLevel);
   }
 
   @Override
-  public Set<String> getChildNodeInNextLevel(Node header, String path) throws TException {
+  public Set<String> getChildNodeInNextLevel(RaftNode header, String path) throws TException {
     return getDataSyncService(header).getChildNodeInNextLevel(header, path);
   }
 
   @Override
-  public Set<String> getChildNodePathInNextLevel(Node header, String path) throws TException {
+  public Set<String> getChildNodePathInNextLevel(RaftNode header, String path) throws TException {
     return getDataSyncService(header).getChildNodePathInNextLevel(header, path);
   }
 
   @Override
-  public ByteBuffer getAllMeasurementSchema(Node header, ByteBuffer planBinary) throws TException {
+  public ByteBuffer getAllMeasurementSchema(RaftNode header, ByteBuffer planBinary)
+      throws TException {
     return getDataSyncService(header).getAllMeasurementSchema(header, planBinary);
   }
 
   @Override
-  public ByteBuffer getDevices(Node header, ByteBuffer planBinary) throws TException {
+  public ByteBuffer getDevices(RaftNode header, ByteBuffer planBinary) throws TException {
     return getDataSyncService(header).getDevices(header, planBinary);
   }
 
@@ -839,7 +955,7 @@ public class DataClusterServer extends RaftServer
   }
 
   @Override
-  public List<String> getUnregisteredTimeseries(Node header, List<String> timeseriesList)
+  public List<String> getUnregisteredTimeseries(RaftNode header, List<String> timeseriesList)
       throws TException {
     return getDataSyncService(header).getUnregisteredTimeseries(header, timeseriesList);
   }
@@ -851,12 +967,12 @@ public class DataClusterServer extends RaftServer
 
   @Override
   public long getGroupByExecutor(GroupByRequest request) throws TException {
-    return getDataSyncService(request.header).getGroupByExecutor(request);
+    return getDataSyncService(request.getHeader()).getGroupByExecutor(request);
   }
 
   @Override
   public List<ByteBuffer> getGroupByResult(
-      Node header, long executorId, long startTime, long endTime) throws TException {
+      RaftNode header, long executorId, long startTime, long endTime) throws TException {
     return getDataSyncService(header).getGroupByResult(header, executorId, startTime, endTime);
   }
 
@@ -881,12 +997,12 @@ public class DataClusterServer extends RaftServer
   }
 
   @Override
-  public int getPathCount(Node header, List<String> pathsToQuery, int level) throws TException {
+  public int getPathCount(RaftNode header, List<String> pathsToQuery, int level) throws TException {
     return getDataSyncService(header).getPathCount(header, pathsToQuery, level);
   }
 
   @Override
-  public boolean onSnapshotApplied(Node header, List<Integer> slots) {
+  public boolean onSnapshotApplied(RaftNode header, List<Integer> slots) {
     return getDataSyncService(header).onSnapshotApplied(header, slots);
   }
 
@@ -921,29 +1037,33 @@ public class DataClusterServer extends RaftServer
   }
 
   @Override
-  public RequestCommitIndexResponse requestCommitIndex(Node header) throws TException {
+  public RequestCommitIndexResponse requestCommitIndex(RaftNode header) throws TException {
     return getDataSyncService(header).requestCommitIndex(header);
   }
 
   @Override
   public ByteBuffer readFile(String filePath, long offset, int length) throws TException {
-    return getDataSyncService(thisNode).readFile(filePath, offset, length);
+    try {
+      return IOUtils.readFile(filePath, offset, length);
+    } catch (IOException e) {
+      throw new TException(e);
+    }
   }
 
   @Override
-  public boolean matchTerm(long index, long term, Node header) {
+  public boolean matchTerm(long index, long term, RaftNode header) {
     return getDataSyncService(header).matchTerm(index, term, header);
   }
 
   @Override
-  public ByteBuffer peekNextNotNullValue(Node header, long executorId, long startTime, long endTime)
-      throws TException {
+  public ByteBuffer peekNextNotNullValue(
+      RaftNode header, long executorId, long startTime, long endTime) throws TException {
     return getDataSyncService(header).peekNextNotNullValue(header, executorId, startTime, endTime);
   }
 
   @Override
   public void peekNextNotNullValue(
-      Node header,
+      RaftNode header,
       long executorId,
       long startTime,
       long endTime,
@@ -955,12 +1075,20 @@ public class DataClusterServer extends RaftServer
 
   @Override
   public void removeHardLink(String hardLinkPath) throws TException {
-    getDataSyncService(thisNode).removeHardLink(hardLinkPath);
+    try {
+      Files.deleteIfExists(new File(hardLinkPath).toPath());
+    } catch (IOException e) {
+      throw new TException(e);
+    }
   }
 
   @Override
   public void removeHardLink(String hardLinkPath, AsyncMethodCallback<Void> resultHandler) {
-    getDataAsyncService(thisNode, resultHandler, hardLinkPath)
-        .removeHardLink(hardLinkPath, resultHandler);
+    try {
+      Files.deleteIfExists(new File(hardLinkPath).toPath());
+      resultHandler.onComplete(null);
+    } catch (IOException e) {
+      resultHandler.onError(e);
+    }
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/MetaClusterServer.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/MetaClusterServer.java
index 475cca0..fce7a87 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/MetaClusterServer.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/MetaClusterServer.java
@@ -215,7 +215,7 @@ public class MetaClusterServer extends RaftServer
 
   @Override
   public void requestCommitIndex(
-      Node header, AsyncMethodCallback<RequestCommitIndexResponse> resultHandler) {
+      RaftNode header, AsyncMethodCallback<RequestCommitIndexResponse> resultHandler) {
     asyncService.requestCommitIndex(header, resultHandler);
   }
 
@@ -225,6 +225,11 @@ public class MetaClusterServer extends RaftServer
   }
 
   @Override
+  public void collectMigrationStatus(AsyncMethodCallback<ByteBuffer> resultHandler) {
+    asyncService.collectMigrationStatus(resultHandler);
+  }
+
+  @Override
   public void readFile(
       String filePath, long offset, int length, AsyncMethodCallback<ByteBuffer> resultHandler) {
     asyncService.readFile(filePath, offset, length, resultHandler);
@@ -251,13 +256,13 @@ public class MetaClusterServer extends RaftServer
   }
 
   @Override
-  public void exile(AsyncMethodCallback<Void> resultHandler) {
-    asyncService.exile(resultHandler);
+  public void exile(ByteBuffer removeNodeLog, AsyncMethodCallback<Void> resultHandler) {
+    asyncService.exile(removeNodeLog, resultHandler);
   }
 
   @Override
   public void matchTerm(
-      long index, long term, Node header, AsyncMethodCallback<Boolean> resultHandler) {
+      long index, long term, RaftNode header, AsyncMethodCallback<Boolean> resultHandler) {
     asyncService.matchTerm(index, term, header, resultHandler);
   }
 
@@ -277,8 +282,8 @@ public class MetaClusterServer extends RaftServer
   }
 
   @Override
-  public void exile() {
-    syncService.exile();
+  public void exile(ByteBuffer removeNodeLog) {
+    syncService.exile(removeNodeLog);
   }
 
   @Override
@@ -292,6 +297,11 @@ public class MetaClusterServer extends RaftServer
   }
 
   @Override
+  public ByteBuffer collectMigrationStatus() {
+    return syncService.collectMigrationStatus();
+  }
+
+  @Override
   public HeartBeatResponse sendHeartbeat(HeartBeatRequest request) {
     return syncService.sendHeartbeat(request);
   }
@@ -322,7 +332,7 @@ public class MetaClusterServer extends RaftServer
   }
 
   @Override
-  public RequestCommitIndexResponse requestCommitIndex(Node header) throws TException {
+  public RequestCommitIndexResponse requestCommitIndex(RaftNode header) throws TException {
     return syncService.requestCommitIndex(header);
   }
 
@@ -332,7 +342,7 @@ public class MetaClusterServer extends RaftServer
   }
 
   @Override
-  public boolean matchTerm(long index, long term, Node header) {
+  public boolean matchTerm(long index, long term, RaftNode header) {
     return syncService.matchTerm(index, term, header);
   }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/PullSnapshotHintService.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/PullSnapshotHintService.java
index 22cee89..c137027 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/PullSnapshotHintService.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/PullSnapshotHintService.java
@@ -24,14 +24,15 @@ import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.log.snapshot.PullSnapshotTaskDescriptor;
+import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
 
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.ConcurrentLinkedDeque;
@@ -39,6 +40,8 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.iotdb.cluster.config.ClusterConstant.THREAD_POLL_WAIT_TERMINATION_TIME_S;
+
 public class PullSnapshotHintService {
 
   private static final Logger logger = LoggerFactory.getLogger(PullSnapshotHintService.class);
@@ -54,7 +57,7 @@ public class PullSnapshotHintService {
 
   public void start() {
     this.service = Executors.newScheduledThreadPool(1);
-    this.service.scheduleAtFixedRate(this::sendHints, 0, 1, TimeUnit.MINUTES);
+    this.service.scheduleAtFixedRate(this::sendHints, 0, 10, TimeUnit.MILLISECONDS);
   }
 
   public void stop() {
@@ -62,9 +65,9 @@ public class PullSnapshotHintService {
       return;
     }
 
-    service.shutdown();
+    service.shutdownNow();
     try {
-      service.awaitTermination(3, TimeUnit.MINUTES);
+      service.awaitTermination(THREAD_POLL_WAIT_TERMINATION_TIME_S, TimeUnit.SECONDS);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       logger.warn("{}: PullSnapshotHintService exiting interrupted", member.getName());
@@ -74,8 +77,8 @@ public class PullSnapshotHintService {
 
   public void registerHint(PullSnapshotTaskDescriptor descriptor) {
     PullSnapshotHint hint = new PullSnapshotHint();
-    hint.receivers = new ArrayList<>(descriptor.getPreviousHolders());
-    hint.header = descriptor.getPreviousHolders().getHeader();
+    hint.partitionGroup = descriptor.getPreviousHolders();
+    hint.receivers = new PartitionGroup(hint.partitionGroup);
     hint.slots = descriptor.getSlots();
     hints.add(hint);
   }
@@ -85,16 +88,30 @@ public class PullSnapshotHintService {
       PullSnapshotHint hint = iterator.next();
       for (Iterator<Node> iter = hint.receivers.iterator(); iter.hasNext(); ) {
         Node receiver = iter.next();
-        try {
-          boolean result = sendHint(receiver, hint);
-          if (result) {
-            iter.remove();
+        // If the receiver is the removed node, ignore the hint
+        if (!member.getMetaGroupMember().getPartitionTable().getAllNodes().contains(receiver)) {
+          iter.remove();
+        } else {
+          try {
+            if (logger.isDebugEnabled()) {
+              logger.debug(
+                  "{}: start to send hint to target group {}, receiver {}, slot is {} and other {}",
+                  member.getName(),
+                  hint.partitionGroup,
+                  receiver,
+                  hint.slots.get(0),
+                  hint.slots.size() - 1);
+            }
+            boolean result = sendHint(receiver, hint);
+            if (result) {
+              iter.remove();
+            }
+          } catch (TException e) {
+            logger.warn("Cannot send pull snapshot hint to {}", receiver);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            logger.warn("Sending hint to {} interrupted", receiver);
           }
-        } catch (TException e) {
-          logger.warn("Cannot send pull snapshot hint to {}", receiver);
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          logger.warn("Sending hint to {} interrupted", receiver);
         }
       }
       // all nodes in remote group know the hint, the hint can be removed
@@ -118,7 +135,7 @@ public class PullSnapshotHintService {
   private boolean sendHintsAsync(Node receiver, PullSnapshotHint hint)
       throws TException, InterruptedException {
     AsyncDataClient asyncDataClient = (AsyncDataClient) member.getAsyncClient(receiver);
-    return SyncClientAdaptor.onSnapshotApplied(asyncDataClient, hint.header, hint.slots);
+    return SyncClientAdaptor.onSnapshotApplied(asyncDataClient, hint.getHeader(), hint.slots);
   }
 
   private boolean sendHintSync(Node receiver, PullSnapshotHint hint) throws TException {
@@ -126,17 +143,25 @@ public class PullSnapshotHintService {
       if (syncDataClient == null) {
         return false;
       }
-      return syncDataClient.onSnapshotApplied(hint.header, hint.slots);
+      return syncDataClient.onSnapshotApplied(hint.getHeader(), hint.slots);
     }
   }
 
   private static class PullSnapshotHint {
 
-    /** Nodes to send this hint; */
-    private List<Node> receivers;
+    /** Nodes to send this hint */
+    private PartitionGroup receivers;
 
-    private Node header;
+    private PartitionGroup partitionGroup;
 
     private List<Integer> slots;
+
+    public RaftNode getHeader() {
+      return partitionGroup.getHeader();
+    }
+
+    public int getRaftId() {
+      return receivers.getId();
+    }
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/Response.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/Response.java
index 523fb0d..006eec1 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/Response.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/Response.java
@@ -37,18 +37,19 @@ public class Response {
   public static final long RESPONSE_IDENTIFIER_CONFLICT = -5;
   // the requested node is unreachable in the network
   public static final long RESPONSE_NO_CONNECTION = -6;
-  // the meta logs of the data group leader candidate is older than the voter, so its partition
-  // table is potentially older and such a node cannot be allowed to be the leader
-  public static final long RESPONSE_META_LOG_STALE = -7;
   // the node does not give a vote because its leader does not time out. This is to avoid a
   // node which cannot connect to the leader changing the leader in the group frequently.
-  public static final long RESPONSE_LEADER_STILL_ONLINE = -8;
+  public static final long RESPONSE_LEADER_STILL_ONLINE = -7;
   // the operation is rejected because the cluster will not be able to have enough replicas after
   // this operation
-  public static final long RESPONSE_CLUSTER_TOO_SMALL = -9;
+  public static final long RESPONSE_CLUSTER_TOO_SMALL = -8;
   // the new node, which tries to join the cluster, contains conflicted parameters with the
   // cluster, so the operation is rejected.
-  public static final long RESPONSE_NEW_NODE_PARAMETER_CONFLICT = -10;
+  public static final long RESPONSE_NEW_NODE_PARAMETER_CONFLICT = -9;
+  // the data migration of previous add/remove node operations is not finished.
+  public static final long RESPONSE_DATA_MIGRATION_NOT_FINISH = -10;
+  // the node has removed from the group, so the operation is rejected.
+  public static final long RESPONSE_NODE_IS_NOT_IN_GROUP = -11;
   // the request is not executed locally anc should be forwarded
   public static final long RESPONSE_NULL = Long.MIN_VALUE;
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/StoppedMemberManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/StoppedMemberManager.java
index 287f411..788e83b 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/StoppedMemberManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/StoppedMemberManager.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.cluster.server;
 
 import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
 import org.apache.iotdb.cluster.server.member.DataGroupMember.Factory;
 import org.apache.iotdb.cluster.utils.ClusterUtils;
@@ -55,7 +56,7 @@ public class StoppedMemberManager {
   private static final String REMOVED = "0";
   private static final String RESUMED = "1";
 
-  private Map<Node, DataGroupMember> removedMemberMap = new HashMap<>();
+  private Map<RaftNode, DataGroupMember> removedMemberMap = new HashMap<>();
   private DataGroupMember.Factory memberFactory;
   private Node thisNode;
 
@@ -69,20 +70,23 @@ public class StoppedMemberManager {
    * When a DataGroupMember is removed, add it here and record this removal, so in next start-up we
    * can recover it as a data source for data transfers.
    *
-   * @param header
+   * @param header When a DataGroupMember is removed, add it here and record this removal, so in
+   *     next start-up we can recover it as a data source for data transfers.
+   * @param raftNode
    * @param dataGroupMember
    */
-  public synchronized void put(Node header, DataGroupMember dataGroupMember) {
-    removedMemberMap.put(header, dataGroupMember);
+  public synchronized void put(RaftNode raftNode, DataGroupMember dataGroupMember) {
+    removedMemberMap.put(raftNode, dataGroupMember);
     try (BufferedWriter writer = new BufferedWriter(new FileWriter(stoppedMembersFileName, true))) {
       StringBuilder builder = new StringBuilder(REMOVED);
+      builder.append(";").append(raftNode.raftId);
       for (Node node : dataGroupMember.getAllNodes()) {
         builder.append(";").append(node.toString());
       }
       writer.write(builder.toString());
       writer.newLine();
     } catch (IOException e) {
-      logger.error("Cannot record removed member of header {}", header, e);
+      logger.error("Cannot record removed member of header {}", raftNode, e);
     }
   }
 
@@ -90,20 +94,20 @@ public class StoppedMemberManager {
    * When a DataGroupMember is resumed, add it here and record this removal, so in next start-up we
    * will not recover it here.
    *
-   * @param header
+   * @param raftNode
    */
-  public synchronized void remove(Node header) {
-    removedMemberMap.remove(header);
+  public synchronized void remove(RaftNode raftNode) {
+    removedMemberMap.remove(raftNode);
     try (BufferedWriter writer = new BufferedWriter(new FileWriter(stoppedMembersFileName, true))) {
-      writer.write(RESUMED + ";" + header);
+      writer.write(RESUMED + ";" + raftNode.getRaftId() + ";" + raftNode.getNode().toString());
       writer.newLine();
     } catch (IOException e) {
-      logger.error("Cannot record resumed member of header {}", header, e);
+      logger.error("Cannot record resumed member of header {}", raftNode, e);
     }
   }
 
-  public synchronized DataGroupMember get(Node header) {
-    return removedMemberMap.get(header);
+  public synchronized DataGroupMember get(RaftNode raftNode) {
+    return removedMemberMap.get(raftNode);
   }
 
   private void recover() {
@@ -140,7 +144,9 @@ public class StoppedMemberManager {
 
   private void parseRemoved(String[] split) {
     PartitionGroup partitionGroup = new PartitionGroup();
-    for (int i = 1; i < split.length; i++) {
+    int raftId = Integer.parseInt(split[1]);
+    partitionGroup.setId(raftId);
+    for (int i = 2; i < split.length; i++) {
       Node node = ClusterUtils.stringToNode(split[i]);
       partitionGroup.add(node);
     }
@@ -150,7 +156,8 @@ public class StoppedMemberManager {
   }
 
   private void parseResumed(String[] split) {
-    Node header = ClusterUtils.stringToNode(split[1]);
-    removedMemberMap.remove(header);
+    int raftId = Integer.parseInt(split[1]);
+    Node header = ClusterUtils.stringToNode(split[2]);
+    removedMemberMap.remove(new RaftNode(header, raftId));
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/clusterinfo/ClusterInfoServiceImpl.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/clusterinfo/ClusterInfoServiceImpl.java
index 82cb1b0..6891a6e 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/clusterinfo/ClusterInfoServiceImpl.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/clusterinfo/ClusterInfoServiceImpl.java
@@ -56,7 +56,7 @@ public class ClusterInfoServiceImpl implements ClusterInfoService.Iface {
   }
 
   @Override
-  public Map<Node, Boolean> getAllNodeStatus() throws TException {
+  public Map<Node, Integer> getAllNodeStatus() throws TException {
     return ClusterMonitor.INSTANCE.getAllNodeStatus();
   }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/ElectionHandler.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/ElectionHandler.java
index 530cde8..6190d20 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/ElectionHandler.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/ElectionHandler.java
@@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.iotdb.cluster.server.Response.RESPONSE_AGREE;
 import static org.apache.iotdb.cluster.server.Response.RESPONSE_LEADER_STILL_ONLINE;
+import static org.apache.iotdb.cluster.server.Response.RESPONSE_NODE_IS_NOT_IN_GROUP;
 
 /**
  * ElectionHandler checks the result from a voter and decides whether the election goes on, succeeds
@@ -105,6 +106,9 @@ public class ElectionHandler implements AsyncMethodCallback<Long> {
           // the rejection from a node with a smaller term means the log of this node falls behind
           logger.info("{}: Election {} rejected: code {}", memberName, currTerm, voterResp);
           onFail();
+        } else if (voterResp == RESPONSE_NODE_IS_NOT_IN_GROUP) {
+          logger.info("{}: This node has removed from the group", memberName);
+          onFail();
         } else {
           // the election is rejected by a node with a bigger term, update current term to it
           logger.info(
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/HeartbeatHandler.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/HeartbeatHandler.java
index de0b83f..ba26259 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/HeartbeatHandler.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/HeartbeatHandler.java
@@ -52,8 +52,10 @@ public class HeartbeatHandler implements AsyncMethodCallback<HeartBeatResponse>
 
   @Override
   public void onComplete(HeartBeatResponse resp) {
-    logger.trace("{}: Received a heartbeat response", memberName);
     long followerTerm = resp.getTerm();
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: Received a heartbeat response {}", memberName, followerTerm);
+    }
     if (followerTerm == RESPONSE_AGREE) {
       // current leadership is still valid
       handleNormalHeartbeatResponse(resp);
@@ -83,14 +85,16 @@ public class HeartbeatHandler implements AsyncMethodCallback<HeartBeatResponse>
     long lastLogTerm = resp.getLastLogTerm();
     long localLastLogIdx = localMember.getLogManager().getLastLogIndex();
     long localLastLogTerm = localMember.getLogManager().getLastLogTerm();
-    logger.trace(
-        "{}: Node {} is still alive, log index: {}/{}, log term: {}/{}",
-        memberName,
-        follower,
-        lastLogIdx,
-        localLastLogIdx,
-        lastLogTerm,
-        localLastLogTerm);
+    if (logger.isDebugEnabled()) {
+      logger.debug(
+          "{}: Node {} is still alive, log index: {}/{}, log term: {}/{}",
+          memberName,
+          follower,
+          lastLogIdx,
+          localLastLogIdx,
+          lastLogTerm,
+          localLastLogTerm);
+    }
 
     Peer peer =
         localMember
@@ -106,7 +110,7 @@ public class HeartbeatHandler implements AsyncMethodCallback<HeartBeatResponse>
         peer.setMatchIndex(-1);
       }
 
-      // only start a catch up when the follower's lastLogIndex remains stall and unchanged for 5
+      // only start a catch up when the follower's lastLogIndex remains stall and unchanged for 3
       // heartbeats
       if (lastLogIdx == peer.getLastHeartBeatIndex()) {
         // the follower's lastLogIndex is unchanged, increase inconsistent counter
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/NodeStatusHandler.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/NodeStatusHandler.java
index 0b18139..4549323 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/NodeStatusHandler.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/NodeStatusHandler.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.cluster.server.handlers.caller;
 
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.utils.nodetool.function.Status;
 
 import org.apache.thrift.async.AsyncMethodCallback;
 
@@ -28,11 +29,11 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 public class NodeStatusHandler implements AsyncMethodCallback<Node> {
 
-  private Map<Node, Boolean> nodeStatusMap;
+  private Map<Node, Integer> nodeStatusMap;
 
   private AtomicInteger countResponse;
 
-  public NodeStatusHandler(Map<Node, Boolean> nodeStatusMap) {
+  public NodeStatusHandler(Map<Node, Integer> nodeStatusMap) {
     this.nodeStatusMap = nodeStatusMap;
     this.countResponse = new AtomicInteger();
   }
@@ -40,7 +41,10 @@ public class NodeStatusHandler implements AsyncMethodCallback<Node> {
   @Override
   public void onComplete(Node response) {
     synchronized (nodeStatusMap) {
-      nodeStatusMap.put(response, true);
+      if (response == null) {
+        return;
+      }
+      nodeStatusMap.put(response, Status.LIVE);
       // except for this node itself
       if (countResponse.incrementAndGet() == nodeStatusMap.size() - 1) {
         nodeStatusMap.notifyAll();
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/DataHeartbeatThread.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/DataHeartbeatThread.java
index e0fe7a8..1044f1b 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/DataHeartbeatThread.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/DataHeartbeatThread.java
@@ -55,19 +55,13 @@ public class DataHeartbeatThread extends HeartbeatThread {
   void startElection() {
     // skip first few elections to let the header have a larger chance to become the leader, so
     // possibly each node will only be one leader at the same time
-    if (!dataGroupMember.getThisNode().equals(dataGroupMember.getHeader())
+    if (!dataGroupMember.getThisNode().equals(dataGroupMember.getHeader().getNode())
         && skippedElectionNumber < MAX_ELECTIONS_TO_SKIP
         && !hasHadLeader) {
       skippedElectionNumber++;
       return;
     }
     electionRequest.setHeader(dataGroupMember.getHeader());
-    electionRequest.setLastLogTerm(
-        dataGroupMember.getMetaGroupMember().getLogManager().getLastLogTerm());
-    electionRequest.setLastLogIndex(
-        dataGroupMember.getMetaGroupMember().getLogManager().getLastLogIndex());
-    electionRequest.setDataLogLastIndex(dataGroupMember.getLogManager().getLastLogIndex());
-    electionRequest.setDataLogLastTerm(dataGroupMember.getLogManager().getLastLogTerm());
 
     super.startElection();
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/HeartbeatThread.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/HeartbeatThread.java
index 0459fef..c9919d3 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/HeartbeatThread.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/HeartbeatThread.java
@@ -84,7 +84,9 @@ public class HeartbeatThread implements Runnable {
           case LEADER:
             // send heartbeats to the followers
             sendHeartbeats();
-            Thread.sleep(RaftServer.getHeartBeatIntervalMs());
+            synchronized (localMember.getHeartBeatWaitObject()) {
+              localMember.getHeartBeatWaitObject().wait(RaftServer.getHeartBeatIntervalMs());
+            }
             hasHadLeader = true;
             break;
           case FOLLOWER:
@@ -101,7 +103,9 @@ public class HeartbeatThread implements Runnable {
                   "{}: Heartbeat from leader {} is still valid",
                   memberName,
                   localMember.getLeader());
-              Thread.sleep(RaftServer.getConnectionTimeoutInMS());
+              synchronized (localMember.getHeartBeatWaitObject()) {
+                localMember.getHeartBeatWaitObject().wait(RaftServer.getConnectionTimeoutInMS());
+              }
             }
             hasHadLeader = true;
             break;
@@ -132,7 +136,7 @@ public class HeartbeatThread implements Runnable {
   }
 
   /** Send each node (except the local node) in the group of the member a heartbeat. */
-  private void sendHeartbeats() {
+  protected void sendHeartbeats() {
     synchronized (localMember.getTerm()) {
       request.setTerm(localMember.getTerm().get());
       request.setLeader(localMember.getThisNode());
@@ -147,7 +151,11 @@ public class HeartbeatThread implements Runnable {
   @SuppressWarnings("java:S2445")
   private void sendHeartbeats(Collection<Node> nodes) {
     if (logger.isDebugEnabled()) {
-      logger.debug("{}: Send heartbeat to {} followers", memberName, nodes.size() - 1);
+      logger.debug(
+          "{}: Send heartbeat to {} followers, commit log index = {}",
+          memberName,
+          nodes.size() - 1,
+          request.getCommitLogIndex());
     }
     synchronized (nodes) {
       // avoid concurrent modification
@@ -268,6 +276,10 @@ public class HeartbeatThread implements Runnable {
   @SuppressWarnings({"java:S2274"})
   // enable timeout
   void startElection() {
+    if (localMember.isSkipElection()) {
+      logger.info("{}: Skip election because this node has stopped.", memberName);
+      return;
+    }
     synchronized (localMember.getTerm()) {
       long nextTerm = localMember.getTerm().incrementAndGet();
       localMember.setVoteFor(localMember.getThisNode());
@@ -290,12 +302,8 @@ public class HeartbeatThread implements Runnable {
 
       electionRequest.setTerm(nextTerm);
       electionRequest.setElector(localMember.getThisNode());
-      if (!electionRequest.isSetLastLogIndex()) {
-        // these field are overridden in DataGroupMember, they will be set to the term and index
-        // of the MetaGroupMember that manages the DataGroupMember so we cannot overwrite them
-        electionRequest.setLastLogTerm(localMember.getLogManager().getLastLogTerm());
-        electionRequest.setLastLogIndex(localMember.getLogManager().getLastLogIndex());
-      }
+      electionRequest.setLastLogTerm(localMember.getLogManager().getLastLogTerm());
+      electionRequest.setLastLogIndex(localMember.getLogManager().getLastLogIndex());
 
       requestVote(
           localMember.getAllNodes(),
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatThread.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatThread.java
index 002c4fe..4036244 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatThread.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatThread.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.cluster.server.heartbeat;
 
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.server.NodeCharacter;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 
 import org.slf4j.Logger;
@@ -70,4 +71,20 @@ public class MetaHeartbeatThread extends HeartbeatThread {
     // erase the sent partition table so it will not be sent in the next heartbeat
     request.unsetPartitionTableBytes();
   }
+
+  @Override
+  void startElection() {
+    super.startElection();
+
+    if (localMetaMember.getCharacter() == NodeCharacter.LEADER) {
+      // A new raft leader needs to have at least one log in its term for committing logs with older
+      // terms.
+      // In the meta group, log frequency is very low. When the leader is changed whiling changing
+      // membership, it's necessary to process an empty log to make sure that cluster expansion
+      // operation can be carried out in time.
+      localMetaMember
+          .getAppendLogThreadPool()
+          .submit(() -> localMetaMember.processEmptyContentLog());
+    }
+  }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
index 6f6936a..4b46e97 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
@@ -31,11 +31,16 @@ import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.exception.LogExecutionException;
 import org.apache.iotdb.cluster.exception.SnapshotInstallationException;
+import org.apache.iotdb.cluster.exception.UnknownLogTypeException;
+import org.apache.iotdb.cluster.log.Log;
 import org.apache.iotdb.cluster.log.LogApplier;
+import org.apache.iotdb.cluster.log.LogParser;
 import org.apache.iotdb.cluster.log.Snapshot;
 import org.apache.iotdb.cluster.log.applier.AsyncDataLogApplier;
 import org.apache.iotdb.cluster.log.applier.DataLogApplier;
+import org.apache.iotdb.cluster.log.logtypes.AddNodeLog;
 import org.apache.iotdb.cluster.log.logtypes.CloseFileLog;
+import org.apache.iotdb.cluster.log.logtypes.RemoveNodeLog;
 import org.apache.iotdb.cluster.log.manage.FilePartitionedSnapshotLogManager;
 import org.apache.iotdb.cluster.log.manage.PartitionedSnapshotLogManager;
 import org.apache.iotdb.cluster.log.snapshot.FileSnapshot;
@@ -56,6 +61,7 @@ import org.apache.iotdb.cluster.rpc.thrift.ElectionRequest;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotResp;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.SendSnapshotRequest;
 import org.apache.iotdb.cluster.server.NodeCharacter;
 import org.apache.iotdb.cluster.server.PullSnapshotHintService;
@@ -80,6 +86,7 @@ import org.apache.iotdb.db.qp.executor.PlanExecutor;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.qp.physical.sys.FlushPlan;
+import org.apache.iotdb.db.qp.physical.sys.LogPlan;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.service.rpc.thrift.EndPoint;
@@ -87,14 +94,15 @@ import org.apache.iotdb.service.rpc.thrift.TSStatus;
 import org.apache.iotdb.tsfile.utils.Pair;
 
 import org.apache.thrift.protocol.TProtocolFactory;
-import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.BufferedInputStream;
 import java.io.DataInputStream;
+import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.file.Files;
@@ -111,6 +119,8 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.iotdb.cluster.config.ClusterConstant.THREAD_POLL_WAIT_TERMINATION_TIME_S;
+
 public class DataGroupMember extends RaftMember {
 
   private static final Logger logger = LoggerFactory.getLogger(DataGroupMember.class);
@@ -151,11 +161,15 @@ public class DataGroupMember extends RaftMember {
    */
   private boolean unchanged;
 
+  private LastAppliedPatitionTableVersion lastAppliedPartitionTableVersion;
+
   @TestOnly
-  public DataGroupMember() {
+  public DataGroupMember(PartitionGroup nodes) {
     // constructor for test
+    allNodes = nodes;
     setQueryManager(new ClusterQueryManager());
     localQueryExecutor = new LocalQueryExecutor(this);
+    lastAppliedPartitionTableVersion = new LastAppliedPatitionTableVersion(getMemberDir());
   }
 
   DataGroupMember(
@@ -164,7 +178,13 @@ public class DataGroupMember extends RaftMember {
       Node thisNode,
       MetaGroupMember metaGroupMember) {
     super(
-        "Data(" + nodes.getHeader().getInternalIp() + ":" + nodes.getHeader().getMetaPort() + ")",
+        "Data("
+            + nodes.getHeader().getNode().getInternalIp()
+            + ":"
+            + nodes.getHeader().getNode().getMetaPort()
+            + ", raftId="
+            + nodes.getId()
+            + ")",
         new AsyncClientPool(new AsyncDataClient.FactoryAsync(factory)),
         new SyncClientPool(new SyncDataClient.FactorySync(factory)),
         new AsyncClientPool(new AsyncDataHeartbeatClient.FactoryAsync(factory)),
@@ -174,7 +194,7 @@ public class DataGroupMember extends RaftMember {
     this.metaGroupMember = metaGroupMember;
     allNodes = nodes;
     setQueryManager(new ClusterQueryManager());
-    slotManager = new SlotManager(ClusterConstant.SLOT_NUM, getMemberDir());
+    slotManager = new SlotManager(ClusterConstant.SLOT_NUM, getMemberDir(), getName());
     LogApplier applier = new DataLogApplier(metaGroupMember, this);
     if (ClusterDescriptor.getInstance().getConfig().isUseAsyncApplier()) {
       applier = new AsyncDataLogApplier(applier, name);
@@ -186,13 +206,12 @@ public class DataGroupMember extends RaftMember {
     term.set(logManager.getHardState().getCurrentTerm());
     voteFor = logManager.getHardState().getVoteFor();
     localQueryExecutor = new LocalQueryExecutor(this);
+    lastAppliedPartitionTableVersion = new LastAppliedPatitionTableVersion(getMemberDir());
   }
 
   /**
    * Start heartbeat, catch-up, pull snapshot services and start all unfinished pull-snapshot-tasks.
    * Calling the method twice does not induce side effects.
-   *
-   * @throws TTransportException
    */
   @Override
   public void start() {
@@ -203,6 +222,7 @@ public class DataGroupMember extends RaftMember {
     heartBeatService.submit(new DataHeartbeatThread(this));
     pullSnapshotService = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
     pullSnapshotHintService = new PullSnapshotHintService(this);
+    pullSnapshotHintService.start();
     resumePullSnapshotTasks();
   }
 
@@ -217,7 +237,7 @@ public class DataGroupMember extends RaftMember {
     if (pullSnapshotService != null) {
       pullSnapshotService.shutdownNow();
       try {
-        pullSnapshotService.awaitTermination(10, TimeUnit.SECONDS);
+        pullSnapshotService.awaitTermination(THREAD_POLL_WAIT_TERMINATION_TIME_S, TimeUnit.SECONDS);
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         logger.error("Unexpected interruption when waiting for pullSnapshotService to end", e);
@@ -234,6 +254,21 @@ public class DataGroupMember extends RaftMember {
     logger.info("{}: stopped", name);
   }
 
+  @Override
+  long checkElectorLogProgress(ElectionRequest electionRequest) {
+    Node elector = electionRequest.getElector();
+    // check if the node is in the group
+    if (!allNodes.contains(elector)) {
+      logger.info(
+          "{}: the elector {} is not in the data group {}, so reject this election.",
+          name,
+          getPartitionGroup(),
+          elector);
+      return Response.RESPONSE_NODE_IS_NOT_IN_GROUP;
+    }
+    return super.checkElectorLogProgress(electionRequest);
+  }
+
   /**
    * The first node (on the hash ring) in this data group is the header. It determines the duty
    * (what range on the ring do the group take responsibility for) of the group and although other
@@ -241,8 +276,8 @@ public class DataGroupMember extends RaftMember {
    * also the identifier of this data group.
    */
   @Override
-  public Node getHeader() {
-    return allNodes.get(0);
+  public RaftNode getHeader() {
+    return allNodes.getHeader();
   }
 
   public ClusterQueryManager getQueryManager() {
@@ -268,36 +303,14 @@ public class DataGroupMember extends RaftMember {
     }
   }
 
-  /**
-   * Try to add a Node into the group to which the member belongs.
-   *
-   * @param node
-   * @return true if this node should leave the group because of the addition of the node, false
-   *     otherwise
-   */
-  public synchronized boolean addNode(Node node, NodeAdditionResult result) {
-    // when a new node is added, start an election instantly to avoid the stale leader still
-    // taking the leadership, which guarantees the valid leader will not have the stale
-    // partition table
-    synchronized (term) {
-      term.incrementAndGet();
-      setLeader(ClusterConstant.EMPTY_NODE);
-      setVoteFor(thisNode);
-      updateHardState(term.get(), getVoteFor());
-      setLastHeartbeatReceivedTime(System.currentTimeMillis());
-      setCharacter(NodeCharacter.ELECTOR);
-    }
-
-    // mark slots that do not belong to this group any more
-    Set<Integer> lostSlots =
-        ((SlotNodeAdditionResult) result)
-            .getLostSlots()
-            .getOrDefault(getHeader(), Collections.emptySet());
-    for (Integer lostSlot : lostSlots) {
-      slotManager.setToSending(lostSlot);
+  public boolean preAddNode(Node node) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to pre adding node {}", name, node);
     }
-
     synchronized (allNodes) {
+      if (allNodes.contains(node)) {
+        return false;
+      }
       int insertIndex = -1;
       // find the position to insert the new node, the nodes are ordered by their identifiers
       for (int i = 0; i < allNodes.size() - 1; i++) {
@@ -315,97 +328,61 @@ public class DataGroupMember extends RaftMember {
       if (insertIndex > 0) {
         allNodes.add(insertIndex, node);
         peerMap.putIfAbsent(node, new Peer(logManager.getLastLogIndex()));
-        // remove the last node because the group size is fixed to replication number
-        Node removedNode = allNodes.remove(allNodes.size() - 1);
-        peerMap.remove(removedNode);
         // if the local node is the last node and the insertion succeeds, this node should leave
         // the group
         logger.debug("{}: Node {} is inserted into the data group {}", name, node, allNodes);
-        return removedNode.equals(thisNode);
       }
-      return false;
+      return insertIndex > 0;
     }
   }
 
   /**
-   * Process the election request from another node in the group. To win the vote from the local
-   * member, a node must have both meta and data logs no older than then local member, or it will be
-   * turned down.
+   * Try to add a Node into the group to which the member belongs.
    *
-   * @param electionRequest
-   * @return Response.RESPONSE_META_LOG_STALE if the meta logs of the elector fall behind
-   *     Response.RESPONSE_LOG_MISMATCH if the data logs of the elector fall behind Response.SUCCESS
-   *     if the vote is given to the elector the term of local member if the elector's term is no
-   *     bigger than the local member
+   * @return true if this node should leave the group because of the addition of the node, false
+   *     otherwise
    */
-  @Override
-  long checkElectorLogProgress(ElectionRequest electionRequest) {
-    // to be a data group leader, a node should also be qualified to be the meta group leader
-    // which guarantees the data group leader has the newest partition table.
-    long thatTerm = electionRequest.getTerm();
-    long thatMetaLastLogIndex = electionRequest.getLastLogIndex();
-    long thatMetaLastLogTerm = electionRequest.getLastLogTerm();
-    long thatDataLastLogIndex = electionRequest.getDataLogLastIndex();
-    long thatDataLastLogTerm = electionRequest.getDataLogLastTerm();
-    logger.info(
-        "{} received an dataGroup election request, term:{}, metaLastLogIndex:{}, metaLastLogTerm:{}, dataLastLogIndex:{}, dataLastLogTerm:{}",
-        name,
-        thatTerm,
-        thatMetaLastLogIndex,
-        thatMetaLastLogTerm,
-        thatDataLastLogIndex,
-        thatDataLastLogTerm);
-
-    // check meta logs
-    // term of the electors' MetaGroupMember is not verified, so 0 and 1 are used to make sure
-    // the verification does not fail
-    long metaResponse = metaGroupMember.checkLogProgress(thatMetaLastLogIndex, thatMetaLastLogTerm);
-    if (metaResponse == Response.RESPONSE_LOG_MISMATCH) {
-      return Response.RESPONSE_META_LOG_STALE;
-    }
-
-    long resp = checkLogProgress(thatDataLastLogIndex, thatDataLastLogTerm);
-    if (resp == Response.RESPONSE_AGREE) {
-      logger.info(
-          "{} accepted an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
-      setCharacter(NodeCharacter.FOLLOWER);
-      lastHeartbeatReceivedTime = System.currentTimeMillis();
-      setVoteFor(electionRequest.getElector());
-      updateHardState(thatTerm, getVoteFor());
-    } else {
-      logger.info(
-          "{} rejected an dataGroup election request, term:{}/{}, dataLogIndex:{}/{}, dataLogTerm:{}/{}, metaLogIndex:{}/{},metaLogTerm:{}/{}",
-          name,
-          thatTerm,
-          term.get(),
-          thatDataLastLogIndex,
-          logManager.getLastLogIndex(),
-          thatDataLastLogTerm,
-          logManager.getLastLogTerm(),
-          thatMetaLastLogIndex,
-          metaGroupMember.getLogManager().getLastLogIndex(),
-          thatMetaLastLogTerm,
-          metaGroupMember.getLogManager().getLastLogTerm());
+  public boolean addNode(Node node, NodeAdditionResult result) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to add node {}", name, node);
+    }
+
+    // mark slots that do not belong to this group any more
+    Set<Integer> lostSlots =
+        ((SlotNodeAdditionResult) result)
+            .getLostSlots()
+            .getOrDefault(getHeader(), Collections.emptySet());
+    for (Integer lostSlot : lostSlots) {
+      slotManager.setToSending(lostSlot, false);
+    }
+    slotManager.save();
+
+    synchronized (allNodes) {
+      preAddNode(node);
+      if (allNodes.contains(node) && allNodes.size() > config.getReplicationNum()) {
+        // remove the last node because the group size is fixed to replication number
+        Node removedNode = allNodes.remove(allNodes.size() - 1);
+        peerMap.remove(removedNode);
+
+        if (removedNode.equals(leader.get()) && !removedNode.equals(thisNode)) {
+          // if the leader is removed, also start an election immediately
+          synchronized (term) {
+            setCharacter(NodeCharacter.ELECTOR);
+            setLeader(null);
+          }
+          synchronized (getHeartBeatWaitObject()) {
+            getHeartBeatWaitObject().notifyAll();
+          }
+        }
+        return removedNode.equals(thisNode);
+      }
+      return false;
     }
-    return resp;
   }
 
   /**
    * Deserialize and install a snapshot sent by the leader. The type of the snapshot must be
    * currently PartitionedSnapshot with FileSnapshot inside.
-   *
-   * @param request
    */
   public void receiveSnapshot(SendSnapshotRequest request) throws SnapshotInstallationException {
     logger.info(
@@ -420,25 +397,24 @@ public class DataGroupMember extends RaftMember {
     if (logger.isDebugEnabled()) {
       logger.debug("{} received a snapshot {}", name, snapshot);
     }
-    snapshot.getDefaultInstaller(this).install(snapshot, -1);
+    snapshot.getDefaultInstaller(this).install(snapshot, -1, false);
   }
 
-  /**
-   * Send the requested snapshots to the applier node.
-   *
-   * @param request
-   */
+  /** Send the requested snapshots to the applier node. */
   public PullSnapshotResp getSnapshot(PullSnapshotRequest request) throws IOException {
-    waitLeader();
-    if (character != NodeCharacter.LEADER && !readOnly) {
-      return null;
-    }
     // if the requester pulls the snapshots because the header of the group is removed, then the
     // member should no longer receive new data
     if (request.isRequireReadOnly()) {
       setReadOnly();
     }
 
+    // Make sure local data is complete.
+    if (character != NodeCharacter.LEADER
+        && lastAppliedPartitionTableVersion.getVersion()
+            != metaGroupMember.getPartitionTable().getLastMetaLogIndex()) {
+      return null;
+    }
+
     List<Integer> requiredSlots = request.getRequiredSlots();
     for (Integer requiredSlot : requiredSlots) {
       // wait if the data of the slot is in another node
@@ -475,7 +451,8 @@ public class DataGroupMember extends RaftMember {
     synchronized (logManager) {
       PullSnapshotResp resp = new PullSnapshotResp();
       Map<Integer, ByteBuffer> resultMap = new HashMap<>();
-      logManager.takeSnapshot();
+      ((PartitionedSnapshotLogManager) logManager)
+          .takeSnapshotForSpecificSlots(requiredSlots, false);
 
       PartitionedSnapshot<Snapshot> allSnapshot = (PartitionedSnapshot) logManager.getSnapshot();
       for (int requiredSlot : requiredSlots) {
@@ -490,40 +467,34 @@ public class DataGroupMember extends RaftMember {
     }
   }
 
-  /**
-   * Pull snapshots from the previous holders after newNode joins the cluster.
-   *
-   * @param slots
-   * @param newNode
-   */
+  /** Pull snapshots from the previous holders after newNode joins the cluster. */
   public void pullNodeAdditionSnapshots(List<Integer> slots, Node newNode) {
+    // group the slots by their owners
+    Map<PartitionGroup, List<Integer>> holderSlotsMap = new HashMap<>();
     synchronized (logManager) {
       logger.info("{} pulling {} slots from remote", name, slots.size());
       PartitionedSnapshot<Snapshot> snapshot = (PartitionedSnapshot) logManager.getSnapshot();
-      Map<Integer, Node> prevHolders =
-          ((SlotPartitionTable) metaGroupMember.getPartitionTable()).getPreviousNodeMap(newNode);
+      Map<Integer, PartitionGroup> prevHolders =
+          ((SlotPartitionTable) metaGroupMember.getPartitionTable())
+              .getPreviousNodeMap(new RaftNode(newNode, getRaftGroupId()));
 
-      // group the slots by their owners
-      Map<Node, List<Integer>> holderSlotsMap = new HashMap<>();
       for (int slot : slots) {
         // skip the slot if the corresponding data is already replicated locally
         if (snapshot.getSnapshot(slot) == null) {
-          Node node = prevHolders.get(slot);
-          if (node != null) {
-            holderSlotsMap.computeIfAbsent(node, n -> new ArrayList<>()).add(slot);
+          PartitionGroup group = prevHolders.get(slot);
+          if (group != null) {
+            holderSlotsMap.computeIfAbsent(group, n -> new ArrayList<>()).add(slot);
           }
         }
       }
+    }
 
-      // pull snapshots from each owner's data group
-      for (Entry<Node, List<Integer>> entry : holderSlotsMap.entrySet()) {
-        Node node = entry.getKey();
-        List<Integer> nodeSlots = entry.getValue();
-        PullSnapshotTaskDescriptor taskDescriptor =
-            new PullSnapshotTaskDescriptor(
-                metaGroupMember.getPartitionTable().getHeaderGroup(node), nodeSlots, false);
-        pullFileSnapshot(taskDescriptor, null);
-      }
+    // pull snapshots from each owner's data group
+    for (Entry<PartitionGroup, List<Integer>> entry : holderSlotsMap.entrySet()) {
+      List<Integer> nodeSlots = entry.getValue();
+      PullSnapshotTaskDescriptor taskDescriptor =
+          new PullSnapshotTaskDescriptor(entry.getKey(), nodeSlots, false);
+      pullFileSnapshot(taskDescriptor, null);
     }
   }
 
@@ -536,6 +507,17 @@ public class DataGroupMember extends RaftMember {
    *     ot null otherwise
    */
   private void pullFileSnapshot(PullSnapshotTaskDescriptor descriptor, File snapshotSave) {
+    // If this node is the member of previous holder, it's unnecessary to pull data again
+    if (descriptor.getPreviousHolders().contains(thisNode)) {
+      logger.info(
+          "{}: {} and other {} don't need to pull because there already has such data locally",
+          name,
+          descriptor.getSlots().get(0),
+          descriptor.getSlots().size() - 1);
+      // inform the previous holders that one member has successfully pulled snapshot directly
+      registerPullSnapshotHint(descriptor);
+      return;
+    }
     Iterator<Integer> iterator = descriptor.getSlots().iterator();
     while (iterator.hasNext()) {
       Integer nodeSlot = iterator.next();
@@ -545,9 +527,11 @@ public class DataGroupMember extends RaftMember {
         iterator.remove();
       } else {
         // mark the slot as pulling to control reads and writes of the pulling slot
-        slotManager.setToPulling(nodeSlot, descriptor.getPreviousHolders().getHeader());
+        slotManager.setToPulling(
+            nodeSlot, descriptor.getPreviousHolders().getHeader().getNode(), false);
       }
     }
+    slotManager.save();
 
     if (descriptor.getSlots().isEmpty()) {
       return;
@@ -605,7 +589,9 @@ public class DataGroupMember extends RaftMember {
         + File.separator
         + "raft"
         + File.separator
-        + getHeader().nodeIdentifier
+        + getHeader().getNode().nodeIdentifier
+        + File.separator
+        + getRaftGroupId()
         + File.separator;
   }
 
@@ -616,14 +602,10 @@ public class DataGroupMember extends RaftMember {
   /**
    * If the member is the leader, let all members in the group close the specified partition of a
    * storage group, else just return false.
-   *
-   * @param storageGroupName
-   * @param partitionId
-   * @param isSeq
    */
-  void closePartition(String storageGroupName, long partitionId, boolean isSeq) {
+  boolean closePartition(String storageGroupName, long partitionId, boolean isSeq) {
     if (character != NodeCharacter.LEADER) {
-      return;
+      return false;
     }
     CloseFileLog log = new CloseFileLog(storageGroupName, partitionId, isSeq);
     synchronized (logManager) {
@@ -635,15 +617,18 @@ public class DataGroupMember extends RaftMember {
       logger.info("Send the close file request of {} to other nodes", log);
     }
     try {
-      appendLogInGroup(log);
+      return appendLogInGroup(log);
     } catch (LogExecutionException e) {
       logger.error("Cannot close partition {}#{} seq:{}", storageGroupName, partitionId, isSeq, e);
     }
+    return false;
   }
 
   public boolean flushFileWhenDoSnapshot(
-      Map<String, List<Pair<Long, Boolean>>> storageGroupPartitions) {
-    if (character != NodeCharacter.LEADER) {
+      Map<String, List<Pair<Long, Boolean>>> storageGroupPartitions,
+      List<Integer> requiredSlots,
+      boolean needLeader) {
+    if (needLeader && character != NodeCharacter.LEADER) {
       return false;
     }
 
@@ -655,15 +640,12 @@ public class DataGroupMember extends RaftMember {
       String storageGroupName = entry.getKey();
       List<Pair<Long, Boolean>> tmpPairList = entry.getValue();
       for (Pair<Long, Boolean> pair : tmpPairList) {
-        long partitionId = pair.left;
-        Node header =
-            metaGroupMember
-                .getPartitionTable()
-                .routeToHeaderByTime(
-                    storageGroupName, partitionId * StorageEngine.getTimePartitionInterval());
-        DataGroupMember localDataMember = metaGroupMember.getLocalDataMember(header);
-        if (localDataMember.getHeader().equals(this.getHeader())) {
-          localListPair.add(new Pair<>(partitionId, pair.right));
+        long timestamp = pair.left * StorageEngine.getTimePartitionInterval();
+        int slotId =
+            SlotPartitionTable.getSlotStrategy()
+                .calculateSlotByTime(storageGroupName, timestamp, ClusterConstant.SLOT_NUM);
+        if (requiredSlots.contains(slotId)) {
+          localListPair.add(pair);
         }
       }
       try {
@@ -692,13 +674,23 @@ public class DataGroupMember extends RaftMember {
    * process through the raft procedure, otherwise the plan will be forwarded to the leader.
    *
    * @param plan a non-query plan.
-   * @return
    */
   @Override
   public TSStatus executeNonQueryPlan(PhysicalPlan plan) {
     if (ClusterDescriptor.getInstance().getConfig().getReplicationNum() == 1) {
       try {
-        getLocalExecutor().processNonQuery(plan);
+        if (plan instanceof LogPlan) {
+          Log log;
+          try {
+            log = LogParser.getINSTANCE().parse(((LogPlan) plan).getLog());
+          } catch (UnknownLogTypeException e) {
+            logger.error("Can not parse LogPlan {}", plan, e);
+            return StatusUtils.PARSE_LOG_ERROR;
+          }
+          handleChangeMembershipLogWithoutRaft(log);
+        } else {
+          getLocalExecutor().processNonQuery(plan);
+        }
         return StatusUtils.OK;
       } catch (Exception e) {
         Throwable cause = IOUtils.getRootCause(e);
@@ -733,6 +725,28 @@ public class DataGroupMember extends RaftMember {
     }
   }
 
+  private void handleChangeMembershipLogWithoutRaft(Log log) {
+    if (log instanceof AddNodeLog) {
+      if (!metaGroupMember
+          .getPartitionTable()
+          .deserialize(((AddNodeLog) log).getPartitionTable())) {
+        return;
+      }
+      preAddNode(((AddNodeLog) log).getNewNode());
+      setAndSaveLastAppliedPartitionTableVersion(((AddNodeLog) log).getMetaLogIndex());
+    } else if (log instanceof RemoveNodeLog) {
+      if (!metaGroupMember
+          .getPartitionTable()
+          .deserialize(((RemoveNodeLog) log).getPartitionTable())) {
+        return;
+      }
+      preRemoveNode(((RemoveNodeLog) log).getRemovedNode());
+      setAndSaveLastAppliedPartitionTableVersion(((RemoveNodeLog) log).getMetaLogIndex());
+    } else {
+      logger.error("Unsupported log: {}", log);
+    }
+  }
+
   private TSStatus executeNonQueryPlanWithKnownLeader(PhysicalPlan plan) {
     if (character == NodeCharacter.LEADER) {
       long startTime = Statistic.DATA_GROUP_MEMBER_LOCAL_EXECUTION.getOperationStartTime();
@@ -771,14 +785,23 @@ public class DataGroupMember extends RaftMember {
               SlotPartitionTable.getSlotStrategy()
                   .calculateSlotByPartitionNum(
                       storageGroupName, timePartitionId, ClusterConstant.SLOT_NUM);
+          /**
+           * If this slot is just held by different raft groups in the same node, it should keep the
+           * data of slot.
+           */
+          if (((SlotPartitionTable) metaGroupMember.getPartitionTable())
+              .judgeHoldSlot(thisNode, slot)) {
+            return false;
+          }
           return slotSet.contains(slot);
         };
     for (PartialPath sg : allStorageGroupNames) {
       StorageEngine.getInstance().removePartitions(sg, filter);
     }
     for (Integer slot : slots) {
-      slotManager.setToNull(slot);
+      slotManager.setToNull(slot, false);
     }
+    slotManager.save();
 
     if (logger.isInfoEnabled()) {
       logger.info(
@@ -786,42 +809,70 @@ public class DataGroupMember extends RaftMember {
     }
   }
 
+  public void preRemoveNode(Node removedNode) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to pre remove node {}", name, removedNode);
+    }
+    synchronized (allNodes) {
+      if (allNodes.contains(removedNode) && allNodes.size() == config.getReplicationNum()) {
+        // update the group if the deleted node was in it
+        PartitionGroup newGroup = metaGroupMember.getPartitionTable().getHeaderGroup(getHeader());
+        if (newGroup == null) {
+          return;
+        }
+        Node newNodeToGroup = newGroup.get(newGroup.size() - 1);
+        allNodes.add(newNodeToGroup);
+        peerMap.putIfAbsent(newNodeToGroup, new Peer(logManager.getLastLogIndex()));
+      }
+    }
+  }
+
   /**
    * When a node is removed and IT IS NOT THE HEADER of the group, the member should take over some
    * slots from the removed group, and add a new node to the group the removed node was in the
    * group.
    */
   @SuppressWarnings("java:S2445") // the reference of allNodes is unchanged
-  public void removeNode(Node removedNode, NodeRemovalResult removalResult) {
+  public void removeNode(Node removedNode) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: start to remove node {}", name, removedNode);
+    }
+
     synchronized (allNodes) {
+      preRemoveNode(removedNode);
       if (allNodes.contains(removedNode)) {
         // update the group if the deleted node was in it
-        allNodes = metaGroupMember.getPartitionTable().getHeaderGroup(getHeader());
-        initPeerMap();
+        allNodes.remove(removedNode);
+        peerMap.remove(removedNode);
         if (removedNode.equals(leader.get())) {
           // if the leader is removed, also start an election immediately
           synchronized (term) {
             setCharacter(NodeCharacter.ELECTOR);
-            setLastHeartbeatReceivedTime(Long.MIN_VALUE);
+            setLeader(null);
+          }
+          synchronized (getHeartBeatWaitObject()) {
+            getHeartBeatWaitObject().notifyAll();
           }
         }
       }
-      List<Integer> slotsToPull =
-          ((SlotNodeRemovalResult) removalResult).getNewSlotOwners().get(getHeader());
-      if (slotsToPull != null) {
-        // pull the slots that should be taken over
-        PullSnapshotTaskDescriptor taskDescriptor =
-            new PullSnapshotTaskDescriptor(removalResult.getRemovedGroup(), slotsToPull, true);
-        pullFileSnapshot(taskDescriptor, null);
-      }
+    }
+  }
+
+  public void pullSlots(NodeRemovalResult removalResult) {
+    List<Integer> slotsToPull =
+        ((SlotNodeRemovalResult) removalResult).getNewSlotOwners().get(getHeader());
+    if (slotsToPull != null) {
+      // pull the slots that should be taken over
+      PullSnapshotTaskDescriptor taskDescriptor =
+          new PullSnapshotTaskDescriptor(
+              removalResult.getRemovedGroup(getRaftGroupId()), new ArrayList<>(slotsToPull), true);
+      pullFileSnapshot(taskDescriptor, null);
     }
   }
 
   /**
    * Generate a report containing the character, leader, term, last log term, last log index, header
    * and readOnly or not of this member.
-   *
-   * @return
    */
   public DataMemberReport genReport() {
     long prevLastLogIndex = lastReportedLogIndex;
@@ -836,7 +887,7 @@ public class DataGroupMember extends RaftMember {
         logManager.getCommitLogTerm(),
         getHeader(),
         readOnly,
-        NodeStatusManager.getINSTANCE().getLastResponseLatency(getHeader()),
+        NodeStatusManager.getINSTANCE().getLastResponseLatency(getHeader().getNode()),
         lastHeartbeatReceivedTime,
         prevLastLogIndex,
         logManager.getMaxHaveAppliedCommitIndex());
@@ -863,13 +914,25 @@ public class DataGroupMember extends RaftMember {
   }
 
   public boolean onSnapshotInstalled(List<Integer> slots) {
+    if (getMetaGroupMember().getPartitionTable().getAllNodes().contains(thisNode)) {
+      getMetaGroupMember()
+          .syncLocalApply(getMetaGroupMember().getPartitionTable().getLastMetaLogIndex(), false);
+    }
+    if (logger.isDebugEnabled()) {
+      logger.debug(
+          "{} received one replication snapshot installed of slot {} and other {} slots",
+          name,
+          slots.get(0),
+          slots.size() - 1);
+    }
     List<Integer> removableSlots = new ArrayList<>();
     for (Integer slot : slots) {
-      int sentReplicaNum = slotManager.sentOneReplication(slot);
-      if (sentReplicaNum >= ClusterDescriptor.getInstance().getConfig().getReplicationNum()) {
+      int sentReplicaNum = slotManager.sentOneReplication(slot, false);
+      if (sentReplicaNum >= config.getReplicationNum()) {
         removableSlots.add(slot);
       }
     }
+    slotManager.save();
     removeLocalData(removableSlots);
     return true;
   }
@@ -878,6 +941,27 @@ public class DataGroupMember extends RaftMember {
     pullSnapshotHintService.registerHint(descriptor);
   }
 
+  /**
+   * Find the groups that should be queried due to data migration. When a slot is in the status of
+   * PULLING or PULLING_WRITABLE, the read of it should merge result to guarantee integrity.
+   */
+  public Map<PartitionGroup, Set<Integer>> getPreviousHolderSlotMap() {
+    Map<PartitionGroup, Set<Integer>> holderSlotMap = new HashMap<>();
+    RaftNode header = getHeader();
+    Map<RaftNode, Map<Integer, PartitionGroup>> previousHolderMap =
+        ((SlotPartitionTable) getMetaGroupMember().getPartitionTable()).getPreviousNodeMap();
+    if (previousHolderMap.containsKey(header)) {
+      for (Entry<Integer, PartitionGroup> entry : previousHolderMap.get(header).entrySet()) {
+        int slot = entry.getKey();
+        PartitionGroup holder = entry.getValue();
+        if (slotManager.checkSlotInDataMigrationStatus(slot)) {
+          holderSlotMap.computeIfAbsent(holder, n -> new HashSet<>()).add(slot);
+        }
+      }
+    }
+    return holderSlotMap;
+  }
+
   public LocalQueryExecutor getLocalQueryExecutor() {
     return localQueryExecutor;
   }
@@ -894,4 +978,57 @@ public class DataGroupMember extends RaftMember {
   public void setUnchanged(boolean unchanged) {
     this.unchanged = unchanged;
   }
+
+  public void setAndSaveLastAppliedPartitionTableVersion(long version) {
+    lastAppliedPartitionTableVersion.setVersion(version);
+    lastAppliedPartitionTableVersion.save();
+  }
+
+  private class LastAppliedPatitionTableVersion {
+
+    private static final String VERSION_FILE_NAME = "LAST_PARTITION_TABLE_VERSION";
+
+    private long version = -1;
+
+    private String filePath;
+
+    public LastAppliedPatitionTableVersion(String memberDir) {
+      this.filePath = memberDir + File.separator + VERSION_FILE_NAME;
+      load();
+    }
+
+    private void load() {
+      File versionFile = new File(filePath);
+      if (!versionFile.exists()) {
+        return;
+      }
+      try (FileInputStream fileInputStream = new FileInputStream(filePath);
+          DataInputStream dataInputStream = new DataInputStream(fileInputStream)) {
+        version = dataInputStream.readLong();
+      } catch (Exception e) {
+        logger.warn("Cannot deserialize last partition table version from {}", filePath, e);
+      }
+    }
+
+    public synchronized void save() {
+      File versionFile = new File(filePath);
+      if (!versionFile.getParentFile().exists() && !versionFile.getParentFile().mkdirs()) {
+        logger.warn("Cannot mkdirs for {}", versionFile);
+      }
+      try (FileOutputStream outputStream = new FileOutputStream(versionFile);
+          DataOutputStream dataOutputStream = new DataOutputStream(outputStream)) {
+        dataOutputStream.writeLong(version);
+      } catch (IOException e) {
+        logger.warn("Last partition table version in {} cannot be saved", filePath, e);
+      }
+    }
+
+    public long getVersion() {
+      return version;
+    }
+
+    public void setVersion(long version) {
+      this.version = version;
+    }
+  }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
index 21dfd68..be1fc7d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
@@ -31,6 +31,7 @@ import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.coordinator.Coordinator;
 import org.apache.iotdb.cluster.exception.AddSelfException;
+import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.exception.ConfigInconsistentException;
 import org.apache.iotdb.cluster.exception.EmptyIntervalException;
 import org.apache.iotdb.cluster.exception.LogExecutionException;
@@ -41,6 +42,7 @@ import org.apache.iotdb.cluster.log.Log;
 import org.apache.iotdb.cluster.log.LogApplier;
 import org.apache.iotdb.cluster.log.applier.MetaLogApplier;
 import org.apache.iotdb.cluster.log.logtypes.AddNodeLog;
+import org.apache.iotdb.cluster.log.logtypes.EmptyContentLog;
 import org.apache.iotdb.cluster.log.logtypes.RemoveNodeLog;
 import org.apache.iotdb.cluster.log.manage.MetaSingleSnapshotLogManager;
 import org.apache.iotdb.cluster.log.snapshot.MetaSimpleSnapshot;
@@ -51,11 +53,14 @@ import org.apache.iotdb.cluster.partition.PartitionTable;
 import org.apache.iotdb.cluster.partition.slot.SlotPartitionTable;
 import org.apache.iotdb.cluster.query.ClusterPlanRouter;
 import org.apache.iotdb.cluster.rpc.thrift.AddNodeResponse;
-import org.apache.iotdb.cluster.rpc.thrift.AppendEntryRequest;
 import org.apache.iotdb.cluster.rpc.thrift.CheckStatusResponse;
+import org.apache.iotdb.cluster.rpc.thrift.ElectionRequest;
 import org.apache.iotdb.cluster.rpc.thrift.HeartBeatRequest;
 import org.apache.iotdb.cluster.rpc.thrift.HeartBeatResponse;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService.Client;
 import org.apache.iotdb.cluster.rpc.thrift.SendSnapshotRequest;
 import org.apache.iotdb.cluster.rpc.thrift.StartUpStatus;
 import org.apache.iotdb.cluster.rpc.thrift.TSMetaService;
@@ -66,7 +71,6 @@ import org.apache.iotdb.cluster.server.HardLinkCleaner;
 import org.apache.iotdb.cluster.server.NodeCharacter;
 import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.Response;
-import org.apache.iotdb.cluster.server.handlers.caller.AppendGroupEntryHandler;
 import org.apache.iotdb.cluster.server.handlers.caller.GenericHandler;
 import org.apache.iotdb.cluster.server.handlers.caller.NodeStatusHandler;
 import org.apache.iotdb.cluster.server.heartbeat.DataHeartbeatServer;
@@ -81,6 +85,7 @@ import org.apache.iotdb.cluster.utils.ClusterUtils;
 import org.apache.iotdb.cluster.utils.PartitionUtils;
 import org.apache.iotdb.cluster.utils.PartitionUtils.Intervals;
 import org.apache.iotdb.cluster.utils.StatusUtils;
+import org.apache.iotdb.cluster.utils.nodetool.function.Status;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.exception.StartupException;
@@ -123,6 +128,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.ExecutorService;
@@ -130,10 +136,10 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 
+import static org.apache.iotdb.cluster.config.ClusterConstant.THREAD_POLL_WAIT_TERMINATION_TIME_S;
 import static org.apache.iotdb.cluster.utils.ClusterUtils.WAIT_START_UP_CHECK_TIME_SEC;
 import static org.apache.iotdb.cluster.utils.ClusterUtils.analyseStartUpCheckResult;
 
@@ -149,6 +155,10 @@ public class MetaGroupMember extends RaftMember {
   /** in case of data loss, some file changes would be made to a temporary file first */
   private static final String TEMP_SUFFIX = ".tmp";
 
+  private static final String MSG_MULTIPLE_ERROR =
+      "The following errors occurred when executing "
+          + "the query, please retry or contact the DBA: ";
+
   private static final Logger logger = LoggerFactory.getLogger(MetaGroupMember.class);
   /**
    * when joining a cluster this node will retry at most "DEFAULT_JOIN_RETRY" times before returning
@@ -162,10 +172,6 @@ public class MetaGroupMember extends RaftMember {
    */
   private static final int REPORT_INTERVAL_SEC = 10;
 
-  /** how many times is a data record replicated, also the number of nodes in a data group */
-  private static final int REPLICATION_NUM =
-      ClusterDescriptor.getInstance().getConfig().getReplicationNum();
-
   /**
    * during snapshot, hardlinks of data files are created to for downloading. hardlinks will be
    * checked every hour by default to see if they have expired, and will be cleaned if so.
@@ -250,7 +256,7 @@ public class MetaGroupMember extends RaftMember {
         new SyncClientPool(new SyncMetaClient.FactorySync(factory)),
         new AsyncClientPool(new AsyncMetaHeartbeatClient.FactoryAsync(factory)),
         new SyncClientPool(new SyncMetaHeartbeatClient.FactorySync(factory)));
-    allNodes = new ArrayList<>();
+    allNodes = new PartitionGroup();
     initPeerMap();
 
     dataClientProvider = new DataClientProvider(factory);
@@ -281,18 +287,16 @@ public class MetaGroupMember extends RaftMember {
    * Find the DataGroupMember that manages the partition of "storageGroupName"@"partitionId", and
    * close the partition through that member. Notice: only partitions owned by this node can be
    * closed by the method.
-   *
-   * @return true if the member is a leader and the partition is closed, false otherwise
    */
-  public void closePartition(String storageGroupName, long partitionId, boolean isSeq) {
-    Node header =
+  public boolean closePartition(String storageGroupName, long partitionId, boolean isSeq) {
+    RaftNode raftNode =
         partitionTable.routeToHeaderByTime(
             storageGroupName, partitionId * StorageEngine.getTimePartitionInterval());
-    DataGroupMember localDataMember = getLocalDataMember(header);
+    DataGroupMember localDataMember = getLocalDataMember(raftNode);
     if (localDataMember == null || localDataMember.getCharacter() != NodeCharacter.LEADER) {
-      return;
+      return false;
     }
-    localDataMember.closePartition(storageGroupName, partitionId, isSeq);
+    return localDataMember.closePartition(storageGroupName, partitionId, isSeq);
   }
 
   public DataClusterServer getDataClusterServer() {
@@ -346,7 +350,7 @@ public class MetaGroupMember extends RaftMember {
     if (reportThread != null) {
       reportThread.shutdownNow();
       try {
-        reportThread.awaitTermination(10, TimeUnit.SECONDS);
+        reportThread.awaitTermination(THREAD_POLL_WAIT_TERMINATION_TIME_S, TimeUnit.SECONDS);
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         logger.error("Unexpected interruption when waiting for reportThread to end", e);
@@ -355,13 +359,13 @@ public class MetaGroupMember extends RaftMember {
     if (hardLinkCleanerThread != null) {
       hardLinkCleanerThread.shutdownNow();
       try {
-        hardLinkCleanerThread.awaitTermination(10, TimeUnit.SECONDS);
+        hardLinkCleanerThread.awaitTermination(
+            THREAD_POLL_WAIT_TERMINATION_TIME_S, TimeUnit.SECONDS);
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         logger.error("Unexpected interruption when waiting for hardlinkCleaner to end", e);
       }
     }
-
     logger.info("{}: stopped", name);
   }
 
@@ -405,22 +409,35 @@ public class MetaGroupMember extends RaftMember {
    * Apply the addition of a new node. Register its identifier, add it to the node list and
    * partition table, serialize the partition table and update the DataGroupMembers.
    */
-  public void applyAddNode(Node newNode) {
+  public void applyAddNode(AddNodeLog addNodeLog) {
+
+    long startTime = System.currentTimeMillis();
+    Node newNode = addNodeLog.getNewNode();
     synchronized (allNodes) {
+      if (logger.isDebugEnabled()) {
+        logger.debug("{}: adding a new node {} into {}", name, newNode, allNodes);
+      }
+
       if (!allNodes.contains(newNode)) {
-        logger.debug("Adding a new node {} into {}", newNode, allNodes);
         registerNodeIdentifier(newNode, newNode.getNodeIdentifier());
         allNodes.add(newNode);
+      }
 
-        // update the partition table
-        NodeAdditionResult result = partitionTable.addNode(newNode);
-        ((SlotPartitionTable) partitionTable).setLastLogIndex(logManager.getLastLogIndex());
-        savePartitionTable();
+      // update the partition table
+      savePartitionTable();
 
-        // update local data members
-        getDataClusterServer().addNode(newNode, result);
+      // update local data members
+      NodeAdditionResult result = partitionTable.getNodeAdditionResult(newNode);
+      getDataClusterServer().addNode(newNode, result);
+      if (logger.isDebugEnabled()) {
+        logger.debug("{}: success to add a new node {} into {}", name, newNode, allNodes);
       }
     }
+    logger.info(
+        "{}: execute adding node {} cost {} ms",
+        name,
+        newNode,
+        (System.currentTimeMillis()) - startTime);
   }
 
   /**
@@ -439,6 +456,7 @@ public class MetaGroupMember extends RaftMember {
         partitionTable = new SlotPartitionTable(allNodes, thisNode);
         logger.info("Partition table is set up");
       }
+      initIdNodeMap();
       router = new ClusterPlanRouter(partitionTable);
       this.coordinator.setRouter(router);
       startSubServers();
@@ -470,8 +488,6 @@ public class MetaGroupMember extends RaftMember {
   /**
    * This node is not a seed node and wants to join an established cluster. Pick up a node randomly
    * from the seed nodes and send a join request to it.
-   *
-   * @return true if the node has successfully joined the cluster, false otherwise.
    */
   public void joinCluster() throws ConfigInconsistentException, StartUpCheckFailureException {
     if (allNodes.size() == 1) {
@@ -519,6 +535,8 @@ public class MetaGroupMember extends RaftMember {
     newStartUpStatus.setReplicationNumber(
         ClusterDescriptor.getInstance().getConfig().getReplicationNum());
     newStartUpStatus.setClusterName(ClusterDescriptor.getInstance().getConfig().getClusterName());
+    newStartUpStatus.setMultiRaftFactor(
+        ClusterDescriptor.getInstance().getConfig().getMultiRaftFactor());
     List<String> seedUrls = ClusterDescriptor.getInstance().getConfig().getSeedNodeUrls();
     List<Node> seedNodeList = new ArrayList<>();
     for (String seedUrl : seedUrls) {
@@ -532,7 +550,7 @@ public class MetaGroupMember extends RaftMember {
    * Send a join cluster request to "node". If the joining is accepted, set the partition table,
    * start DataClusterServer and ClientServer and initialize DataGroupMembers.
    *
-   * @return rue if the node has successfully joined the cluster, false otherwise.
+   * @return true if the node has successfully joined the cluster, false otherwise.
    */
   private boolean joinCluster(Node node, StartUpStatus startUpStatus)
       throws TException, InterruptedException, ConfigInconsistentException {
@@ -574,6 +592,9 @@ public class MetaGroupMember extends RaftMember {
       setNodeIdentifier(genNodeIdentifier());
     } else if (resp.getRespNum() == Response.RESPONSE_NEW_NODE_PARAMETER_CONFLICT) {
       handleConfigInconsistency(resp);
+    } else if (resp.getRespNum() == Response.RESPONSE_DATA_MIGRATION_NOT_FINISH) {
+      logger.warn(
+          "The data migration of the previous membership change operation is not finished. Please try again later");
     } else {
       logger.warn("Joining the cluster is rejected by {} for response {}", node, resp.getRespNum());
     }
@@ -581,21 +602,35 @@ public class MetaGroupMember extends RaftMember {
   }
 
   private void handleConfigInconsistency(AddNodeResponse resp) throws ConfigInconsistentException {
-    if (logger.isErrorEnabled()) {
-      CheckStatusResponse checkStatusResponse = resp.getCheckStatusResponse();
-      String parameters =
-          (checkStatusResponse.isPartitionalIntervalEquals() ? "" : ", partition interval")
-              + (checkStatusResponse.isHashSaltEquals() ? "" : ", hash salt")
-              + (checkStatusResponse.isReplicationNumEquals() ? "" : ", replication number")
-              + (checkStatusResponse.isSeedNodeEquals() ? "" : ", seedNodes")
-              + (checkStatusResponse.isClusterNameEquals() ? "" : ", clusterName");
-      logger.error(
-          "The start up configuration{} conflicts the cluster. Please reset the configurations. ",
-          parameters.substring(1));
-    }
+    CheckStatusResponse checkStatusResponse = resp.getCheckStatusResponse();
+    String parameters =
+        (checkStatusResponse.isPartitionalIntervalEquals() ? "" : ", partition interval")
+            + (checkStatusResponse.isHashSaltEquals() ? "" : ", hash salt")
+            + (checkStatusResponse.isReplicationNumEquals() ? "" : ", replication number")
+            + (checkStatusResponse.isSeedNodeEquals() ? "" : ", seedNodes")
+            + (checkStatusResponse.isClusterNameEquals() ? "" : ", clusterName")
+            + (checkStatusResponse.isMultiRaftFactorEquals() ? "" : ", multiRaftFactor");
+    logger.error(
+        "The start up configuration{} conflicts the cluster. Please reset the configurations. ",
+        parameters.substring(1));
     throw new ConfigInconsistentException();
   }
 
+  @Override
+  long checkElectorLogProgress(ElectionRequest electionRequest) {
+    Node elector = electionRequest.getElector();
+    // check if the node is in the group
+    if (partitionTable != null && !allNodes.contains(elector)) {
+      logger.info(
+          "{}: the elector {} is not in the data group {}, so reject this election.",
+          name,
+          getPartitionGroup(),
+          elector);
+      return Response.RESPONSE_NODE_IS_NOT_IN_GROUP;
+    }
+    return super.checkElectorLogProgress(electionRequest);
+  }
+
   /**
    * Process the heartbeat request from a valid leader. Generate and tell the leader the identifier
    * of the node if necessary. If the partition table is missing, use the one from the request or
@@ -641,8 +676,8 @@ public class MetaGroupMember extends RaftMember {
     newTable.deserialize(partitionTableBuffer);
     // avoid overwriting current partition table with a previous one
     if (partitionTable != null) {
-      long currIndex = ((SlotPartitionTable) partitionTable).getLastLogIndex();
-      long incomingIndex = newTable.getLastLogIndex();
+      long currIndex = partitionTable.getLastMetaLogIndex();
+      long incomingIndex = newTable.getLastMetaLogIndex();
       logger.info(
           "Current partition table index {}, new partition table index {}",
           currIndex,
@@ -667,7 +702,7 @@ public class MetaGroupMember extends RaftMember {
   }
 
   private void updateNodeList(Collection<Node> nodes) {
-    allNodes = new ArrayList<>(nodes);
+    allNodes = new PartitionGroup(nodes);
     initPeerMap();
     logger.info("All nodes in the partition table: {}", allNodes);
     initIdNodeMap();
@@ -696,7 +731,7 @@ public class MetaGroupMember extends RaftMember {
         // leader through the first heartbeat. After the leader knows the node information of all
         // nodes, it can replace the incomplete node information previously saved locally, and build
         // partitionTable to send it to other followers.
-        allNodes = new ArrayList<>(idNodeMap.values());
+        allNodes = new PartitionGroup(idNodeMap.values());
         if (partitionTable == null) {
           partitionTable = new SlotPartitionTable(allNodes, thisNode);
           logger.info("Partition table is set up");
@@ -814,7 +849,8 @@ public class MetaGroupMember extends RaftMember {
    * @param node cannot be the local node
    */
   public AddNodeResponse addNode(Node node, StartUpStatus startUpStatus)
-      throws AddSelfException, LogExecutionException {
+      throws AddSelfException, LogExecutionException, InterruptedException,
+          CheckConsistencyException {
     AddNodeResponse response = new AddNodeResponse();
     if (partitionTable == null) {
       logger.info("Cannot add node now because the partition table is not set");
@@ -841,19 +877,34 @@ public class MetaGroupMember extends RaftMember {
    * immediately. If the identifier of "node" conflicts with an existing node, the request will be
    * turned down.
    *
-   * @param node cannot be the local node
+   * @param newNode cannot be the local node
    * @param startUpStatus the start up status of the new node
    * @param response the response that will be sent to "node"
    * @return true if the process is over, false if the request should be forwarded
    */
   private boolean processAddNodeLocally(
-      Node node, StartUpStatus startUpStatus, AddNodeResponse response)
-      throws LogExecutionException {
+      Node newNode, StartUpStatus startUpStatus, AddNodeResponse response)
+      throws LogExecutionException, InterruptedException, CheckConsistencyException {
     if (character != NodeCharacter.LEADER) {
       return false;
     }
-    if (allNodes.contains(node)) {
-      logger.debug("Node {} is already in the cluster", node);
+
+    if (!waitDataMigrationEnd()) {
+      response.setRespNum((int) Response.RESPONSE_DATA_MIGRATION_NOT_FINISH);
+      return true;
+    }
+
+    for (Node node : partitionTable.getAllNodes()) {
+      if (node.internalIp.equals(newNode.internalIp)
+          && newNode.dataPort == node.dataPort
+          && newNode.metaPort == node.metaPort
+          && newNode.clientPort == node.clientPort) {
+        newNode.nodeIdentifier = node.nodeIdentifier;
+        break;
+      }
+    }
+    if (allNodes.contains(newNode)) {
+      logger.debug("Node {} is already in the cluster", newNode);
       response.setRespNum((int) Response.RESPONSE_AGREE);
       synchronized (partitionTable) {
         response.setPartitionTableBytes(partitionTable.serialize());
@@ -861,9 +912,9 @@ public class MetaGroupMember extends RaftMember {
       return true;
     }
 
-    Node idConflictNode = idNodeMap.get(node.getNodeIdentifier());
+    Node idConflictNode = idNodeMap.get(newNode.getNodeIdentifier());
     if (idConflictNode != null) {
-      logger.debug("{}'s id conflicts with {}", node, idConflictNode);
+      logger.debug("{}'s id conflicts with {}", newNode, idConflictNode);
       response.setRespNum((int) Response.RESPONSE_IDENTIFIER_CONFLICT);
       return true;
     }
@@ -873,39 +924,112 @@ public class MetaGroupMember extends RaftMember {
       return true;
     }
 
+    AddNodeLog addNodeLog = new AddNodeLog();
     // node adding is serialized to reduce potential concurrency problem
     synchronized (logManager) {
-      AddNodeLog addNodeLog = new AddNodeLog();
+      // update partition table
+      PartitionTable table = new SlotPartitionTable(thisNode);
+      table.deserialize(partitionTable.serialize());
+      table.addNode(newNode);
+      table.setLastMetaLogIndex(logManager.getLastLogIndex() + 1);
+
+      addNodeLog.setPartitionTable(table.serialize());
       addNodeLog.setCurrLogTerm(getTerm().get());
       addNodeLog.setCurrLogIndex(logManager.getLastLogIndex() + 1);
+      addNodeLog.setMetaLogIndex(logManager.getLastLogIndex() + 1);
 
-      addNodeLog.setNewNode(node);
+      addNodeLog.setNewNode(newNode);
 
       logManager.append(addNodeLog);
+    }
 
-      int retryTime = 1;
-      while (true) {
-        logger.info("Send the join request of {} to other nodes, retry time: {}", node, retryTime);
-        AppendLogResult result = sendLogToAllGroups(addNodeLog);
-        switch (result) {
-          case OK:
-            logger.info("Join request of {} is accepted", node);
-            commitLog(addNodeLog);
-
-            synchronized (partitionTable) {
-              response.setPartitionTableBytes(partitionTable.serialize());
-            }
-            response.setRespNum((int) Response.RESPONSE_AGREE);
-            logger.info("Sending join response of {}", node);
-            return true;
-          case TIME_OUT:
-            logger.info("Join request of {} timed out", node);
-            retryTime++;
-            continue;
-          case LEADERSHIP_STALE:
-          default:
-            return false;
-        }
+    int retryTime = 0;
+    while (true) {
+      logger.info(
+          "{}: Send the join request of {} to other nodes, retry time: {}",
+          name,
+          newNode,
+          retryTime);
+      AppendLogResult result = sendLogToFollowers(addNodeLog);
+      switch (result) {
+        case OK:
+          commitLog(addNodeLog);
+          logger.info("{}: Join request of {} is accepted", name, newNode);
+
+          synchronized (partitionTable) {
+            response.setPartitionTableBytes(partitionTable.serialize());
+          }
+          response.setRespNum((int) Response.RESPONSE_AGREE);
+          logger.info("{}: Sending join response of {}", name, newNode);
+          return true;
+        case TIME_OUT:
+          logger.debug("{}: log {} timed out, retrying...", name, addNodeLog);
+          try {
+            Thread.sleep(ClusterConstant.RETRY_WAIT_TIME_MS);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+          }
+          logger.info("{}: Join request of {} timed out", name, newNode);
+          retryTime++;
+          break;
+        case LEADERSHIP_STALE:
+        default:
+          return false;
+      }
+    }
+  }
+
+  /** Check if there has data migration due to previous change membership operation. */
+  private boolean waitDataMigrationEnd() throws InterruptedException, CheckConsistencyException {
+    // try 5 time
+    int retryTime = 0;
+    while (true) {
+      Map<PartitionGroup, Integer> res = collectAllPartitionMigrationStatus();
+      if (res != null && res.isEmpty()) {
+        return true;
+      }
+      if (++retryTime == 5) {
+        break;
+      }
+      Thread.sleep(ClusterConstant.RETRY_WAIT_TIME_MS);
+    }
+    return false;
+  }
+
+  /** Process empty log for leader to commit all previous log. */
+  public void processEmptyContentLog() {
+    Log log = new EmptyContentLog();
+
+    synchronized (logManager) {
+      log.setCurrLogTerm(getTerm().get());
+      log.setCurrLogIndex(logManager.getLastLogIndex() + 1);
+      logManager.append(log);
+    }
+
+    int retryTime = 0;
+    while (true) {
+      logger.debug("{} Send empty content log to other nodes, retry time: {}", name, retryTime);
+      AppendLogResult result = sendLogToFollowers(log);
+      switch (result) {
+        case OK:
+          try {
+            commitLog(log);
+          } catch (LogExecutionException e) {
+            logger.error("{}: Fail to execute empty content log", name, e);
+          }
+          return;
+        case TIME_OUT:
+          logger.debug("{}: add empty content log timed out, retry.", name);
+          try {
+            Thread.sleep(ClusterConstant.RETRY_WAIT_TIME_MS);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+          }
+          retryTime++;
+          break;
+        case LEADERSHIP_STALE:
+        default:
+          return;
       }
     }
   }
@@ -914,13 +1038,16 @@ public class MetaGroupMember extends RaftMember {
     long remotePartitionInterval = remoteStartUpStatus.getPartitionInterval();
     int remoteHashSalt = remoteStartUpStatus.getHashSalt();
     int remoteReplicationNum = remoteStartUpStatus.getReplicationNumber();
+    int remoteMultiRaftFactor = remoteStartUpStatus.getMultiRaftFactor();
     String remoteClusterName = remoteStartUpStatus.getClusterName();
     List<Node> remoteSeedNodeList = remoteStartUpStatus.getSeedNodeList();
     long localPartitionInterval = IoTDBDescriptor.getInstance().getConfig().getPartitionInterval();
     int localHashSalt = ClusterConstant.HASH_SALT;
-    int localReplicationNum = ClusterDescriptor.getInstance().getConfig().getReplicationNum();
-    String localClusterName = ClusterDescriptor.getInstance().getConfig().getClusterName();
+    int localReplicationNum = config.getReplicationNum();
+    String localClusterName = config.getClusterName();
+    int localMultiRaftFactor = config.getMultiRaftFactor();
     boolean partitionIntervalEquals = true;
+    boolean multiRaftFactorEquals = true;
     boolean hashSaltEquals = true;
     boolean replicationNumEquals = true;
     boolean seedNodeEquals = true;
@@ -933,6 +1060,13 @@ public class MetaGroupMember extends RaftMember {
           localPartitionInterval,
           remotePartitionInterval);
     }
+    if (localMultiRaftFactor != remoteMultiRaftFactor) {
+      multiRaftFactorEquals = false;
+      logger.info(
+          "Remote multi-raft factor conflicts with the leader's. Leader: {}, remote: {}",
+          localMultiRaftFactor,
+          remoteMultiRaftFactor);
+    }
     if (localHashSalt != remoteHashSalt) {
       hashSaltEquals = false;
       logger.info(
@@ -967,7 +1101,8 @@ public class MetaGroupMember extends RaftMember {
         && hashSaltEquals
         && replicationNumEquals
         && seedNodeEquals
-        && clusterNameEquals)) {
+        && clusterNameEquals
+        && multiRaftFactorEquals)) {
       response.setRespNum((int) Response.RESPONSE_NEW_NODE_PARAMETER_CONFLICT);
       response.setCheckStatusResponse(
           new CheckStatusResponse(
@@ -975,7 +1110,8 @@ public class MetaGroupMember extends RaftMember {
               hashSaltEquals,
               replicationNumEquals,
               seedNodeEquals,
-              clusterNameEquals));
+              clusterNameEquals,
+              multiRaftFactorEquals));
       return false;
     }
     return true;
@@ -1066,8 +1202,8 @@ public class MetaGroupMember extends RaftMember {
   }
 
   private CheckStatusResponse checkStatus(Node seedNode) {
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-      AsyncMetaClient client = (AsyncMetaClient) getAsyncClient(seedNode, false);
+    if (config.isUseAsyncServer()) {
+      AsyncMetaClient client = (AsyncMetaClient) getAsyncClient(seedNode);
       if (client == null) {
         return null;
       }
@@ -1096,129 +1232,6 @@ public class MetaGroupMember extends RaftMember {
     return null;
   }
 
-  /**
-   * Send the log the all data groups and return a success only when each group's quorum has
-   * accepted this log.
-   */
-  private AppendLogResult sendLogToAllGroups(Log log) {
-    List<Node> nodeRing = partitionTable.getAllNodes();
-
-    AtomicLong newLeaderTerm = new AtomicLong(term.get());
-    AtomicBoolean leaderShipStale = new AtomicBoolean(false);
-    AppendEntryRequest request = buildAppendEntryRequest(log, true);
-
-    // ask for votes from each node
-    int[] groupRemainings = askGroupVotes(nodeRing, request, leaderShipStale, log, newLeaderTerm);
-
-    if (!leaderShipStale.get()) {
-      // if all quorums of all groups have received this log, it is considered succeeded.
-      for (int remaining : groupRemainings) {
-        if (remaining > 0) {
-          return AppendLogResult.TIME_OUT;
-        }
-      }
-    } else {
-      return AppendLogResult.LEADERSHIP_STALE;
-    }
-
-    return AppendLogResult.OK;
-  }
-
-  /**
-   * Send "request" to each node in "nodeRing" and when a node returns a success, decrease all
-   * counters of the groups it is in of "groupRemainings"
-   *
-   * @return a int array indicating how many votes are left in each group to make an agreement
-   */
-  @SuppressWarnings({"java:S2445", "java:S2274"})
-  // groupRemaining is shared with the handlers,
-  // and we do not wait infinitely to enable timeouts
-  private int[] askGroupVotes(
-      List<Node> nodeRing,
-      AppendEntryRequest request,
-      AtomicBoolean leaderShipStale,
-      Log log,
-      AtomicLong newLeaderTerm) {
-    // each node will be the header of a group, we use the node to represent the group
-    int nodeSize = nodeRing.size();
-    // the decreasing counters of how many nodes in a group has received the log, each time a
-    // node receive the log, the counters of all groups it is in will decrease by 1
-    int[] groupRemainings = new int[nodeSize];
-    // a group is considered successfully received the log if such members receive the log
-    int groupQuorum = REPLICATION_NUM / 2 + 1;
-    Arrays.fill(groupRemainings, groupQuorum);
-
-    synchronized (groupRemainings) {
-      // ask a vote from every node
-      for (int i = 0; i < nodeSize; i++) {
-        Node node = nodeRing.get(i);
-        if (node.equals(thisNode)) {
-          // this node automatically gives an agreement, decrease counters of all groups the local
-          // node is in
-          for (int j = 0; j < REPLICATION_NUM; j++) {
-            int groupIndex = i - j;
-            if (groupIndex < 0) {
-              groupIndex += groupRemainings.length;
-            }
-            groupRemainings[groupIndex]--;
-          }
-        } else {
-          askRemoteGroupVote(
-              node, groupRemainings, i, leaderShipStale, log, newLeaderTerm, request);
-        }
-      }
-
-      try {
-        groupRemainings.wait(RaftServer.getWriteOperationTimeoutMS());
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        logger.error("Unexpected interruption when waiting for the group votes", e);
-      }
-    }
-    return groupRemainings;
-  }
-
-  private void askRemoteGroupVote(
-      Node node,
-      int[] groupRemainings,
-      int nodeIndex,
-      AtomicBoolean leaderShipStale,
-      Log log,
-      AtomicLong newLeaderTerm,
-      AppendEntryRequest request) {
-    AppendGroupEntryHandler handler =
-        new AppendGroupEntryHandler(
-            groupRemainings, nodeIndex, node, leaderShipStale, log, newLeaderTerm, this);
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-      AsyncMetaClient client = (AsyncMetaClient) getAsyncClient(node);
-      try {
-        if (client != null) {
-          client.appendEntry(request, handler);
-        }
-      } catch (TException e) {
-        logger.error("Cannot send log to node {}", node, e);
-      }
-    } else {
-      SyncMetaClient client = (SyncMetaClient) getSyncClient(node);
-      if (client == null) {
-        logger.error("No available client for {}", node);
-        return;
-      }
-      getSerialToParallelPool()
-          .submit(
-              () -> {
-                try {
-                  handler.onComplete(client.appendEntry(request));
-                } catch (TException e) {
-                  client.getInputProtocol().getTransport().close();
-                  handler.onError(e);
-                } finally {
-                  ClientUtils.putBackSyncClient(client);
-                }
-              });
-    }
-  }
-
   public Set<Node> getIdConflictNodes() {
     return idConflictNodes;
   }
@@ -1363,7 +1376,7 @@ public class MetaGroupMember extends RaftMember {
   public void receiveSnapshot(SendSnapshotRequest request) throws SnapshotInstallationException {
     MetaSimpleSnapshot snapshot = new MetaSimpleSnapshot();
     snapshot.deserialize(request.snapshotBytes);
-    snapshot.getDefaultInstaller(this).install(snapshot, -1);
+    snapshot.getDefaultInstaller(this).install(snapshot, -1, false);
   }
 
   /**
@@ -1405,8 +1418,9 @@ public class MetaGroupMember extends RaftMember {
     } else if (!ClusterConstant.EMPTY_NODE.equals(leader.get())) {
       TSStatus result = forwardPlan(plan, leader.get(), null);
       if (!StatusUtils.NO_LEADER.equals(result)) {
-        result.setRedirectNode(
-            new EndPoint(leader.get().getClientIp(), leader.get().getClientPort()));
+        result =
+            StatusUtils.getStatus(
+                result, new EndPoint(leader.get().getInternalIp(), leader.get().getClientPort()));
         return result;
       }
     }
@@ -1428,6 +1442,32 @@ public class MetaGroupMember extends RaftMember {
   }
 
   /**
+   * Forward a non-query plan to the data port of "receiver"
+   *
+   * @param plan a non-query plan
+   * @param header to determine which DataGroupMember of "receiver" will process the request.
+   * @return a TSStatus indicating if the forwarding is successful.
+   */
+  private TSStatus forwardDataPlanAsync(PhysicalPlan plan, Node receiver, RaftNode header)
+      throws IOException {
+    RaftService.AsyncClient client =
+        getClientProvider().getAsyncDataClient(receiver, RaftServer.getWriteOperationTimeoutMS());
+    return forwardPlanAsync(plan, receiver, header, client);
+  }
+
+  private TSStatus forwardDataPlanSync(PhysicalPlan plan, Node receiver, RaftNode header)
+      throws IOException {
+    Client client;
+    try {
+      client =
+          getClientProvider().getSyncDataClient(receiver, RaftServer.getWriteOperationTimeoutMS());
+    } catch (TException e) {
+      throw new IOException(e);
+    }
+    return forwardPlanSync(plan, receiver, header, client);
+  }
+
+  /**
    * Get the data groups that should be queried when querying "path" with "filter". First, the time
    * interval qualified by the filter will be extracted. If any side of the interval is open, query
    * all groups. Otherwise compute all involved groups w.r.t. the time partitioning.
@@ -1452,7 +1492,7 @@ public class MetaGroupMember extends RaftMember {
   public List<PartitionGroup> routeIntervals(Intervals intervals, PartialPath path)
       throws StorageEngineException {
     List<PartitionGroup> partitionGroups = new ArrayList<>();
-    PartialPath storageGroupName = null;
+    PartialPath storageGroupName;
     try {
       storageGroupName = IoTDB.metaManager.getStorageGroupPath(path);
     } catch (MetadataException e) {
@@ -1476,7 +1516,7 @@ public class MetaGroupMember extends RaftMember {
     } else {
       // compute the related data groups of all intervals
       // TODO-Cluster#690: change to a broadcast when the computation is too expensive
-      Set<Node> groupHeaders = new HashSet<>();
+      Set<RaftNode> groupHeaders = new HashSet<>();
       for (int i = 0; i < intervals.getIntervalSize(); i++) {
         // compute the headers of groups involved in every interval
         PartitionUtils.getIntervalHeaders(
@@ -1487,7 +1527,7 @@ public class MetaGroupMember extends RaftMember {
             groupHeaders);
       }
       // translate the headers to groups
-      for (Node groupHeader : groupHeaders) {
+      for (RaftNode groupHeader : groupHeaders) {
         partitionGroups.add(partitionTable.getHeaderGroup(groupHeader));
       }
     }
@@ -1495,18 +1535,18 @@ public class MetaGroupMember extends RaftMember {
   }
 
   @SuppressWarnings("java:S2274")
-  public Map<Node, Boolean> getAllNodeStatus() {
+  public Map<Node, Integer> getAllNodeStatus() {
     if (getPartitionTable() == null) {
       // the cluster is being built.
       return null;
     }
-    Map<Node, Boolean> nodeStatus = new HashMap<>();
+    Map<Node, Integer> nodeStatus = new HashMap<>();
     for (Node node : allNodes) {
-      nodeStatus.put(node, thisNode.equals(node));
+      nodeStatus.put(node, thisNode.equals(node) ? Status.LIVE : Status.OFFLINE);
     }
 
     try {
-      if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      if (config.isUseAsyncServer()) {
         getNodeStatusAsync(nodeStatus);
       } else {
         getNodeStatusSync(nodeStatus);
@@ -1517,11 +1557,20 @@ public class MetaGroupMember extends RaftMember {
       Thread.currentThread().interrupt();
       logger.warn("Cannot get the status of all nodes", e);
     }
+
+    for (Node node : partitionTable.getAllNodes()) {
+      nodeStatus.putIfAbsent(node, Status.JOINING);
+    }
+    for (Node node : allNodes) {
+      if (!partitionTable.getAllNodes().contains(node)) {
+        nodeStatus.put(node, Status.LEAVING);
+      }
+    }
     return nodeStatus;
   }
 
   @SuppressWarnings({"java:S2445", "java:S2274"})
-  private void getNodeStatusAsync(Map<Node, Boolean> nodeStatus)
+  private void getNodeStatusAsync(Map<Node, Integer> nodeStatus)
       throws TException, InterruptedException {
     NodeStatusHandler nodeStatusHandler = new NodeStatusHandler(nodeStatus);
     synchronized (nodeStatus) {
@@ -1535,7 +1584,7 @@ public class MetaGroupMember extends RaftMember {
     }
   }
 
-  private void getNodeStatusSync(Map<Node, Boolean> nodeStatus) {
+  private void getNodeStatusSync(Map<Node, Integer> nodeStatus) {
     NodeStatusHandler nodeStatusHandler = new NodeStatusHandler(nodeStatus);
     for (Node node : allNodes) {
       SyncMetaClient client = (SyncMetaClient) getSyncClient(node);
@@ -1553,6 +1602,47 @@ public class MetaGroupMember extends RaftMember {
     }
   }
 
+  public Map<PartitionGroup, Integer> collectMigrationStatus(Node node) {
+    try {
+      if (config.isUseAsyncServer()) {
+        return collectMigrationStatusAsync(node);
+      } else {
+        return collectMigrationStatusSync(node);
+      }
+    } catch (TException e) {
+      logger.error("{}: Cannot get the status of node {}", name, node, e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      logger.error("{}: Cannot get the status of node {}", name, node, e);
+    }
+    return null;
+  }
+
+  private Map<PartitionGroup, Integer> collectMigrationStatusAsync(Node node)
+      throws TException, InterruptedException {
+    AtomicReference<ByteBuffer> resultRef = new AtomicReference<>();
+    GenericHandler<ByteBuffer> migrationStatusHandler = new GenericHandler<>(node, resultRef);
+    AsyncMetaClient client = (AsyncMetaClient) getAsyncClient(node);
+    if (client == null) {
+      return null;
+    }
+    client.collectMigrationStatus(migrationStatusHandler);
+    synchronized (resultRef) {
+      if (resultRef.get() == null) {
+        resultRef.wait(RaftServer.getConnectionTimeoutInMS());
+      }
+    }
+    return ClusterUtils.deserializeMigrationStatus(resultRef.get());
+  }
+
+  private Map<PartitionGroup, Integer> collectMigrationStatusSync(Node node) throws TException {
+    SyncMetaClient client = (SyncMetaClient) getSyncClient(node);
+    if (client == null) {
+      return null;
+    }
+    return ClusterUtils.deserializeMigrationStatus(client.collectMigrationStatus());
+  }
+
   @TestOnly
   public void setPartitionTable(PartitionTable partitionTable) {
     this.partitionTable = partitionTable;
@@ -1573,7 +1663,8 @@ public class MetaGroupMember extends RaftMember {
    * @param node the node to be removed.
    */
   public long removeNode(Node node)
-      throws PartitionTableUnavailableException, LogExecutionException {
+      throws PartitionTableUnavailableException, LogExecutionException, InterruptedException,
+          CheckConsistencyException {
     if (partitionTable == null) {
       logger.info("Cannot add node now because the partition table is not set");
       throw new PartitionTableUnavailableException(thisNode);
@@ -1592,16 +1683,21 @@ public class MetaGroupMember extends RaftMember {
    * @param node the node to be removed.
    * @return Long.MIN_VALUE if further forwarding is required, or the execution result
    */
-  private long processRemoveNodeLocally(Node node) throws LogExecutionException {
+  private long processRemoveNodeLocally(Node node)
+      throws LogExecutionException, InterruptedException, CheckConsistencyException {
     if (character != NodeCharacter.LEADER) {
       return Response.RESPONSE_NULL;
     }
 
     // if we cannot have enough replica after the removal, reject it
-    if (allNodes.size() <= ClusterDescriptor.getInstance().getConfig().getReplicationNum()) {
+    if (allNodes.size() <= config.getReplicationNum()) {
       return Response.RESPONSE_CLUSTER_TOO_SMALL;
     }
 
+    if (!waitDataMigrationEnd()) {
+      return Response.RESPONSE_DATA_MIGRATION_NOT_FINISH;
+    }
+
     // find the node to be removed in the node list
     Node target = null;
     synchronized (allNodes) {
@@ -1618,37 +1714,50 @@ public class MetaGroupMember extends RaftMember {
       return Response.RESPONSE_REJECT;
     }
 
+    RemoveNodeLog removeNodeLog = new RemoveNodeLog();
     // node removal must be serialized to reduce potential concurrency problem
     synchronized (logManager) {
-      RemoveNodeLog removeNodeLog = new RemoveNodeLog();
+      // update partition table
+      PartitionTable table = new SlotPartitionTable((SlotPartitionTable) partitionTable);
+      table.removeNode(target);
+      table.setLastMetaLogIndex(logManager.getLastLogIndex() + 1);
+
+      removeNodeLog.setPartitionTable(table.serialize());
       removeNodeLog.setCurrLogTerm(getTerm().get());
       removeNodeLog.setCurrLogIndex(logManager.getLastLogIndex() + 1);
+      removeNodeLog.setMetaLogIndex(logManager.getLastLogIndex() + 1);
 
       removeNodeLog.setRemovedNode(target);
 
       logManager.append(removeNodeLog);
+    }
 
-      int retryTime = 1;
-      while (true) {
-        logger.info(
-            "Send the node removal request of {} to other nodes, retry time: {}",
-            target,
-            retryTime);
-        AppendLogResult result = sendLogToAllGroups(removeNodeLog);
-
-        switch (result) {
-          case OK:
-            logger.info("Removal request of {} is accepted", target);
-            commitLog(removeNodeLog);
-            return Response.RESPONSE_AGREE;
-          case TIME_OUT:
-            logger.info("Removal request of {} timed out", target);
-            break;
-            // retry
-          case LEADERSHIP_STALE:
-          default:
-            return Response.RESPONSE_NULL;
-        }
+    int retryTime = 0;
+    while (true) {
+      logger.info(
+          "{}: Send the node removal request of {} to other nodes, retry time: {}",
+          name,
+          target,
+          retryTime);
+      AppendLogResult result = sendLogToFollowers(removeNodeLog);
+      switch (result) {
+        case OK:
+          commitLog(removeNodeLog);
+          logger.info("{}: Removal request of {} is accepted", name, target);
+          return Response.RESPONSE_AGREE;
+        case TIME_OUT:
+          logger.info("{}: Removal request of {} timed out", name, target);
+          try {
+            Thread.sleep(ClusterConstant.RETRY_WAIT_TIME_MS);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+          }
+          retryTime++;
+          break;
+          // retry
+        case LEADERSHIP_STALE:
+        default:
+          return Response.RESPONSE_NULL;
       }
     }
   }
@@ -1658,54 +1767,84 @@ public class MetaGroupMember extends RaftMember {
    * node is the local node, also stop heartbeat and catch-up service of metadata, but the heartbeat
    * and catch-up service of data are kept alive for other nodes to pull data. If the removed node
    * is a leader, send an exile to the removed node so that it can know it is removed.
-   *
-   * @param oldNode the node to be removed
    */
-  public void applyRemoveNode(Node oldNode) {
+  public void applyRemoveNode(RemoveNodeLog removeNodeLog) {
+
+    long startTime = System.currentTimeMillis();
+    Node oldNode = removeNodeLog.getRemovedNode();
     synchronized (allNodes) {
+      if (logger.isDebugEnabled()) {
+        logger.debug("{}: Removing a node {} from {}", name, oldNode, allNodes);
+      }
+
       if (allNodes.contains(oldNode)) {
-        logger.debug("Removing a node {} from {}", oldNode, allNodes);
         allNodes.remove(oldNode);
         idNodeMap.remove(oldNode.nodeIdentifier);
+      }
+
+      // save the updated partition table
+      savePartitionTable();
 
-        // update the partition table
-        NodeRemovalResult result = partitionTable.removeNode(oldNode);
-        ((SlotPartitionTable) partitionTable).setLastLogIndex(logManager.getLastLogIndex());
+      // update DataGroupMembers, as the node is removed, the members of some groups are
+      // changed and there will also be one less group
+      NodeRemovalResult result = partitionTable.getNodeRemovalResult();
+      getDataClusterServer().removeNode(oldNode, result);
 
-        // update DataGroupMembers, as the node is removed, the members of some groups are
-        // changed and there will also be one less group
-        getDataClusterServer().removeNode(oldNode, result);
-        // the leader is removed, start the next election ASAP
-        if (oldNode.equals(leader.get())) {
+      // the leader is removed, start the next election ASAP
+      if (oldNode.equals(leader.get()) && !oldNode.equals(thisNode)) {
+        synchronized (term) {
           setCharacter(NodeCharacter.ELECTOR);
-          lastHeartbeatReceivedTime = Long.MIN_VALUE;
+          setLeader(null);
         }
-
-        if (oldNode.equals(thisNode)) {
-          // use super.stop() so that the data server will not be closed because other nodes may
-          // want to pull data from this node
-          super.stop();
-          if (clientServer != null) {
-            clientServer.stop();
-          }
-        } else if (thisNode.equals(leader.get())) {
-          // as the old node is removed, it cannot know this by heartbeat or log, so it should be
-          // directly kicked out of the cluster
-          exileNode(oldNode);
+        synchronized (getHeartBeatWaitObject()) {
+          getHeartBeatWaitObject().notifyAll();
         }
+      }
 
-        // save the updated partition table
-        savePartitionTable();
+      if (oldNode.equals(thisNode)) {
+        // use super.stop() so that the data server will not be closed because other nodes may
+        // want to pull data from this node
+        new Thread(
+                () -> {
+                  try {
+                    Thread.sleep(RaftServer.getHeartBeatIntervalMs());
+                  } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    // ignore
+                  }
+                  super.stop();
+                  if (clientServer != null) {
+                    clientServer.stop();
+                  }
+                  logger.info("{} has been removed from the cluster", name);
+                })
+            .start();
+      } else if (thisNode.equals(leader.get())) {
+        // as the old node is removed, it cannot know this by heartbeat or log, so it should be
+        // directly kicked out of the cluster
+        getAppendLogThreadPool().submit(() -> exileNode(removeNodeLog));
+      }
+
+      if (logger.isDebugEnabled()) {
+        logger.debug("{}: Success to remove a node {} from {}", name, oldNode, allNodes);
       }
+
+      logger.info(
+          "{}: execute removing node {} cost {} ms",
+          name,
+          oldNode,
+          (System.currentTimeMillis()) - startTime);
     }
   }
 
-  private void exileNode(Node node) {
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+  protected void exileNode(RemoveNodeLog removeNodeLog) {
+    logger.debug("Exile node {}: start.", removeNodeLog.getRemovedNode());
+    Node node = removeNodeLog.getRemovedNode();
+    if (config.isUseAsyncServer()) {
       AsyncMetaClient asyncMetaClient = (AsyncMetaClient) getAsyncClient(node);
       try {
         if (asyncMetaClient != null) {
-          asyncMetaClient.exile(new GenericHandler<>(node, null));
+          asyncMetaClient.exile(removeNodeLog.serialize(), new GenericHandler<>(node, null));
         }
       } catch (TException e) {
         logger.warn("Cannot inform {} its removal", node, e);
@@ -1716,7 +1855,7 @@ public class MetaGroupMember extends RaftMember {
         return;
       }
       try {
-        client.exile();
+        client.exile(removeNodeLog.serialize());
       } catch (TException e) {
         client.getInputProtocol().getTransport().close();
         logger.warn("Cannot inform {} its removal", node, e);
@@ -1758,9 +1897,62 @@ public class MetaGroupMember extends RaftMember {
     return report;
   }
 
+  /**
+   * Collect data migration status of data group in all cluster nodes.
+   *
+   * @return key: data group; value: slot num in data migration
+   */
+  public Map<PartitionGroup, Integer> collectAllPartitionMigrationStatus()
+      throws CheckConsistencyException {
+    syncLeader(null);
+    Map<PartitionGroup, Integer> res = new HashMap<>();
+    for (Node node : allNodes) {
+      if (logger.isDebugEnabled()) {
+        logger.debug("{}: start to get migration status of {}", name, node);
+      }
+      Map<PartitionGroup, Integer> oneNodeRes;
+      if (node.equals(thisNode)) {
+        oneNodeRes = collectMigrationStatus();
+      } else {
+        oneNodeRes = collectMigrationStatus(node);
+      }
+      if (oneNodeRes == null) {
+        return null;
+      }
+      for (Entry<PartitionGroup, Integer> entry : oneNodeRes.entrySet()) {
+        res.put(entry.getKey(), Math.max(res.getOrDefault(entry.getKey(), 0), entry.getValue()));
+      }
+    }
+    return res;
+  }
+
+  /**
+   * Collect data migration status of data group in all cluster nodes.
+   *
+   * @return key: data group; value: slot num in data migration
+   */
+  public Map<PartitionGroup, Integer> collectMigrationStatus() {
+    logger.info("{}: start to collect migration status locally.", name);
+    Map<PartitionGroup, Integer> groupSlotMap = new HashMap<>();
+    if (getPartitionTable() == null) {
+      return groupSlotMap;
+    }
+    Map<RaftNode, DataGroupMember> headerMap = getDataClusterServer().getHeaderGroupMap();
+    syncLocalApply(getPartitionTable().getLastMetaLogIndex(), false);
+    synchronized (headerMap) {
+      for (DataGroupMember dataMember : headerMap.values()) {
+        int num = dataMember.getSlotManager().getSloNumInDataMigration();
+        if (num > 0) {
+          groupSlotMap.put(dataMember.getPartitionGroup(), num);
+        }
+      }
+    }
+    return groupSlotMap;
+  }
+
   @Override
-  public void setAllNodes(List<Node> allNodes) {
-    super.setAllNodes(allNodes);
+  public void setAllNodes(PartitionGroup allNodes) {
+    super.setAllNodes(new PartitionGroup(allNodes));
     initPeerMap();
     idNodeMap = new HashMap<>();
     for (Node node : allNodes) {
@@ -1775,17 +1967,17 @@ public class MetaGroupMember extends RaftMember {
    * @param request the toString() of this parameter should explain what the request is and it is
    *     only used in logs for tracing
    */
-  public DataGroupMember getLocalDataMember(Node header, Object request) {
+  public DataGroupMember getLocalDataMember(RaftNode header, Object request) {
     return dataClusterServer.getDataMember(header, null, request);
   }
 
   /**
    * Get a local DataGroupMember that is in the group of "header" for an internal request.
    *
-   * @param header the header of the group which the local node is in
+   * @param raftNode the header of the group which the local node is in
    */
-  public DataGroupMember getLocalDataMember(Node header) {
-    return dataClusterServer.getDataMember(header, null, "Internal call");
+  public DataGroupMember getLocalDataMember(RaftNode raftNode) {
+    return dataClusterServer.getDataMember(raftNode, null, "Internal call");
   }
 
   public DataClientProvider getClientProvider() {
@@ -1809,6 +2001,10 @@ public class MetaGroupMember extends RaftMember {
     this.dataClientProvider = dataClientProvider;
   }
 
+  public void setRouter(ClusterPlanRouter router) {
+    this.router = router;
+  }
+
   public void handleHandshake(Node sender) {
     NodeStatusManager.getINSTANCE().activate(sender);
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/RaftMember.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/RaftMember.java
index f2187f9..a5a58b0 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/RaftMember.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/RaftMember.java
@@ -38,6 +38,7 @@ import org.apache.iotdb.cluster.log.LogParser;
 import org.apache.iotdb.cluster.log.catchup.CatchUpTask;
 import org.apache.iotdb.cluster.log.logtypes.PhysicalPlanLog;
 import org.apache.iotdb.cluster.log.manage.RaftLogManager;
+import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.rpc.thrift.AppendEntriesRequest;
 import org.apache.iotdb.cluster.rpc.thrift.AppendEntryRequest;
 import org.apache.iotdb.cluster.rpc.thrift.ElectionRequest;
@@ -45,6 +46,7 @@ import org.apache.iotdb.cluster.rpc.thrift.ExecutNonQueryReq;
 import org.apache.iotdb.cluster.rpc.thrift.HeartBeatRequest;
 import org.apache.iotdb.cluster.rpc.thrift.HeartBeatResponse;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.RaftService.AsyncClient;
 import org.apache.iotdb.cluster.rpc.thrift.RaftService.Client;
 import org.apache.iotdb.cluster.rpc.thrift.RequestCommitIndexResponse;
@@ -71,6 +73,7 @@ import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.executor.PlanExecutor;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.LogPlan;
 import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
@@ -106,6 +109,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
+import static org.apache.iotdb.cluster.config.ClusterConstant.THREAD_POLL_WAIT_TERMINATION_TIME_S;
+
 /**
  * RaftMember process the common raft logic like leader election, log appending, catch-up and so on.
  */
@@ -135,9 +140,12 @@ public abstract class RaftMember {
   /** the lock is to make sure that only one thread can apply snapshot at the same time */
   private final Object snapshotApplyLock = new Object();
 
+  private final Object heartBeatWaitObject = new Object();
+
   protected Node thisNode = ClusterConstant.EMPTY_NODE;
+
   /** the nodes that belong to the same raft group as thisNode. */
-  protected List<Node> allNodes;
+  protected PartitionGroup allNodes;
 
   ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
   /** the name of the member, to distinguish several members in the logs. */
@@ -221,12 +229,16 @@ public abstract class RaftMember {
   private ExecutorService serialToParallelPool;
   /** a thread pool that is used to do commit log tasks asynchronous in heartbeat thread */
   private ExecutorService commitLogPool;
+
   /**
    * logDispatcher buff the logs orderly according to their log indexes and send them sequentially,
    * which avoids the followers receiving out-of-order logs, forcing them to wait for previous logs.
    */
   private LogDispatcher logDispatcher;
 
+  /** If this node can not be the leader, this parameter will be set true. */
+  private volatile boolean skipElection = false;
+
   /**
    * localExecutor is used to directly execute plans like load configuration in the underlying IoTDB
    */
@@ -323,6 +335,7 @@ public abstract class RaftMember {
    * induce side effects.
    */
   public void stop() {
+    setSkipElection(true);
     closeLogManager();
     if (heartBeatService == null) {
       return;
@@ -332,9 +345,9 @@ public abstract class RaftMember {
     catchUpService.shutdownNow();
     appendLogThreadPool.shutdownNow();
     try {
-      heartBeatService.awaitTermination(10, TimeUnit.SECONDS);
-      catchUpService.awaitTermination(10, TimeUnit.SECONDS);
-      appendLogThreadPool.awaitTermination(10, TimeUnit.SECONDS);
+      heartBeatService.awaitTermination(THREAD_POLL_WAIT_TERMINATION_TIME_S, TimeUnit.SECONDS);
+      catchUpService.awaitTermination(THREAD_POLL_WAIT_TERMINATION_TIME_S, TimeUnit.SECONDS);
+      appendLogThreadPool.awaitTermination(THREAD_POLL_WAIT_TERMINATION_TIME_S, TimeUnit.SECONDS);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       logger.error(
@@ -345,7 +358,8 @@ public abstract class RaftMember {
     if (serialToParallelPool != null) {
       serialToParallelPool.shutdownNow();
       try {
-        serialToParallelPool.awaitTermination(10, TimeUnit.SECONDS);
+        serialToParallelPool.awaitTermination(
+            THREAD_POLL_WAIT_TERMINATION_TIME_S, TimeUnit.SECONDS);
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         logger.error("Unexpected interruption when waiting for asyncThreadPool to end", e);
@@ -355,12 +369,13 @@ public abstract class RaftMember {
     if (commitLogPool != null) {
       commitLogPool.shutdownNow();
       try {
-        commitLogPool.awaitTermination(10, TimeUnit.SECONDS);
+        commitLogPool.awaitTermination(THREAD_POLL_WAIT_TERMINATION_TIME_S, TimeUnit.SECONDS);
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         logger.error("Unexpected interruption when waiting for commitLogPool to end", e);
       }
     }
+    leader.set(ClusterConstant.EMPTY_NODE);
     catchUpService = null;
     heartBeatService = null;
     appendLogThreadPool = null;
@@ -390,8 +405,8 @@ public abstract class RaftMember {
         // a leader with a term lower than this node is invalid, send it the local term to inform
         // it to resign
         response.setTerm(thisTerm);
-        if (logger.isTraceEnabled()) {
-          logger.trace("{} received a heartbeat from a stale leader {}", name, request.getLeader());
+        if (logger.isDebugEnabled()) {
+          logger.debug("{} received a heartbeat from a stale leader {}", name, request.getLeader());
         }
       } else {
         // try updating local term
@@ -408,11 +423,16 @@ public abstract class RaftMember {
         response.setTerm(Response.RESPONSE_AGREE);
         // tell the leader who I am in case of catch-up
         response.setFollower(thisNode);
+        // tell the leader the local log progress so it may decide whether to perform a catch up
+        response.setLastLogIndex(logManager.getLastLogIndex());
+        response.setLastLogTerm(logManager.getLastLogTerm());
 
-        synchronized (logManager) {
-          // tell the leader the local log progress so it may decide whether to perform a catch up
-          response.setLastLogIndex(logManager.getLastLogIndex());
-          response.setLastLogTerm(logManager.getLastLogTerm());
+        if (logger.isDebugEnabled()) {
+          logger.debug(
+              "{}: log commit log index = {}, max have applied commit index = {}",
+              name,
+              logManager.getCommitLogIndex(),
+              logManager.getMaxHaveAppliedCommitIndex());
         }
 
         tryUpdateCommitIndex(leaderTerm, request.getCommitLogIndex(), request.getCommitLogTerm());
@@ -455,6 +475,10 @@ public abstract class RaftMember {
    * whether to accept by examining the log status of the elector.
    */
   public long processElectionRequest(ElectionRequest electionRequest) {
+    if (logger.isDebugEnabled()) {
+      logger.debug(
+          "{}: start to handle request from elector {}", name, electionRequest.getElector());
+    }
     synchronized (term) {
       long currentTerm = term.get();
       long response =
@@ -672,7 +696,11 @@ public abstract class RaftMember {
     return allNodes;
   }
 
-  public void setAllNodes(List<Node> allNodes) {
+  public PartitionGroup getPartitionGroup() {
+    return allNodes;
+  }
+
+  public void setAllNodes(PartitionGroup allNodes) {
     this.allNodes = allNodes;
   }
 
@@ -709,7 +737,8 @@ public abstract class RaftMember {
     logger.info("{}: Start to make {} catch up", name, follower);
     if (!catchUpService.isShutdown()) {
       Future<?> future =
-          catchUpService.submit(new CatchUpTask(follower, peerMap.get(follower), this, lastLogIdx));
+          catchUpService.submit(
+              new CatchUpTask(follower, getRaftGroupId(), peerMap.get(follower), this, lastLogIdx));
       catchUpService.submit(
           () -> {
             try {
@@ -898,7 +927,7 @@ public abstract class RaftMember {
       tryUpdateCommitIndex(
           response.getTerm(), response.getCommitLogIndex(), response.getCommitLogTerm());
 
-      return syncLocalApply(leaderCommitId);
+      return syncLocalApply(leaderCommitId, true);
     } catch (TException e) {
       logger.error(MSG_NO_LEADER_COMMIT_INDEX, name, leader.get(), e);
     } catch (InterruptedException e) {
@@ -910,6 +939,7 @@ public abstract class RaftMember {
             leaderCommitId, logManager.getMaxHaveAppliedCommitIndex());
       }
     }
+    logger.debug("Start to sync with leader, leader commit id is {}", leaderCommitId);
     return false;
   }
 
@@ -917,12 +947,22 @@ public abstract class RaftMember {
    * sync local applyId to leader commitId
    *
    * @param leaderCommitId leader commit id
+   * @param fastFail if enable, when log differ too much, return false directly.
    * @return true if leaderCommitId <= localAppliedId
    */
-  private boolean syncLocalApply(long leaderCommitId) {
+  public boolean syncLocalApply(long leaderCommitId, boolean fastFail) {
     long startTime = System.currentTimeMillis();
     long waitedTime = 0;
-    long localAppliedId = 0;
+    long localAppliedId;
+
+    if (fastFail) {
+      if (leaderCommitId - logManager.getMaxHaveAppliedCommitIndex() > config.getMaxSyncLogLag()) {
+        logger.info(
+            "{}: The raft log of this member is too backward to provide service directly.", name);
+        return false;
+      }
+    }
+
     while (waitedTime < RaftServer.getSyncLeaderMaxWaitMs()) {
       try {
         localAppliedId = logManager.getMaxHaveAppliedCommitIndex();
@@ -931,7 +971,8 @@ public abstract class RaftMember {
           // this node has caught up
           if (logger.isDebugEnabled()) {
             waitedTime = System.currentTimeMillis() - startTime;
-            logger.debug("{}: synchronized with the leader after {}ms", name, waitedTime);
+            logger.debug(
+                "{}: synchronized to target index {} after {}ms", name, leaderCommitId, waitedTime);
           }
           return true;
         }
@@ -947,8 +988,11 @@ public abstract class RaftMember {
         logger.error(MSG_NO_LEADER_COMMIT_INDEX, name, leader.get(), e);
       }
     }
-    logger.warn("{}: Failed to synchronize with the leader after {}ms", name, waitedTime);
-
+    logger.warn(
+        "{}: Failed to synchronize to target index {} after {}ms",
+        name,
+        leaderCommitId,
+        waitedTime);
     return false;
   }
 
@@ -965,19 +1009,29 @@ public abstract class RaftMember {
     }
 
     logger.debug("{}: Processing plan {}", name, plan);
-    if (readOnly) {
+    if (readOnly && !(plan instanceof LogPlan)) {
       return StatusUtils.NODE_READ_ONLY;
     }
     long startTime = Timer.Statistic.RAFT_SENDER_APPEND_LOG.getOperationStartTime();
-    PhysicalPlanLog log = new PhysicalPlanLog();
+
+    Log log;
     // assign term and index to the new log and append it
     synchronized (logManager) {
+      if (plan instanceof LogPlan) {
+        try {
+          log = LogParser.getINSTANCE().parse(((LogPlan) plan).getLog());
+        } catch (UnknownLogTypeException e) {
+          logger.error("Can not parse LogPlan {}", plan, e);
+          return StatusUtils.PARSE_LOG_ERROR;
+        }
+      } else {
+        log = new PhysicalPlanLog();
+        ((PhysicalPlanLog) log).setPlan(plan);
+        plan.setIndex(logManager.getLastLogIndex() + 1);
+      }
       log.setCurrLogTerm(getTerm().get());
       log.setCurrLogIndex(logManager.getLastLogIndex() + 1);
 
-      log.setPlan(plan);
-      plan.setIndex(log.getCurrLogIndex());
-      // appendLogInGroup will serialize log, and set log size, and we will use the size after it
       logManager.append(log);
     }
     Timer.Statistic.RAFT_SENDER_APPEND_LOG.calOperationCostTimeFromStart(startTime);
@@ -987,7 +1041,7 @@ public abstract class RaftMember {
         return StatusUtils.OK;
       }
     } catch (LogExecutionException e) {
-      return handleLogExecutionException(log.getPlan(), IOUtils.getRootCause(e));
+      return handleLogExecutionException(log, IOUtils.getRootCause(e));
     }
     return StatusUtils.TIME_OUT;
   }
@@ -997,20 +1051,30 @@ public abstract class RaftMember {
     if (readOnly) {
       return StatusUtils.NODE_READ_ONLY;
     }
-    PhysicalPlanLog log = new PhysicalPlanLog();
     // assign term and index to the new log and append it
     SendLogRequest sendLogRequest;
 
     long startTime =
         Statistic.RAFT_SENDER_COMPETE_LOG_MANAGER_BEFORE_APPEND_V2.getOperationStartTime();
+    Log log;
     synchronized (logManager) {
       Statistic.RAFT_SENDER_COMPETE_LOG_MANAGER_BEFORE_APPEND_V2.calOperationCostTimeFromStart(
           startTime);
 
+      if (plan instanceof LogPlan) {
+        try {
+          log = LogParser.getINSTANCE().parse(((LogPlan) plan).getLog());
+        } catch (UnknownLogTypeException e) {
+          logger.error("Can not parse LogPlan {}", plan, e);
+          return StatusUtils.PARSE_LOG_ERROR;
+        }
+      } else {
+        log = new PhysicalPlanLog();
+        ((PhysicalPlanLog) log).setPlan(plan);
+        plan.setIndex(logManager.getLastLogIndex() + 1);
+      }
       log.setCurrLogTerm(getTerm().get());
       log.setCurrLogIndex(logManager.getLastLogIndex() + 1);
-      log.setPlan(plan);
-      plan.setIndex(log.getCurrLogIndex());
 
       startTime = Timer.Statistic.RAFT_SENDER_APPEND_LOG_V2.getOperationStartTime();
       // logDispatcher will serialize log, and set log size, and we will use the size after it
@@ -1051,7 +1115,7 @@ public abstract class RaftMember {
           break;
       }
     } catch (LogExecutionException e) {
-      return handleLogExecutionException(log.getPlan(), IOUtils.getRootCause(e));
+      return handleLogExecutionException(log, IOUtils.getRootCause(e));
     }
     return StatusUtils.TIME_OUT;
   }
@@ -1095,7 +1159,7 @@ public abstract class RaftMember {
     }
     synchronized (commitIdResult) {
       client.requestCommitIndex(getHeader(), new GenericHandler<>(leader.get(), commitIdResult));
-      commitIdResult.wait(RaftServer.getSyncLeaderMaxWaitMs());
+      commitIdResult.wait(RaftServer.getReadOperationTimeoutMS());
     }
     return commitIdResult.get();
   }
@@ -1167,6 +1231,10 @@ public abstract class RaftMember {
     return serialToParallelPool;
   }
 
+  public ExecutorService getAppendLogThreadPool() {
+    return appendLogThreadPool;
+  }
+
   public Object getSyncLock() {
     return syncLock;
   }
@@ -1186,6 +1254,7 @@ public abstract class RaftMember {
     long thatTerm = electionRequest.getTerm();
     long thatLastLogIndex = electionRequest.getLastLogIndex();
     long thatLastLogTerm = electionRequest.getLastLogTerm();
+    Node elector = electionRequest.getElector();
 
     // check the log progress of the elector
     long resp = checkLogProgress(thatLastLogIndex, thatLastLogTerm);
@@ -1201,7 +1270,7 @@ public abstract class RaftMember {
           logManager.getLastLogTerm());
       setCharacter(NodeCharacter.FOLLOWER);
       lastHeartbeatReceivedTime = System.currentTimeMillis();
-      setVoteFor(electionRequest.getElector());
+      setVoteFor(elector);
       updateHardState(thatTerm, getVoteFor());
     } else {
       logger.info(
@@ -1245,7 +1314,7 @@ public abstract class RaftMember {
    *     communication
    * @return a TSStatus indicating if the forwarding is successful.
    */
-  public TSStatus forwardPlan(PhysicalPlan plan, Node node, Node header) {
+  public TSStatus forwardPlan(PhysicalPlan plan, Node node, RaftNode header) {
     if (node == null || node.equals(thisNode)) {
       logger.debug("{}: plan {} has no where to be forwarded", name, plan);
       return StatusUtils.NO_LEADER;
@@ -1277,7 +1346,7 @@ public abstract class RaftMember {
    * @param header to determine which DataGroupMember of "receiver" will process the request.
    * @return a TSStatus indicating if the forwarding is successful.
    */
-  private TSStatus forwardPlanAsync(PhysicalPlan plan, Node receiver, Node header) {
+  private TSStatus forwardPlanAsync(PhysicalPlan plan, Node receiver, RaftNode header) {
     AsyncClient client = getAsyncClient(receiver);
     if (client == null) {
       logger.debug("{}: can not get client for node={}", name, receiver);
@@ -1289,7 +1358,7 @@ public abstract class RaftMember {
   }
 
   public TSStatus forwardPlanAsync(
-      PhysicalPlan plan, Node receiver, Node header, AsyncClient client) {
+      PhysicalPlan plan, Node receiver, RaftNode header, AsyncClient client) {
     try {
       TSStatus tsStatus = SyncClientAdaptor.executeNonQuery(client, plan, header, receiver);
       if (tsStatus == null) {
@@ -1307,7 +1376,7 @@ public abstract class RaftMember {
     }
   }
 
-  private TSStatus forwardPlanSync(PhysicalPlan plan, Node receiver, Node header) {
+  private TSStatus forwardPlanSync(PhysicalPlan plan, Node receiver, RaftNode header) {
     Client client = getSyncClient(receiver);
     if (client == null) {
       logger.warn(MSG_FORWARD_TIMEOUT, name, plan, receiver);
@@ -1316,7 +1385,8 @@ public abstract class RaftMember {
     return forwardPlanSync(plan, receiver, header, client);
   }
 
-  public TSStatus forwardPlanSync(PhysicalPlan plan, Node receiver, Node header, Client client) {
+  public TSStatus forwardPlanSync(
+      PhysicalPlan plan, Node receiver, RaftNode header, Client client) {
     try {
       ExecutNonQueryReq req = new ExecutNonQueryReq();
       req.setPlanBytes(PlanSerializer.getInstance().serialize(plan));
@@ -1483,7 +1553,7 @@ public abstract class RaftMember {
     }
   }
 
-  protected TSStatus handleLogExecutionException(PhysicalPlan log, Throwable cause) {
+  protected TSStatus handleLogExecutionException(Object log, Throwable cause) {
     if (cause instanceof BatchProcessException) {
       return RpcUtils.getStatus(Arrays.asList(((BatchProcessException) cause).getFailingStatus()));
     }
@@ -1551,7 +1621,8 @@ public abstract class RaftMember {
         logger.info("{} has update it's term to {}", getName(), newTerm);
         term.set(newTerm);
         setVoteFor(null);
-        setLeader(ClusterConstant.EMPTY_NODE);
+        setCharacter(NodeCharacter.ELECTOR);
+        setLeader(null);
         updateHardState(newTerm, getVoteFor());
       }
 
@@ -1573,7 +1644,7 @@ public abstract class RaftMember {
   }
 
   /** @return the header of the data raft group or null if this is in a meta group. */
-  public Node getHeader() {
+  public RaftNode getHeader() {
     return null;
   }
 
@@ -1616,10 +1687,10 @@ public abstract class RaftMember {
       long startTime = Timer.Statistic.RAFT_SENDER_SEND_LOG_TO_FOLLOWERS.getOperationStartTime();
       logger.debug("{}: Send log {} to other nodes, retry times: {}", name, log, retryTime);
       if (character != NodeCharacter.LEADER) {
-        logger.debug("Has lose leadership, so need not to send log");
+        logger.debug("{}: Has lose leadership, so need not to send log", name);
         return false;
       }
-      AppendLogResult result = sendLogToFollowers(log, allNodes.size() / 2);
+      AppendLogResult result = sendLogToFollowers(log);
       Timer.Statistic.RAFT_SENDER_SEND_LOG_TO_FOLLOWERS.calOperationCostTimeFromStart(startTime);
       switch (result) {
         case OK:
@@ -1631,7 +1702,7 @@ public abstract class RaftMember {
         case TIME_OUT:
           logger.debug("{}: log {} timed out, retrying...", name, log);
           try {
-            Thread.sleep(1000);
+            Thread.sleep(ClusterConstant.RETRY_WAIT_TIME_MS);
           } catch (InterruptedException e) {
             Thread.currentThread().interrupt();
           }
@@ -1652,14 +1723,13 @@ public abstract class RaftMember {
    * Send the given log to all the followers and decide the result by how many followers return a
    * success.
    *
-   * @param requiredQuorum the number of votes needed to make the log valid, when requiredQuorum <=
-   *     0, half of the cluster size will be used.
    * @return an AppendLogResult
    */
-  private AppendLogResult sendLogToFollowers(Log log, int requiredQuorum) {
+  protected AppendLogResult sendLogToFollowers(Log log) {
+    int requiredQuorum = allNodes.size() / 2;
     if (requiredQuorum <= 0) {
       // use half of the members' size as the quorum
-      return sendLogToFollowers(log, new AtomicInteger(allNodes.size() / 2));
+      return sendLogToFollowers(log, new AtomicInteger(requiredQuorum));
     } else {
       // make sure quorum does not exceed the number of members - 1
       return sendLogToFollowers(
@@ -1845,7 +1915,7 @@ public abstract class RaftMember {
    * @return Response.RESPONSE_AGREE when the log is successfully appended or Response
    *     .RESPONSE_LOG_MISMATCH if the previous log of "log" is not found.
    */
-  private long appendEntry(long prevLogIndex, long prevLogTerm, long leaderCommit, Log log) {
+  protected long appendEntry(long prevLogIndex, long prevLogTerm, long leaderCommit, Log log) {
     long resp = checkPrevLogIndex(prevLogIndex);
     if (resp != Response.RESPONSE_AGREE) {
       return resp;
@@ -1982,9 +2052,25 @@ public abstract class RaftMember {
     return Response.RESPONSE_AGREE;
   }
 
+  public int getRaftGroupId() {
+    return allNodes.getId();
+  }
+
   enum AppendLogResult {
     OK,
     TIME_OUT,
     LEADERSHIP_STALE
   }
+
+  public Object getHeartBeatWaitObject() {
+    return heartBeatWaitObject;
+  }
+
+  public boolean isSkipElection() {
+    return skipElection;
+  }
+
+  public void setSkipElection(boolean skipElection) {
+    this.skipElection = skipElection;
+  }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/monitor/NodeReport.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/monitor/NodeReport.java
index 313333f..10538be 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/monitor/NodeReport.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/monitor/NodeReport.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.cluster.server.monitor;
 
 import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.server.NodeCharacter;
 import org.apache.iotdb.rpc.RpcStat;
 import org.apache.iotdb.rpc.RpcTransportFactory;
@@ -195,7 +196,7 @@ public class NodeReport {
    * belongs to.
    */
   public static class DataMemberReport extends RaftMemberReport {
-    Node header;
+    RaftNode header;
     long headerLatency;
 
     public DataMemberReport(
@@ -206,7 +207,7 @@ public class NodeReport {
         long lastLogIndex,
         long commitIndex,
         long commitTerm,
-        Node header,
+        RaftNode header,
         boolean isReadOnly,
         long headerLatency,
         long lastHeartbeatReceivedTime,
@@ -232,7 +233,9 @@ public class NodeReport {
     public String toString() {
       return "DataMemberReport{"
           + "header="
-          + header
+          + header.getNode()
+          + ", raftId="
+          + header.getRaftId()
           + ", character="
           + character
           + ", Leader="
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/service/BaseAsyncService.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/service/BaseAsyncService.java
index 8673078..f7c69c4 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/service/BaseAsyncService.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/service/BaseAsyncService.java
@@ -27,7 +27,7 @@ import org.apache.iotdb.cluster.rpc.thrift.ElectionRequest;
 import org.apache.iotdb.cluster.rpc.thrift.ExecutNonQueryReq;
 import org.apache.iotdb.cluster.rpc.thrift.HeartBeatRequest;
 import org.apache.iotdb.cluster.rpc.thrift.HeartBeatResponse;
-import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.RaftService;
 import org.apache.iotdb.cluster.rpc.thrift.RaftService.AsyncClient;
 import org.apache.iotdb.cluster.rpc.thrift.RequestCommitIndexResponse;
@@ -87,7 +87,7 @@ public abstract class BaseAsyncService implements RaftService.AsyncIface {
 
   @Override
   public void requestCommitIndex(
-      Node header, AsyncMethodCallback<RequestCommitIndexResponse> resultHandler) {
+      RaftNode header, AsyncMethodCallback<RequestCommitIndexResponse> resultHandler) {
     long commitIndex;
     long commitTerm;
     long curTerm;
@@ -140,7 +140,7 @@ public abstract class BaseAsyncService implements RaftService.AsyncIface {
 
   @Override
   public void matchTerm(
-      long index, long term, Node header, AsyncMethodCallback<Boolean> resultHandler) {
+      long index, long term, RaftNode header, AsyncMethodCallback<Boolean> resultHandler) {
     resultHandler.onComplete(member.matchLog(index, term));
   }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/service/BaseSyncService.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/service/BaseSyncService.java
index ce200ab..a54a9cc 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/service/BaseSyncService.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/service/BaseSyncService.java
@@ -27,7 +27,7 @@ import org.apache.iotdb.cluster.rpc.thrift.ElectionRequest;
 import org.apache.iotdb.cluster.rpc.thrift.ExecutNonQueryReq;
 import org.apache.iotdb.cluster.rpc.thrift.HeartBeatRequest;
 import org.apache.iotdb.cluster.rpc.thrift.HeartBeatResponse;
-import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.RaftService;
 import org.apache.iotdb.cluster.rpc.thrift.RaftService.Client;
 import org.apache.iotdb.cluster.rpc.thrift.RequestCommitIndexResponse;
@@ -94,7 +94,7 @@ public abstract class BaseSyncService implements RaftService.Iface {
   }
 
   @Override
-  public RequestCommitIndexResponse requestCommitIndex(Node header) throws TException {
+  public RequestCommitIndexResponse requestCommitIndex(RaftNode header) throws TException {
 
     long commitIndex;
     long commitTerm;
@@ -147,7 +147,7 @@ public abstract class BaseSyncService implements RaftService.Iface {
   }
 
   @Override
-  public boolean matchTerm(long index, long term, Node header) {
+  public boolean matchTerm(long index, long term, RaftNode header) {
     return member.matchLog(index, term);
   }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/service/DataAsyncService.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/service/DataAsyncService.java
index a592599..eab4334 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/service/DataAsyncService.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/service/DataAsyncService.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.cluster.server.service;
 
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.exception.LeaderUnknownException;
 import org.apache.iotdb.cluster.exception.ReaderNotFoundException;
@@ -35,6 +36,7 @@ import org.apache.iotdb.cluster.rpc.thrift.PullSchemaRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSchemaResp;
 import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotResp;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.SendSnapshotRequest;
 import org.apache.iotdb.cluster.rpc.thrift.SingleSeriesQueryRequest;
 import org.apache.iotdb.cluster.rpc.thrift.TSDataService;
@@ -97,7 +99,8 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
       PullSnapshotRequest request, AsyncMethodCallback<PullSnapshotResp> resultHandler) {
     // if this node has been set readOnly, then it must have been synchronized with the leader
     // otherwise forward the request to the leader
-    if (dataGroupMember.getLeader() != null) {
+    if (dataGroupMember.getLeader() != null
+        && !ClusterConstant.EMPTY_NODE.equals(dataGroupMember.getLeader())) {
       logger.debug(
           "{} forwarding a pull snapshot request to the leader {}",
           name,
@@ -129,7 +132,16 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
             dataGroupMember.getLocalQueryExecutor().queryTimeSeriesSchema(request));
         return;
       } catch (CheckConsistencyException | MetadataException e) {
-        resultHandler.onError(e);
+        // maybe the partition table of this node is not up-to-date, try again after updating
+        // partition table
+        try {
+          dataGroupMember.getMetaGroupMember().syncLeaderWithConsistencyCheck(false);
+          resultHandler.onComplete(
+              dataGroupMember.getLocalQueryExecutor().queryTimeSeriesSchema(request));
+          return;
+        } catch (CheckConsistencyException | MetadataException ex) {
+          resultHandler.onError(ex);
+        }
       }
     }
 
@@ -166,7 +178,16 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
             dataGroupMember.getLocalQueryExecutor().queryMeasurementSchema(request));
         return;
       } catch (CheckConsistencyException | MetadataException e) {
-        resultHandler.onError(e);
+        // maybe the partition table of this node is not up-to-date, try again after updating
+        // partition table
+        try {
+          dataGroupMember.getMetaGroupMember().syncLeaderWithConsistencyCheck(false);
+          resultHandler.onComplete(
+              dataGroupMember.getLocalQueryExecutor().queryMeasurementSchema(request));
+          return;
+        } catch (CheckConsistencyException | MetadataException ex) {
+          resultHandler.onError(ex);
+        }
       }
     }
 
@@ -216,7 +237,7 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void endQuery(
-      Node header, Node requester, long queryId, AsyncMethodCallback<Void> resultHandler) {
+      RaftNode header, Node requester, long queryId, AsyncMethodCallback<Void> resultHandler) {
     try {
       dataGroupMember.getQueryManager().endQuery(requester, queryId);
       resultHandler.onComplete(null);
@@ -227,7 +248,7 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void fetchSingleSeries(
-      Node header, long readerId, AsyncMethodCallback<ByteBuffer> resultHandler) {
+      RaftNode header, long readerId, AsyncMethodCallback<ByteBuffer> resultHandler) {
     try {
       resultHandler.onComplete(dataGroupMember.getLocalQueryExecutor().fetchSingleSeries(readerId));
     } catch (ReaderNotFoundException | IOException e) {
@@ -237,7 +258,7 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void fetchMultSeries(
-      Node header,
+      RaftNode header,
       long readerId,
       List<String> paths,
       AsyncMethodCallback<Map<String, ByteBuffer>> resultHandler)
@@ -252,7 +273,7 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void fetchSingleSeriesByTimestamps(
-      Node header,
+      RaftNode header,
       long readerId,
       List<Long> timestamps,
       AsyncMethodCallback<ByteBuffer> resultHandler) {
@@ -269,7 +290,7 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void getAllPaths(
-      Node header,
+      RaftNode header,
       List<String> paths,
       boolean withAlias,
       AsyncMethodCallback<GetAllPathsResult> resultHandler) {
@@ -283,7 +304,7 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void getAllDevices(
-      Node header, List<String> path, AsyncMethodCallback<Set<String>> resultHandler) {
+      RaftNode header, List<String> path, AsyncMethodCallback<Set<String>> resultHandler) {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
       resultHandler.onComplete(((CMManager) IoTDB.metaManager).getAllDevices(path));
@@ -294,7 +315,7 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void getDevices(
-      Node header, ByteBuffer planBinary, AsyncMethodCallback<ByteBuffer> resultHandler) {
+      RaftNode header, ByteBuffer planBinary, AsyncMethodCallback<ByteBuffer> resultHandler) {
     try {
       resultHandler.onComplete(dataGroupMember.getLocalQueryExecutor().getDevices(planBinary));
     } catch (CheckConsistencyException | IOException | MetadataException e) {
@@ -304,7 +325,10 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void getNodeList(
-      Node header, String path, int nodeLevel, AsyncMethodCallback<List<String>> resultHandler) {
+      RaftNode header,
+      String path,
+      int nodeLevel,
+      AsyncMethodCallback<List<String>> resultHandler) {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
       resultHandler.onComplete(((CMManager) IoTDB.metaManager).getNodeList(path, nodeLevel));
@@ -315,7 +339,7 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void getChildNodeInNextLevel(
-      Node header, String path, AsyncMethodCallback<Set<String>> resultHandler) {
+      RaftNode header, String path, AsyncMethodCallback<Set<String>> resultHandler) {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
       resultHandler.onComplete(((CMManager) IoTDB.metaManager).getChildNodeInNextLevel(path));
@@ -326,7 +350,7 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void getChildNodePathInNextLevel(
-      Node header, String path, AsyncMethodCallback<Set<String>> resultHandler) {
+      RaftNode header, String path, AsyncMethodCallback<Set<String>> resultHandler) {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
       resultHandler.onComplete(((CMManager) IoTDB.metaManager).getChildNodePathInNextLevel(path));
@@ -337,7 +361,7 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void getAllMeasurementSchema(
-      Node header, ByteBuffer planBinary, AsyncMethodCallback<ByteBuffer> resultHandler) {
+      RaftNode header, ByteBuffer planBinary, AsyncMethodCallback<ByteBuffer> resultHandler) {
     try {
       resultHandler.onComplete(
           dataGroupMember.getLocalQueryExecutor().getAllMeasurementSchema(planBinary));
@@ -358,7 +382,9 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void getUnregisteredTimeseries(
-      Node header, List<String> timeseriesList, AsyncMethodCallback<List<String>> resultHandler) {
+      RaftNode header,
+      List<String> timeseriesList,
+      AsyncMethodCallback<List<String>> resultHandler) {
     try {
       resultHandler.onComplete(
           dataGroupMember.getLocalQueryExecutor().getUnregisteredTimeseries(timeseriesList));
@@ -378,7 +404,7 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void getGroupByResult(
-      Node header,
+      RaftNode header,
       long executorId,
       long startTime,
       long endTime,
@@ -419,7 +445,7 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void getPathCount(
-      Node header,
+      RaftNode header,
       List<String> pathsToQuery,
       int level,
       AsyncMethodCallback<Integer> resultHandler) {
@@ -433,13 +459,13 @@ public class DataAsyncService extends BaseAsyncService implements TSDataService.
 
   @Override
   public void onSnapshotApplied(
-      Node header, List<Integer> slots, AsyncMethodCallback<Boolean> resultHandler) {
+      RaftNode header, List<Integer> slots, AsyncMethodCallback<Boolean> resultHandler) {
     resultHandler.onComplete(dataGroupMember.onSnapshotInstalled(slots));
   }
 
   @Override
   public void peekNextNotNullValue(
-      Node header,
+      RaftNode header,
       long executorId,
       long startTime,
       long endTime,
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/service/DataSyncService.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/service/DataSyncService.java
index 062dcc7..c43b617 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/service/DataSyncService.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/service/DataSyncService.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.cluster.server.service;
 
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.exception.LeaderUnknownException;
 import org.apache.iotdb.cluster.exception.ReaderNotFoundException;
@@ -35,6 +36,7 @@ import org.apache.iotdb.cluster.rpc.thrift.PullSchemaRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSchemaResp;
 import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotResp;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.SendSnapshotRequest;
 import org.apache.iotdb.cluster.rpc.thrift.SingleSeriesQueryRequest;
 import org.apache.iotdb.cluster.rpc.thrift.TSDataService;
@@ -94,7 +96,8 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   private PullSnapshotResp forwardPullSnapshot(PullSnapshotRequest request) throws TException {
     // if this node has been set readOnly, then it must have been synchronized with the leader
     // otherwise forward the request to the leader
-    if (dataGroupMember.getLeader() != null) {
+    if (dataGroupMember.getLeader() != null
+        && !ClusterConstant.EMPTY_NODE.equals(dataGroupMember.getLeader())) {
       logger.debug(
           "{} forwarding a pull snapshot request to the leader {}",
           name,
@@ -134,7 +137,14 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
       try {
         return dataGroupMember.getLocalQueryExecutor().queryTimeSeriesSchema(request);
       } catch (CheckConsistencyException | MetadataException e) {
-        throw new TException(e);
+        // maybe the partition table of this node is not up-to-date, try again after updating
+        // partition table
+        try {
+          dataGroupMember.getMetaGroupMember().syncLeaderWithConsistencyCheck(false);
+          return dataGroupMember.getLocalQueryExecutor().queryTimeSeriesSchema(request);
+        } catch (CheckConsistencyException | MetadataException ex) {
+          throw new TException(ex);
+        }
       }
     }
 
@@ -169,8 +179,15 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
     if (dataGroupMember.getCharacter() == NodeCharacter.LEADER) {
       try {
         return dataGroupMember.getLocalQueryExecutor().queryMeasurementSchema(request);
-      } catch (CheckConsistencyException | IllegalPathException e) {
-        throw new TException(e);
+      } catch (CheckConsistencyException | MetadataException e) {
+        // maybe the partition table of this node is not up-to-date, try again after updating
+        // partition table
+        try {
+          dataGroupMember.getMetaGroupMember().syncLeaderWithConsistencyCheck(false);
+          return dataGroupMember.getLocalQueryExecutor().queryMeasurementSchema(request);
+        } catch (CheckConsistencyException | MetadataException ex) {
+          throw new TException(ex);
+        }
       }
     }
 
@@ -221,7 +238,7 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public void endQuery(Node header, Node requester, long queryId) throws TException {
+  public void endQuery(RaftNode header, Node requester, long queryId) throws TException {
     try {
       dataGroupMember.getQueryManager().endQuery(requester, queryId);
     } catch (StorageEngineException e) {
@@ -230,7 +247,7 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public ByteBuffer fetchSingleSeries(Node header, long readerId) throws TException {
+  public ByteBuffer fetchSingleSeries(RaftNode header, long readerId) throws TException {
     try {
       return dataGroupMember.getLocalQueryExecutor().fetchSingleSeries(readerId);
     } catch (ReaderNotFoundException | IOException e) {
@@ -239,7 +256,7 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public Map<String, ByteBuffer> fetchMultSeries(Node header, long readerId, List<String> paths)
+  public Map<String, ByteBuffer> fetchMultSeries(RaftNode header, long readerId, List<String> paths)
       throws TException {
     try {
       return dataGroupMember.getLocalQueryExecutor().fetchMultSeries(readerId, paths);
@@ -249,8 +266,8 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public ByteBuffer fetchSingleSeriesByTimestamps(Node header, long readerId, List<Long> timestamps)
-      throws TException {
+  public ByteBuffer fetchSingleSeriesByTimestamps(
+      RaftNode header, long readerId, List<Long> timestamps) throws TException {
     try {
       return dataGroupMember
           .getLocalQueryExecutor()
@@ -262,7 +279,7 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public GetAllPathsResult getAllPaths(Node header, List<String> paths, boolean withAlias)
+  public GetAllPathsResult getAllPaths(RaftNode header, List<String> paths, boolean withAlias)
       throws TException {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
@@ -273,7 +290,7 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public Set<String> getAllDevices(Node header, List<String> path) throws TException {
+  public Set<String> getAllDevices(RaftNode header, List<String> path) throws TException {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
       return ((CMManager) IoTDB.metaManager).getAllDevices(path);
@@ -283,7 +300,7 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public ByteBuffer getDevices(Node header, ByteBuffer planBinary) throws TException {
+  public ByteBuffer getDevices(RaftNode header, ByteBuffer planBinary) throws TException {
     try {
       return dataGroupMember.getLocalQueryExecutor().getDevices(planBinary);
     } catch (CheckConsistencyException | IOException | MetadataException e) {
@@ -292,7 +309,7 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public List<String> getNodeList(Node header, String path, int nodeLevel) throws TException {
+  public List<String> getNodeList(RaftNode header, String path, int nodeLevel) throws TException {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
       return ((CMManager) IoTDB.metaManager).getNodeList(path, nodeLevel);
@@ -302,7 +319,7 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public Set<String> getChildNodeInNextLevel(Node header, String path) throws TException {
+  public Set<String> getChildNodeInNextLevel(RaftNode header, String path) throws TException {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
       return ((CMManager) IoTDB.metaManager).getChildNodeInNextLevel(path);
@@ -312,7 +329,7 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public Set<String> getChildNodePathInNextLevel(Node header, String path) throws TException {
+  public Set<String> getChildNodePathInNextLevel(RaftNode header, String path) throws TException {
     try {
       dataGroupMember.syncLeaderWithConsistencyCheck(false);
       return ((CMManager) IoTDB.metaManager).getChildNodePathInNextLevel(path);
@@ -322,7 +339,8 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public ByteBuffer getAllMeasurementSchema(Node header, ByteBuffer planBinary) throws TException {
+  public ByteBuffer getAllMeasurementSchema(RaftNode header, ByteBuffer planBinary)
+      throws TException {
     try {
       return dataGroupMember.getLocalQueryExecutor().getAllMeasurementSchema(planBinary);
     } catch (CheckConsistencyException | IOException | MetadataException e) {
@@ -340,7 +358,7 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public List<String> getUnregisteredTimeseries(Node header, List<String> timeseriesList)
+  public List<String> getUnregisteredTimeseries(RaftNode header, List<String> timeseriesList)
       throws TException {
     try {
       return dataGroupMember.getLocalQueryExecutor().getUnregisteredTimeseries(timeseriesList);
@@ -360,7 +378,7 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
 
   @Override
   public List<ByteBuffer> getGroupByResult(
-      Node header, long executorId, long startTime, long endTime) throws TException {
+      RaftNode header, long executorId, long startTime, long endTime) throws TException {
     try {
       return dataGroupMember
           .getLocalQueryExecutor()
@@ -396,7 +414,7 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public int getPathCount(Node header, List<String> pathsToQuery, int level) throws TException {
+  public int getPathCount(RaftNode header, List<String> pathsToQuery, int level) throws TException {
     try {
       return dataGroupMember.getLocalQueryExecutor().getPathCount(pathsToQuery, level);
     } catch (CheckConsistencyException | MetadataException e) {
@@ -405,13 +423,13 @@ public class DataSyncService extends BaseSyncService implements TSDataService.If
   }
 
   @Override
-  public boolean onSnapshotApplied(Node header, List<Integer> slots) {
+  public boolean onSnapshotApplied(RaftNode header, List<Integer> slots) {
     return dataGroupMember.onSnapshotInstalled(slots);
   }
 
   @Override
-  public ByteBuffer peekNextNotNullValue(Node header, long executorId, long startTime, long endTime)
-      throws TException {
+  public ByteBuffer peekNextNotNullValue(
+      RaftNode header, long executorId, long startTime, long endTime) throws TException {
     try {
       return dataGroupMember
           .getLocalQueryExecutor()
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/service/MetaAsyncService.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/service/MetaAsyncService.java
index bd665f6..4f547a3 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/service/MetaAsyncService.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/service/MetaAsyncService.java
@@ -19,10 +19,13 @@
 
 package org.apache.iotdb.cluster.server.service;
 
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.exception.AddSelfException;
+import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.exception.LeaderUnknownException;
 import org.apache.iotdb.cluster.exception.LogExecutionException;
 import org.apache.iotdb.cluster.exception.PartitionTableUnavailableException;
+import org.apache.iotdb.cluster.log.logtypes.RemoveNodeLog;
 import org.apache.iotdb.cluster.rpc.thrift.AddNodeResponse;
 import org.apache.iotdb.cluster.rpc.thrift.AppendEntryRequest;
 import org.apache.iotdb.cluster.rpc.thrift.CheckStatusResponse;
@@ -41,6 +44,8 @@ import org.apache.thrift.async.AsyncMethodCallback;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.nio.ByteBuffer;
+
 public class MetaAsyncService extends BaseAsyncService implements TSMetaService.AsyncIface {
 
   private static final Logger logger = LoggerFactory.getLogger(MetaAsyncService.class);
@@ -70,7 +75,10 @@ public class MetaAsyncService extends BaseAsyncService implements TSMetaService.
     AddNodeResponse addNodeResponse = null;
     try {
       addNodeResponse = metaGroupMember.addNode(node, startUpStatus);
-    } catch (AddSelfException | LogExecutionException e) {
+    } catch (AddSelfException | LogExecutionException | CheckConsistencyException e) {
+      resultHandler.onError(e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
       resultHandler.onError(e);
     }
     if (addNodeResponse != null) {
@@ -78,7 +86,9 @@ public class MetaAsyncService extends BaseAsyncService implements TSMetaService.
       return;
     }
 
-    if (member.getCharacter() == NodeCharacter.FOLLOWER && member.getLeader() != null) {
+    if (member.getCharacter() == NodeCharacter.FOLLOWER
+        && member.getLeader() != null
+        && !ClusterConstant.EMPTY_NODE.equals(member.getLeader())) {
       logger.info("Forward the join request of {} to leader {}", node, member.getLeader());
       if (forwardAddNode(node, startUpStatus, resultHandler)) {
         return;
@@ -143,12 +153,27 @@ public class MetaAsyncService extends BaseAsyncService implements TSMetaService.
   }
 
   @Override
+  public void collectMigrationStatus(AsyncMethodCallback<ByteBuffer> resultHandler) {
+    resultHandler.onComplete(
+        ClusterUtils.serializeMigrationStatus(metaGroupMember.collectMigrationStatus()));
+  }
+
+  @Override
   public void removeNode(Node node, AsyncMethodCallback<Long> resultHandler) {
-    long result = Response.RESPONSE_NULL;
+    long result;
     try {
       result = metaGroupMember.removeNode(node);
-    } catch (PartitionTableUnavailableException | LogExecutionException e) {
+    } catch (PartitionTableUnavailableException
+        | LogExecutionException
+        | CheckConsistencyException e) {
+      logger.error("Can not remove node {}", node, e);
+      resultHandler.onError(e);
+      return;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      logger.error("Can not remove node {}", node, e);
       resultHandler.onError(e);
+      return;
     }
 
     if (result != Response.RESPONSE_NULL) {
@@ -196,8 +221,13 @@ public class MetaAsyncService extends BaseAsyncService implements TSMetaService.
    * @param resultHandler
    */
   @Override
-  public void exile(AsyncMethodCallback<Void> resultHandler) {
-    metaGroupMember.applyRemoveNode(metaGroupMember.getThisNode());
+  public void exile(ByteBuffer removeNodeLogBuffer, AsyncMethodCallback<Void> resultHandler) {
+    logger.info("{}: start to exile.", name);
+    removeNodeLogBuffer.get();
+    RemoveNodeLog removeNodeLog = new RemoveNodeLog();
+    removeNodeLog.deserialize(removeNodeLogBuffer);
+    metaGroupMember.getPartitionTable().deserialize(removeNodeLog.getPartitionTable());
+    metaGroupMember.applyRemoveNode(removeNodeLog);
     resultHandler.onComplete(null);
   }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/service/MetaSyncService.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/service/MetaSyncService.java
index 88626ba..d0fe5d1 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/service/MetaSyncService.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/service/MetaSyncService.java
@@ -20,10 +20,13 @@
 package org.apache.iotdb.cluster.server.service;
 
 import org.apache.iotdb.cluster.client.sync.SyncMetaClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.exception.AddSelfException;
+import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.exception.LeaderUnknownException;
 import org.apache.iotdb.cluster.exception.LogExecutionException;
 import org.apache.iotdb.cluster.exception.PartitionTableUnavailableException;
+import org.apache.iotdb.cluster.log.logtypes.RemoveNodeLog;
 import org.apache.iotdb.cluster.rpc.thrift.AddNodeResponse;
 import org.apache.iotdb.cluster.rpc.thrift.AppendEntryRequest;
 import org.apache.iotdb.cluster.rpc.thrift.CheckStatusResponse;
@@ -42,6 +45,8 @@ import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.nio.ByteBuffer;
+
 public class MetaSyncService extends BaseSyncService implements TSMetaService.Iface {
 
   private static final Logger logger = LoggerFactory.getLogger(MetaSyncService.class);
@@ -69,14 +74,19 @@ public class MetaSyncService extends BaseSyncService implements TSMetaService.If
     AddNodeResponse addNodeResponse;
     try {
       addNodeResponse = metaGroupMember.addNode(node, startUpStatus);
-    } catch (AddSelfException | LogExecutionException e) {
+    } catch (AddSelfException | LogExecutionException | CheckConsistencyException e) {
+      throw new TException(e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
       throw new TException(e);
     }
     if (addNodeResponse != null) {
       return addNodeResponse;
     }
 
-    if (member.getCharacter() == NodeCharacter.FOLLOWER && member.getLeader() != null) {
+    if (member.getCharacter() == NodeCharacter.FOLLOWER
+        && member.getLeader() != null
+        && !ClusterConstant.EMPTY_NODE.equals(member.getLeader())) {
       logger.info("Forward the join request of {} to leader {}", node, member.getLeader());
       addNodeResponse = forwardAddNode(node, startUpStatus);
       if (addNodeResponse != null) {
@@ -138,11 +148,23 @@ public class MetaSyncService extends BaseSyncService implements TSMetaService.If
   }
 
   @Override
+  public ByteBuffer collectMigrationStatus() {
+    return ClusterUtils.serializeMigrationStatus(metaGroupMember.collectMigrationStatus());
+  }
+
+  @Override
   public long removeNode(Node node) throws TException {
     long result;
     try {
       result = metaGroupMember.removeNode(node);
-    } catch (PartitionTableUnavailableException | LogExecutionException e) {
+    } catch (PartitionTableUnavailableException
+        | LogExecutionException
+        | CheckConsistencyException e) {
+      logger.error("Can not remove node {}", node, e);
+      throw new TException(e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      logger.error("Can not remove node {}", node, e);
       throw new TException(e);
     }
 
@@ -190,8 +212,13 @@ public class MetaSyncService extends BaseSyncService implements TSMetaService.If
    * must tell it directly.
    */
   @Override
-  public void exile() {
-    metaGroupMember.applyRemoveNode(metaGroupMember.getThisNode());
+  public void exile(ByteBuffer removeNodeLogBuffer) {
+    logger.info("{}: start to exile.", name);
+    removeNodeLogBuffer.get();
+    RemoveNodeLog removeNodeLog = new RemoveNodeLog();
+    removeNodeLog.deserialize(removeNodeLogBuffer);
+    metaGroupMember.getPartitionTable().deserialize(removeNodeLog.getPartitionTable());
+    metaGroupMember.applyRemoveNode(removeNodeLog);
... 4325 lines suppressed ...