You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by hx...@apache.org on 2020/12/08 15:21:50 UTC

[iotdb] branch master updated: [IOTDB-68] New shared-nothing cluster (#460)

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

hxd 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 580b012  [IOTDB-68] New shared-nothing cluster (#460)
580b012 is described below

commit 580b0128d6eccbcdcb5cece3074b755b1af05c83
Author: Jialin Qiao <qj...@mails.tsinghua.edu.cn>
AuthorDate: Tue Dec 8 23:21:35 2020 +0800

    [IOTDB-68] New shared-nothing cluster (#460)
    
    Co-authored-by: HouliangQi <ne...@163.com>
    Co-authored-by: wangyanhong <11...@qq.com>
    Co-authored-by: chaow <ru...@foxmail.com>
    Co-authored-by: Jiang Tian <jt...@163.com>
    Co-authored-by: chaow <xu...@gmail.com>
    Co-authored-by: LebronAl <TX...@gmail.com>
    Co-authored-by: Ring-k <yu...@hotmail.com>
    Co-authored-by: xiangdong huang <sa...@gmail.com>
---
 .gitignore                                         |    6 +
 .../java/org/apache/iotdb/cli/AbstractScript.java  |    5 +
 cluster/pom.xml                                    |  183 ++
 cluster/src/assembly/cluster.xml                   |   27 +
 .../resources/conf/iotdb-cluster.properties        |  151 ++
 cluster/src/assembly/resources/sbin/add-node.bat   |  108 +
 cluster/src/assembly/resources/sbin/add-node.sh    |   76 +
 cluster/src/assembly/resources/sbin/nodetool.bat   |   58 +
 cluster/src/assembly/resources/sbin/nodetool.sh    |   48 +
 cluster/src/assembly/resources/sbin/start-node.bat |  129 ++
 cluster/src/assembly/resources/sbin/start-node.sh  |  100 +
 cluster/src/assembly/resources/sbin/stop-node.bat  |   27 +
 cluster/src/assembly/resources/sbin/stop-node.sh   |   39 +
 .../java/org/apache/iotdb/cluster/ClientMain.java  |  461 +++++
 .../iotdb/cluster/ClusterFileFlushPolicy.java      |   67 +
 .../java/org/apache/iotdb/cluster/ClusterMain.java |  319 +++
 .../apache/iotdb/cluster/RemoteTsFileResource.java |  174 ++
 .../iotdb/cluster/client/DataClientProvider.java   |   79 +
 .../cluster/client/async/AsyncClientFactory.java   |   62 +
 .../cluster/client/async/AsyncClientPool.java      |  234 +++
 .../cluster/client/async/AsyncDataClient.java      |  148 ++
 .../client/async/AsyncDataHeartbeatClient.java     |   69 +
 .../cluster/client/async/AsyncMetaClient.java      |  125 ++
 .../client/async/AsyncMetaHeartbeatClient.java     |   70 +
 .../cluster/client/sync/SyncClientAdaptor.java     |  456 ++++
 .../cluster/client/sync/SyncClientFactory.java     |   25 +-
 .../iotdb/cluster/client/sync/SyncClientPool.java  |  137 ++
 .../iotdb/cluster/client/sync/SyncDataClient.java  |  105 +
 .../client/sync/SyncDataHeartbeatClient.java       |   71 +
 .../iotdb/cluster/client/sync/SyncMetaClient.java  |   80 +
 .../client/sync/SyncMetaHeartbeatClient.java       |   71 +
 .../apache/iotdb/cluster/config/ClusterConfig.java |  444 ++++
 .../iotdb/cluster/config/ClusterConstant.java      |   28 +-
 .../iotdb/cluster/config/ClusterDescriptor.java    |  371 ++++
 .../iotdb/cluster/config/ConsistencyLevel.java     |   69 +
 .../iotdb/cluster/exception/AddSelfException.java  |   19 +-
 .../exception/BadSeedUrlFormatException.java       |   17 +-
 .../exception/CheckConsistencyException.java       |   22 +-
 .../exception/ConfigInconsistentException.java     |   17 +-
 .../cluster/exception/EmptyIntervalException.java  |   17 +-
 .../cluster/exception/EntryCompactedException.java |   18 +-
 .../exception/EntryUnavailableException.java       |   17 +-
 .../GetEntriesWrongParametersException.java        |   16 +-
 .../cluster/exception/LeaderUnknownException.java  |   20 +-
 .../cluster/exception/LogExecutionException.java   |   16 +-
 .../cluster/exception/MemberReadOnlyException.java |   19 +-
 .../cluster/exception/NoHeaderNodeException.java   |   20 +-
 .../cluster/exception/NotInSameGroupException.java |   20 +-
 .../cluster/exception/NotManagedSlotException.java |   19 +-
 .../PartitionTableUnavailableException.java        |   21 +-
 .../iotdb/cluster/exception/PullFileException.java |   15 +-
 .../cluster/exception/QueryTimeOutException.java   |   16 +-
 .../cluster/exception/ReaderNotFoundException.java |   16 +-
 .../cluster/exception/RequestTimeOutException.java |   18 +-
 .../exception/SnapshotInstallationException.java   |   22 +-
 .../exception/StartUpCheckFailureException.java    |   20 +-
 .../exception/TruncateCommittedEntryException.java |   18 +-
 .../cluster/exception/UnknownLogTypeException.java |   19 +-
 .../exception/UnsupportedPlanException.java        |   18 +-
 .../iotdb/cluster/log/CommitLogCallback.java       |   29 +-
 .../apache/iotdb/cluster/log/CommitLogTask.java    |   68 +
 .../org/apache/iotdb/cluster/log/HardState.java    |  133 ++
 .../java/org/apache/iotdb/cluster/log/Log.java     |  130 ++
 .../org/apache/iotdb/cluster/log/LogApplier.java   |   22 +-
 .../apache/iotdb/cluster/log/LogDispatcher.java    |  424 ++++
 .../org/apache/iotdb/cluster/log/LogParser.java    |  100 +
 .../org/apache/iotdb/cluster/log/Snapshot.java     |   71 +
 .../iotdb/cluster/log/StableEntryManager.java      |   55 +
 .../cluster/log/applier/AsyncDataLogApplier.java   |  270 +++
 .../iotdb/cluster/log/applier/BaseApplier.java     |  150 ++
 .../iotdb/cluster/log/applier/DataLogApplier.java  |  114 +
 .../iotdb/cluster/log/applier/MetaLogApplier.java  |   70 +
 .../iotdb/cluster/log/catchup/CatchUpTask.java     |  359 ++++
 .../iotdb/cluster/log/catchup/LogCatchUpTask.java  |  343 ++++
 .../cluster/log/catchup/SnapshotCatchUpTask.java   |  153 ++
 .../iotdb/cluster/log/logtypes/AddNodeLog.java     |   93 +
 .../iotdb/cluster/log/logtypes/CloseFileLog.java   |  119 ++
 .../cluster/log/logtypes/EmptyContentLog.java      |   63 +
 .../iotdb/cluster/log/logtypes/LargeTestLog.java   |   78 +
 .../cluster/log/logtypes/PhysicalPlanLog.java      |  112 +
 .../iotdb/cluster/log/logtypes/RemoveNodeLog.java  |   87 +
 .../cluster/log/manage/CommittedEntryManager.java  |  236 +++
 .../manage/FilePartitionedSnapshotLogManager.java  |  205 ++
 .../log/manage/MetaSingleSnapshotLogManager.java   |   84 +
 .../log/manage/PartitionedSnapshotLogManager.java  |  100 +
 .../iotdb/cluster/log/manage/RaftLogManager.java   |  949 +++++++++
 .../log/manage/UnCommittedEntryManager.java        |  259 +++
 .../log/manage/serializable/LogManagerMeta.java    |  136 ++
 .../serializable/SyncLogDequeSerializer.java       | 1329 ++++++++++++
 .../iotdb/cluster/log/snapshot/FileSnapshot.java   |  621 ++++++
 .../cluster/log/snapshot/MetaSimpleSnapshot.java   |  300 +++
 .../cluster/log/snapshot/PartitionedSnapshot.java  |  212 ++
 .../cluster/log/snapshot/PullSnapshotTask.java     |  231 +++
 .../log/snapshot/PullSnapshotTaskDescriptor.java   |  122 ++
 .../cluster/log/snapshot/SnapshotFactory.java      |   17 +-
 .../cluster/log/snapshot/SnapshotInstaller.java    |   22 +-
 .../log/snapshot/TimeseriesSchemaSnapshot.java     |   19 +-
 .../apache/iotdb/cluster/metadata/CMManager.java   | 1428 +++++++++++++
 .../apache/iotdb/cluster/metadata/MetaPuller.java  |  207 ++
 .../cluster/partition/NodeAdditionResult.java      |   25 +-
 .../iotdb/cluster/partition/NodeRemovalResult.java |   37 +-
 .../iotdb/cluster/partition/PartitionGroup.java    |   65 +
 .../iotdb/cluster/partition/PartitionTable.java    |  130 ++
 .../iotdb/cluster/partition/slot/SlotManager.java  |  315 +++
 .../partition/slot/SlotNodeAdditionResult.java     |   24 +-
 .../partition/slot/SlotNodeRemovalResult.java      |   27 +-
 .../cluster/partition/slot/SlotPartitionTable.java |  508 +++++
 .../iotdb/cluster/partition/slot/SlotStrategy.java |   36 +
 .../cluster/query/ClusterConcatPathOptimizer.java  |   26 +-
 .../cluster/query/ClusterDataQueryExecutor.java    |  103 +
 .../cluster/query/ClusterPhysicalGenerator.java    |  104 +
 .../iotdb/cluster/query/ClusterPlanExecutor.java   |  543 +++++
 .../iotdb/cluster/query/ClusterPlanRouter.java     |  365 ++++
 .../apache/iotdb/cluster/query/ClusterPlanner.java |   57 +
 .../iotdb/cluster/query/ClusterQueryRouter.java    |   88 +
 .../iotdb/cluster/query/LocalQueryExecutor.java    |  690 +++++++
 .../iotdb/cluster/query/RemoteQueryContext.java    |   72 +
 .../query/aggregate/ClusterAggregateExecutor.java  |   96 +
 .../cluster/query/aggregate/ClusterAggregator.java |  224 ++
 .../query/dataset/ClusterAlignByDeviceDataSet.java |   60 +
 .../cluster/query/fill/ClusterFillExecutor.java    |   61 +
 .../cluster/query/fill/ClusterLinearFill.java      |   70 +
 .../cluster/query/fill/ClusterPreviousFill.java    |  222 ++
 .../cluster/query/fill/PreviousFillArguments.java  |   67 +
 .../iotdb/cluster/query/filter/SlotSgFilter.java   |   34 +-
 .../cluster/query/filter/SlotTsFileFilter.java     |   57 +
 .../groupby/ClusterGroupByNoVFilterDataSet.java    |   59 +
 .../groupby/ClusterGroupByVFilterDataSet.java      |   74 +
 .../query/groupby/MergeGroupByExecutor.java        |  134 ++
 .../query/groupby/RemoteGroupByExecutor.java       |  144 ++
 .../query/last/ClusterLastQueryExecutor.java       |  249 +++
 .../cluster/query/manage/ClusterQueryManager.java  |  130 ++
 .../iotdb/cluster/query/manage/NodeStatus.java     |   87 +
 .../cluster/query/manage/QueryCoordinator.java     |  169 ++
 .../cluster/query/reader/ClusterReaderFactory.java |  571 +++++
 .../cluster/query/reader/ClusterTimeGenerator.java |   75 +
 .../iotdb/cluster/query/reader/DataSourceInfo.java |  249 +++
 .../iotdb/cluster/query/reader/EmptyReader.java    |  170 ++
 .../cluster/query/reader/ManagedMergeReader.java   |   93 +
 .../cluster/query/reader/MergedReaderByTime.java   |   33 +-
 .../reader/RemoteSeriesReaderByTimestamp.java      |  105 +
 .../query/reader/RemoteSimpleSeriesReader.java     |  159 ++
 .../apache/iotdb/cluster/server/ClientServer.java  |  322 +++
 .../iotdb/cluster/server/DataClusterServer.java    |  860 ++++++++
 .../iotdb/cluster/server/HardLinkCleaner.java      |  109 +
 .../iotdb/cluster/server/MetaClusterServer.java    |  346 ++++
 .../apache/iotdb/cluster/server/NodeCharacter.java |   19 +-
 .../apache/iotdb/cluster/server/NodeReport.java    |  183 ++
 .../java/org/apache/iotdb/cluster/server/Peer.java |   70 +
 .../cluster/server/PullSnapshotHintService.java    |  141 ++
 .../apache/iotdb/cluster/server/RaftServer.java    |  241 +++
 .../org/apache/iotdb/cluster/server/Response.java  |   59 +
 .../iotdb/cluster/server/StoppedMemberManager.java |  153 ++
 .../org/apache/iotdb/cluster/server/Timer.java     |  261 +++
 .../handlers/caller/AppendGroupEntryHandler.java   |  147 ++
 .../handlers/caller/AppendNodeEntryHandler.java    |  171 ++
 .../server/handlers/caller/ElectionHandler.java    |  127 ++
 .../server/handlers/caller/GenericHandler.java     |   91 +
 .../caller/GetChildNodeNextLevelPathHandler.java   |   57 +
 .../handlers/caller/GetNodesListHandler.java       |   58 +
 .../caller/GetTimeseriesSchemaHandler.java         |   57 +
 .../server/handlers/caller/HeartbeatHandler.java   |  132 ++
 .../server/handlers/caller/JoinClusterHandler.java |   63 +
 .../server/handlers/caller/LogCatchUpHandler.java  |  104 +
 .../handlers/caller/LogCatchUpInBatchHandler.java  |  104 +
 .../server/handlers/caller/NodeStatusHandler.java  |   54 +
 .../handlers/caller/PreviousFillHandler.java       |   78 +
 .../caller/PullMeasurementSchemaHandler.java       |   69 +
 .../handlers/caller/PullSnapshotHandler.java       |   78 +
 .../caller/PullTimeseriesSchemaHandler.java        |   69 +
 .../handlers/caller/SnapshotCatchUpHandler.java    |   62 +
 .../server/handlers/caller/package-info.java       |   15 +-
 .../handlers/forwarder/ForwardPlanHandler.java     |   66 +
 .../server/heartbeat/DataHeartbeatServer.java      |   82 +
 .../server/heartbeat/DataHeartbeatThread.java      |   74 +
 .../cluster/server/heartbeat/HeartbeatServer.java  |  212 ++
 .../cluster/server/heartbeat/HeartbeatThread.java  |  383 ++++
 .../server/heartbeat/MetaHeartbeatServer.java      |   83 +
 .../server/heartbeat/MetaHeartbeatThread.java      |   72 +
 .../cluster/server/member/DataGroupMember.java     |  828 ++++++++
 .../cluster/server/member/MetaGroupMember.java     | 2172 ++++++++++++++++++++
 .../iotdb/cluster/server/member/RaftMember.java    | 1869 +++++++++++++++++
 .../iotdb/cluster/server/member/package-info.java  |   17 +-
 .../cluster/server/service/BaseAsyncService.java   |  159 ++
 .../cluster/server/service/BaseSyncService.java    |  175 ++
 .../cluster/server/service/DataAsyncService.java   |  349 ++++
 .../cluster/server/service/DataSyncService.java    |  350 ++++
 .../cluster/server/service/MetaAsyncService.java   |  202 ++
 .../cluster/server/service/MetaSyncService.java    |  194 ++
 .../iotdb/cluster/server/service/package-info.java |   17 +-
 .../apache/iotdb/cluster/utils/ClientUtils.java    |   70 +
 .../iotdb/cluster/utils/ClusterConsistent.java     |   23 +-
 .../apache/iotdb/cluster/utils/ClusterNode.java    |   71 +
 .../iotdb/cluster/utils/ClusterQueryUtils.java     |   68 +
 .../apache/iotdb/cluster/utils/ClusterUtils.java   |  328 +++
 .../org/apache/iotdb/cluster/utils/IOUtils.java    |   87 +
 .../apache/iotdb/cluster/utils/PartitionUtils.java |  401 ++++
 .../apache/iotdb/cluster/utils/PlanSerializer.java |   51 +
 .../apache/iotdb/cluster/utils/StatusUtils.java    |  219 ++
 .../cluster/utils/nodetool/ClusterMonitor.java     |  179 ++
 .../utils/nodetool/ClusterMonitorMBean.java        |   84 +
 .../iotdb/cluster/utils/nodetool/NodeTool.java     |   47 +
 .../iotdb/cluster/utils/nodetool/Printer.java      |   24 +-
 .../cluster/utils/nodetool/function/Host.java      |   70 +
 .../cluster/utils/nodetool/function/LogView.java   |   82 +
 .../utils/nodetool/function/NodeToolCmd.java       |  102 +
 .../cluster/utils/nodetool/function/Partition.java |   85 +
 .../cluster/utils/nodetool/function/Ring.java      |   42 +
 .../cluster/utils/nodetool/function/Status.java    |   44 +
 .../cluster/client/DataClientProviderTest.java     |   73 +
 .../cluster/client/async/AsyncClientPoolTest.java  |  188 ++
 .../cluster/client/async/AsyncDataClientTest.java  |   61 +
 .../client/async/AsyncDataHeartbeatClientTest.java |   41 +
 .../cluster/client/async/AsyncMetaClientTest.java  |   61 +
 .../client/async/AsyncMetaHeartbeatClientTest.java |   41 +
 .../cluster/client/sync/SyncClientAdaptorTest.java |  366 ++++
 .../cluster/client/sync/SyncClientPoolTest.java    |  153 ++
 .../cluster/client/sync/SyncDataClientTest.java    |   80 +
 .../client/sync/SyncDataHeartbeatClientTest.java   |   63 +
 .../cluster/client/sync/SyncMetaClientTest.java    |   62 +
 .../client/sync/SyncMetaHeartbeatClientTest.java   |   63 +
 .../iotdb/cluster/common/EnvironmentUtils.java     |  218 ++
 .../org/apache/iotdb/cluster/common/IoTDBTest.java |  172 ++
 .../iotdb/cluster/common/TestAsyncClient.java      |   35 +-
 .../cluster/common/TestAsyncClientFactory.java     |   50 +
 .../iotdb/cluster/common/TestAsyncDataClient.java  |  190 ++
 .../iotdb/cluster/common/TestAsyncMetaClient.java  |   37 +-
 .../iotdb/cluster/common/TestDataGroupMember.java  |   46 +
 .../apache/iotdb/cluster/common/TestException.java |   16 +-
 .../org/apache/iotdb/cluster/common/TestLog.java   |   67 +
 .../iotdb/cluster/common/TestLogApplier.java       |   65 +
 .../iotdb/cluster/common/TestLogManager.java       |   28 +-
 .../cluster/common/TestManagedSeriesReader.java    |   91 +
 .../iotdb/cluster/common/TestMetaGroupMember.java  |   23 +-
 .../cluster/common/TestPartitionedLogManager.java  |   48 +
 .../apache/iotdb/cluster/common/TestSnapshot.java  |  110 +
 .../iotdb/cluster/common/TestSyncClient.java       |   29 +-
 .../cluster/common/TestSyncClientFactory.java      |   71 +
 .../org/apache/iotdb/cluster/common/TestUtils.java |  388 ++++
 .../cluster/integration/BaseSingleNodeTest.java    |   80 +
 .../iotdb/cluster/integration/SingleNodeTest.java  |   93 +
 .../iotdb/cluster/log/CommitLogCallbackTest.java   |   51 +
 .../iotdb/cluster/log/CommitLogTaskTest.java       |   72 +
 .../apache/iotdb/cluster/log/HardStateTest.java    |   36 +-
 .../iotdb/cluster/log/LogDispatcherTest.java       |  254 +++
 .../apache/iotdb/cluster/log/LogParserTest.java    |   99 +
 .../log/applier/AsyncDataLogApplierTest.java       |  166 ++
 .../cluster/log/applier/DataLogApplierTest.java    |  316 +++
 .../cluster/log/applier/MetaLogApplierTest.java    |  124 ++
 .../iotdb/cluster/log/catchup/CatchUpTaskTest.java |  393 ++++
 .../cluster/log/catchup/LogCatchUpTaskTest.java    |  289 +++
 .../log/catchup/SnapshotCatchUpTaskTest.java       |  270 +++
 .../cluster/log/logtypes/SerializeLogTest.java     |  128 ++
 .../log/manage/CommittedEntryManagerTest.java      |  429 ++++
 .../FilePartitionedSnapshotLogManagerTest.java     |  101 +
 .../manage/MetaSingleSnapshotLogManagerTest.java   |   91 +
 .../cluster/log/manage/RaftLogManagerTest.java     | 1412 +++++++++++++
 .../log/manage/UnCommittedEntryManagerTest.java    |  442 ++++
 .../serializable/SyncLogDequeSerializerTest.java   |  595 ++++++
 .../cluster/log/snapshot/DataSnapshotTest.java     |  163 ++
 .../cluster/log/snapshot/FileSnapshotTest.java     |  300 +++
 .../log/snapshot/MetaSimpleSnapshotTest.java       |  187 ++
 .../log/snapshot/PartitionedSnapshotTest.java      |  116 ++
 .../snapshot/PullSnapshotTaskDescriptorTest.java   |   62 +
 .../cluster/log/snapshot/PullSnapshotTaskTest.java |  299 +++
 .../iotdb/cluster/log/snapshot/SimpleSnapshot.java |  132 ++
 .../iotdb/cluster/partition/MManagerWhiteBox.java  |   53 +
 .../iotdb/cluster/partition/SlotManagerTest.java   |  149 ++
 .../cluster/partition/SlotPartitionTableTest.java  |  533 +++++
 .../apache/iotdb/cluster/query/BaseQueryTest.java  |  115 ++
 .../query/ClusterAggregateExecutorTest.java        |  129 ++
 .../query/ClusterDataQueryExecutorTest.java        |   66 +
 .../cluster/query/ClusterFillExecutorTest.java     |  115 ++
 .../query/ClusterPhysicalGeneratorTest.java        |   63 +
 .../cluster/query/ClusterPlanExecutorTest.java     |   83 +
 .../iotdb/cluster/query/ClusterPlannerTest.java    |   54 +
 .../cluster/query/ClusterQueryRouterTest.java      |  259 +++
 .../ClusterGroupByNoVFilterDataSetTest.java        |   83 +
 .../groupby/ClusterGroupByVFilterDataSetTest.java  |   94 +
 .../query/groupby/MergeGroupByExecutorTest.java    |   98 +
 .../query/groupby/RemoteGroupByExecutorTest.java   |  153 ++
 .../query/manage/ClusterQueryManagerTest.java      |  192 ++
 .../cluster/query/manage/QueryCoordinatorTest.java |  117 ++
 .../query/reader/ClusterTimeGeneratorTest.java     |   70 +
 .../cluster/query/reader/DatasourceInfoTest.java   |   77 +
 .../reader/RemoteSeriesReaderByTimestampTest.java  |  202 ++
 .../query/reader/RemoteSimpleSeriesReaderTest.java |  206 ++
 .../caller/AppendGroupEntryHandlerTest.java        |  159 ++
 .../caller/AppendNodeEntryHandlerTest.java         |  172 ++
 .../handlers/caller/ElectionHandlerTest.java       |  133 ++
 .../server/handlers/caller/GenericHandlerTest.java |   57 +
 .../handlers/caller/HeartbeatHandlerTest.java      |   99 +
 .../handlers/caller/JoinClusterHandlerTest.java    |   66 +
 .../handlers/caller/LogCatchUpHandlerTest.java     |  125 ++
 .../caller/PullMeasurementSchemaHandlerTest.java   |   89 +
 .../handlers/caller/PullSnapshotHandlerTest.java   |   81 +
 .../caller/SnapshotCatchUpHandlerTest.java         |   57 +
 .../handlers/forwarder/ForwardPlanHandlerTest.java |   57 +
 .../server/heartbeat/DataHeartbeatThreadTest.java  |  158 ++
 .../server/heartbeat/HeartbeatThreadTest.java      |  246 +++
 .../server/heartbeat/MetaHeartbeatThreadTest.java  |  214 ++
 .../cluster/server/member/DataGroupMemberTest.java | 1047 ++++++++++
 .../iotdb/cluster/server/member/MemberTest.java    |  418 ++++
 .../cluster/server/member/MetaGroupMemberTest.java | 1211 +++++++++++
 .../iotdb/cluster/utils/SerializeUtilTest.java     |  179 ++
 {server => cluster}/src/test/resources/logback.xml |   24 +-
 docs/UserGuide/Server/Cluster Setup.md             |   50 +-
 docs/UserGuide/Server/Config Manual.md             |   29 +
 docs/UserGuide/System Tools/NodeTool.md            |   77 +-
 docs/zh/UserGuide/Server/Cluster Setup.md          |   51 +-
 docs/zh/UserGuide/Server/Config Manual.md          |   38 +-
 docs/zh/UserGuide/System Tools/NodeTool.md         |   84 +-
 .../org/apache/iotdb/jdbc/IoTDBConnection.java     |   16 +-
 .../java/org/apache/iotdb/jdbc/IoTDBStatement.java |   54 +-
 pom.xml                                            |    1 +
 .../resources/conf/iotdb-engine.properties         |    9 +
 server/src/assembly/resources/conf/logback.xml     |    6 +-
 .../java/org/apache/iotdb/db/auth/entity/Role.java |    8 +
 .../java/org/apache/iotdb/db/auth/entity/User.java |   12 +
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |   17 +-
 .../org/apache/iotdb/db/conf/IoTDBConstant.java    |    3 +
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |    8 +
 .../org/apache/iotdb/db/engine/StorageEngine.java  |   77 +-
 .../no/NoCompactionTsFileManagement.java           |    2 +-
 .../engine/compaction/utils/CompactionUtils.java   |    5 +-
 .../iotdb/db/engine/flush/MemTableFlushTask.java   |    4 +-
 .../db/engine/memtable/PrimitiveMemTable.java      |    4 +
 .../iotdb/db/engine/merge/task/MergeFileTask.java  |   32 +-
 .../engine/storagegroup/StorageGroupProcessor.java |  229 +--
 .../db/engine/storagegroup/TsFileProcessor.java    |  162 +-
 .../db/engine/storagegroup/TsFileResource.java     |  174 +-
 ...onException.java => BatchProcessException.java} |   13 +-
 .../db/exception/TsFileProcessorException.java     |    2 +-
 ...eryProcessException.java => PathException.java} |   18 +-
 .../db/exception/query/QueryProcessException.java  |    4 +
 .../org/apache/iotdb/db/metadata/MManager.java     |  217 +-
 .../java/org/apache/iotdb/db/metadata/MTree.java   |   10 +-
 .../org/apache/iotdb/db/metadata/MetaUtils.java    |   10 +-
 .../org/apache/iotdb/db/metadata/PartialPath.java  |   40 +-
 .../org/apache/iotdb/db/monitor/StatMonitor.java   |    3 +-
 .../apache/iotdb/db/qp/executor/IPlanExecutor.java |    4 +-
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  264 +--
 .../apache/iotdb/db/qp/physical/PhysicalPlan.java  |   31 +-
 .../iotdb/db/qp/physical/crud/InsertPlan.java      |   22 +-
 .../iotdb/db/qp/physical/crud/InsertRowPlan.java   |  119 +-
 .../db/qp/physical/crud/InsertTabletPlan.java      |   65 +-
 .../db/qp/physical/crud/RawDataQueryPlan.java      |    3 +
 .../qp/physical/sys/CreateMultiTimeSeriesPlan.java |  117 +-
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |   19 +-
 .../qp/strategy/optimizer/ConcatPathOptimizer.java |    4 +-
 .../db/query/dataset/ShowTimeseriesDataSet.java    |   13 +-
 .../dataset/groupby/GroupByEngineDataSet.java      |    7 +-
 .../query/dataset/groupby/GroupByFillDataSet.java  |    2 +-
 .../groupby/GroupByWithValueFilterDataSet.java     |    5 +
 .../db/query/executor/AggregationExecutor.java     |    1 +
 .../iotdb/db/query/executor/LastQueryExecutor.java |   34 +-
 .../db/query/executor/RawDataQueryExecutor.java    |    2 +-
 .../iotdb/db/query/reader/chunk/MemPageReader.java |    7 +-
 .../reader/series/BaseManagedSeriesReader.java}    |   29 +-
 .../reader/series/SeriesReaderByTimestamp.java     |    4 +
 .../org/apache/iotdb/db/service/TSServiceImpl.java |  193 +-
 .../db/service/thrift/ThriftServiceThread.java     |   31 +-
 .../iotdb/db/sync/sender/transfer/SyncClient.java  |    6 +-
 .../iotdb/db/tools/TsFileResourcePrinter.java      |   13 +-
 .../org/apache/iotdb/db/utils/CommonUtils.java     |   60 +-
 .../org/apache/iotdb/db/utils/FileLoaderUtils.java |    2 +
 .../org/apache/iotdb/db/utils/SchemaUtils.java     |   20 +-
 .../org/apache/iotdb/db/utils/SerializeUtils.java  |  280 ++-
 .../org/apache/iotdb/db/writelog/io/LogWriter.java |    9 +-
 .../db/writelog/node/ExclusiveWriteLogNode.java    |  154 +-
 .../writelog/recover/TsFileRecoverPerformer.java   |   24 +-
 .../iotdb/db/engine/merge/MergeOverLapTest.java    |    9 +-
 .../apache/iotdb/db/engine/merge/MergeTest.java    |    9 +-
 .../storagegroup/StorageGroupProcessorTest.java    |   13 +-
 .../engine/storagegroup/TsFileProcessorTest.java   |    8 +-
 .../iotdb/db/integration/IoTDBQuotedPathIT.java    |    4 +-
 .../db/integration/IoTDBRpcCompressionIT.java      |  153 ++
 .../iotdb/db/integration/IoTDBSimpleQueryIT.java   |    6 +-
 .../iotdb/db/metadata/MManagerBasicTest.java       |    2 +-
 .../org/apache/iotdb/db/metadata/MTreeTest.java    |   13 +-
 .../apache/iotdb/db/metadata/MetaUtilsTest.java    |    6 +-
 .../reader/series/SeriesAggregateReaderTest.java   |    3 +-
 .../reader/series/SeriesReaderByTimestampTest.java |    3 +-
 .../db/query/reader/series/SeriesReaderTest.java   |    3 +-
 .../query/reader/series/SeriesReaderTestUtil.java  |   16 +-
 .../db/sync/receiver/load/FileLoaderTest.java      |    7 +-
 .../apache/iotdb/db/writelog/WriteLogNodeTest.java |   11 +-
 .../db/writelog/recover/SeqTsFileRecoverTest.java  |    3 -
 server/src/test/resources/logback.xml              |    2 +-
 .../org/apache/iotdb/rpc/AutoResizingBuffer.java   |   80 +
 .../iotdb/rpc/AutoScalingBufferReadTransport.java  |   79 +
 .../iotdb/rpc/AutoScalingBufferWriteTransport.java |   63 +
 .../apache/iotdb/rpc/IoTDBConnectionException.java |    2 +-
 ...nectionException.java => NonOpenTransport.java} |   27 +-
 ...cutionException.java => RedirectException.java} |   33 +-
 ...atementExecutionException.java => RpcStat.java} |   33 +-
 .../org/apache/iotdb/rpc/RpcTransportFactory.java  |   56 +
 .../main/java/org/apache/iotdb/rpc/RpcUtils.java   |   66 +-
 .../iotdb/rpc/StatementExecutionException.java     |    2 +-
 .../rpc/TCompressedElasticFramedTransport.java     |  113 +
 .../apache/iotdb/rpc/TElasticFramedTransport.java  |  145 ++
 .../java/org/apache/iotdb/rpc/TSStatusCode.java    |    3 +
 .../iotdb/rpc/TSnappyElasticFramedTransport.java   |   80 +
 .../rpc/TimeoutChangeableTFastFramedTransport.java |   55 +
 .../TimeoutChangeableTSnappyFramedTransport.java   |   54 +
 ...eption.java => TimeoutChangeableTransport.java} |   19 +-
 .../main/java/org/apache/iotdb/session/Config.java |    2 +
 .../java/org/apache/iotdb/session/Session.java     | 1343 ++++++------
 .../apache/iotdb/session/SessionConnection.java    |  624 ++++++
 .../java/org/apache/iotdb/session/SessionUT.java   |    4 +-
 thrift/src/main/thrift/cluster.thrift              |   39 +-
 thrift/src/main/thrift/rpc.thrift                  |    6 +
 tsfile/format-changelist.md                        |    5 +
 .../iotdb/tsfile/file/footer/ChunkGroupFooter.java |   47 +-
 .../iotdb/tsfile/read/TsFileSequenceReader.java    |   12 +
 .../tsfile/read/query/dataset/QueryDataSet.java    |   14 +-
 .../iotdb/tsfile/read/reader/page/PageReader.java  |    7 +-
 .../iotdb/tsfile/utils/ReadWriteIOUtils.java       |  116 +-
 .../write/writer/RestorableTsFileIOWriter.java     |   15 +
 .../iotdb/tsfile/write/writer/TsFileIOWriter.java  |   21 +-
 420 files changed, 55786 insertions(+), 2672 deletions(-)

diff --git a/.gitignore b/.gitignore
index d10c08b..a136207 100644
--- a/.gitignore
+++ b/.gitignore
@@ -3,6 +3,7 @@
 **/logs/*
 **/lib/**
 **/data/**
+**/raft/**
 
 # Python runtime file
 **/__pycache__/**
@@ -87,9 +88,14 @@ partitions.tmp
 partitions
 node_identifier
 
+### temporary file of the distributed version ###
+remote/
+
 # gitpod
 .theia/
 
+classes/
+
 ### Cmake files ###
 *.cmake
 Makefile
diff --git a/cli/src/test/java/org/apache/iotdb/cli/AbstractScript.java b/cli/src/test/java/org/apache/iotdb/cli/AbstractScript.java
index b938efb..56810cb 100644
--- a/cli/src/test/java/org/apache/iotdb/cli/AbstractScript.java
+++ b/cli/src/test/java/org/apache/iotdb/cli/AbstractScript.java
@@ -48,6 +48,11 @@ public abstract class AbstractScript {
     r.close();
     p.destroy();
 
+    System.out.println("Process output:");
+    for (String s : outputList) {
+      System.out.println(s);
+    }
+
     for (int i = 0; i < output.length; i++) {
       assertEquals(output[output.length - 1 - i], outputList.get(outputList.size() - 1 - i));
     }
diff --git a/cluster/pom.xml b/cluster/pom.xml
new file mode 100644
index 0000000..876afa4
--- /dev/null
+++ b/cluster/pom.xml
@@ -0,0 +1,183 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    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.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>iotdb-parent</artifactId>
+        <groupId>org.apache.iotdb</groupId>
+        <version>0.12.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>cluster</artifactId>
+    <name>cluster</name>
+    <properties>
+        <cluster.test.skip>false</cluster.test.skip>
+        <cluster.it.skip>${cluster.test.skip}</cluster.it.skip>
+        <cluster.ut.skip>${cluster.test.skip}</cluster.ut.skip>
+    </properties>
+    <repositories>
+        <!--   repository for moquette    -->
+        <repository>
+            <id>bintray</id>
+            <url>https://jcenter.bintray.com</url>
+            <releases>
+                <enabled>true</enabled>
+            </releases>
+            <snapshots>
+                <enabled>false</enabled>
+            </snapshots>
+        </repository>
+    </repositories>
+    <dependencies>
+        <!-- The version of thrift is overridden because using 0.13.0 in the cluster module
+        will cause unclear bugs -->
+        <dependency>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libthrift</artifactId>
+            <version>0.12.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>service-rpc</artifactId>
+            <version>0.12.0-SNAPSHOT</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.thrift</groupId>
+                    <artifactId>libthrift</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-server</artifactId>
+            <version>0.12.0-SNAPSHOT</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.thrift</groupId>
+                    <artifactId>libthrift</artifactId>
+                </exclusion>
+            </exclusions>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-session</artifactId>
+            <version>0.12.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-jdbc</artifactId>
+            <version>0.12.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>commons-cli</groupId>
+            <artifactId>commons-cli</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.awaitility</groupId>
+            <artifactId>awaitility</artifactId>
+            <version>4.0.2</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <profiles>
+        <profile>
+            <id>skipClusterTests</id>
+            <activation>
+                <property>
+                    <name>skipTests</name>
+                    <value>true</value>
+                </property>
+            </activation>
+            <properties>
+                <cluster.test.skip>true</cluster.test.skip>
+                <cluster.ut.skip>true</cluster.ut.skip>
+                <cluster.it.skip>true</cluster.it.skip>
+            </properties>
+        </profile>
+        <profile>
+            <id>skipUT_Cluster_Tests</id>
+            <activation>
+                <property>
+                    <name>skipUTs</name>
+                    <value>true</value>
+                </property>
+            </activation>
+            <properties>
+                <cluster.ut.skip>true</cluster.ut.skip>
+            </properties>
+        </profile>
+    </profiles>
+    <build>
+        <plugins>
+            <!--using `mvn test` to run UT, `mvn verify` to run ITs
+                        Reference: https://antoniogoncalves.org/2012/12/13/lets-turn-integration-tests-with-maven-to-a-first-class-citizen/-->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <configuration>
+                    <skipTests>${cluster.ut.skip}</skipTests>
+                    <forkMode>pertest</forkMode>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-failsafe-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>run-integration-tests</id>
+                        <phase>integration-test</phase>
+                        <goals>
+                            <goal>integration-test</goal>
+                            <goal>verify</goal>
+                        </goals>
+                    </execution>
+                </executions>
+                <configuration>
+                    <skipTests>${cluster.test.skip}</skipTests>
+                    <skipITs>${cluster.it.skip}</skipITs>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <executions>
+                    <!-- Package binaries-->
+                    <execution>
+                        <id>cluster-assembly</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                        <configuration>
+                            <descriptors>
+                                <descriptor>src/assembly/cluster.xml</descriptor>
+                            </descriptors>
+                            <appendAssemblyId>false</appendAssemblyId>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>
diff --git a/cluster/src/assembly/cluster.xml b/cluster/src/assembly/cluster.xml
new file mode 100644
index 0000000..5c0c581
--- /dev/null
+++ b/cluster/src/assembly/cluster.xml
@@ -0,0 +1,27 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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 a [...]
+  -->
+<assembly>
+    <id>cluster</id>
+    <formats>
+        <format>dir</format>
+        <format>zip</format>
+    </formats>
+    <includeBaseDirectory>false</includeBaseDirectory>
+    <dependencySets>
+        <dependencySet>
+            <outputDirectory>lib</outputDirectory>
+        </dependencySet>
+    </dependencySets>
+    <fileSets>
+        <fileSet>
+            <directory>src/assembly/resources</directory>
+            <outputDirectory>/</outputDirectory>
+        </fileSet>
+        <fileSet>
+            <directory>${maven.multiModuleProjectDirectory}/server/src/assembly/resources</directory>
+            <outputDirectory>/</outputDirectory>
+        </fileSet>
+    </fileSets>
+</assembly>
diff --git a/cluster/src/assembly/resources/conf/iotdb-cluster.properties b/cluster/src/assembly/resources/conf/iotdb-cluster.properties
new file mode 100644
index 0000000..640cfba
--- /dev/null
+++ b/cluster/src/assembly/resources/conf/iotdb-cluster.properties
@@ -0,0 +1,151 @@
+#
+# 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.
+#
+
+cluster_rpc_ip=127.0.0.1
+
+#-------------------------------------------IMPORTANT---------------------------------------------#
+# Note that the system will automatically create a heartbeat port for each metadata service       #
+# and data service. The default metadata heartbeat port is internal_meta_port + 1,                #
+# The default data heartbeat port is internal_data_port + 1.                                      #
+# So when you configure these two items and seed_nodes, pay attention to reserve a port for       #
+# heartbeat service.                                                                              #
+#-------------------------------------------IMPORTANT---------------------------------------------#
+
+# port for metadata service
+internal_meta_port=9003
+
+# port for data service
+internal_data_port=40010
+
+# port for client service
+cluster_rpc_port=55560
+
+# comma-separated {IP/DOMAIN}:meta_port:data_port:client_port pairs
+# when used by start-node.sh(.bat), this configuration means the nodes that will form the initial
+# cluster, some every node that use start-node.sh(.bat) should have the SAME SEED_NODES, or the
+# building of the initial cluster will fail. WARNING: if the initial cluster is built, this
+# should not be changed before the environment is cleaned.
+# when used by add-node.sh(.bat), this means the nodes to which that the application of joining
+# the cluster will be sent, as all nodes can respond to a request, this configuration can be any
+# nodes that already in the cluster, unnecessary to be the nodes that were used to build the
+# initial cluster by start-node.sh(.bat). Several nodes will be picked randomly to send the
+# request, the number of nodes picked depends on the number of retries.
+seed_nodes=127.0.0.1:9003:40010:55560,127.0.0.1:9005:40012:55561,127.0.0.1:9007:40014:55562
+
+# whether to use thrift compressed protocol for internal communications. If you want to change
+# compression settings for external clients, please modify 'rpc_thrift_compression_enable' in
+# 'iotdb-engine.properties'.
+# WARNING: this must be consistent across all nodes in the cluster
+rpc_thrift_compression_enable=false
+
+# max client connections created by thrift
+# this configuration applies separately to data/meta/client connections and thus does not control
+# the number of global connections
+max_concurrent_client_num=10000
+
+# number of replications for one partition
+default_replica_num=2
+
+# cluster name to identify different clusters
+# all node's cluster_name in one cluster are the same
+cluster_name=default
+
+# connection time out (ms) among raft nodes
+connection_timeout_ms=20000
+
+# write operation timeout threshold (ms), this is only for internal communications,
+# not for the whole operation.
+write_operation_timeout_ms=30000
+
+# read operation timeout threshold (ms), this is only for internal communications,
+# not for the whole operation.
+read_operation_timeout_ms=30000
+
+# catch up timeout threshold (ms), this is used for a follower behind the leader too much,
+# so the leader will send logs(snapshot) to the follower,
+# NOTICE, it may cost minutes of time to send a snapshot,
+# so this parameter should be larger than the snapshot cost time.
+catch_up_timeout_ms=300000
+
+# whether to use batch append entries in log catch up
+use_batch_in_catch_up=true
+
+# the minimum number of committed logs in memory, after each log deletion, at most such number of logs
+# will remain in memory. Increasing the number will reduce the chance to use snapshot in catch-ups,
+# but will also increase the memory footprint
+min_num_of_logs_in_mem=1000
+
+# maximum number of committed logs in memory, when reached, a log deletion will be triggered.
+# Increasing the number will reduce the chance to use snapshot in catch-ups, but will also increase
+# memory footprint
+max_num_of_logs_in_mem=2000
+
+# deletion check period of the submitted log
+log_deletion_check_interval_second=-1
+
+# Whether creating schema automatically is enabled, this will replace the one in iotdb-engine.properties
+enable_auto_create_schema=true
+
+# consistency level, now three consistency levels are supported: strong, mid, and weak.
+# Strong consistency means the server will first try to synchronize with the leader to get the
+# newest data, if failed(timeout), directly report an error to the user;
+# While mid consistency means the server will first try to synchronize with the leader,
+# but if failed(timeout), it will give up and just use current data it has cached before;
+# Weak consistency does not synchronize with the leader and simply use the local data
+consistency_level=mid
+
+# Whether to use asynchronous server
+is_use_async_server=false
+
+# Whether to use asynchronous applier
+is_use_async_applier=true
+
+# is raft log persistence enabled
+is_enable_raft_log_persistence=true
+
+# When a certain amount of raft log is reached, it will be flushed to disk
+# It is possible to lose at most flush_raft_log_threshold operations
+flush_raft_log_threshold=10000
+
+# Size of log buffer in each RaftMember's LogManager(in byte).
+raft_log_buffer_size=16777216
+
+# The maximum value of the raft log index stored in the memory per raft group,
+# These indexes are used to index the location of the log on the disk
+max_raft_log_index_size_in_memory=10000
+
+# The maximum size of the raft log saved on disk for each file (in bytes) of each raft group.
+# The default size is 1GB
+max_raft_log_persist_data_size_per_file=1073741824
+
+# The maximum number of persistent raft log files on disk per raft group,
+# So each raft group's log takes up disk space approximately equals
+# max_raft_log_persist_data_size_per_file*max_number_of_persist_raft_log_files
+max_number_of_persist_raft_log_files=5
+
+# The maximum number of logs saved on the disk
+max_persist_raft_log_number_on_disk=1000000
+
+# whether enable use persist log on disk to catch up when no logs found in memory, if set false,
+# will use snapshot to catch up when no logs found in memory.
+enable_use_persist_log_on_disk_to_catch_up=true
+
+# The number of logs read on the disk at one time, which is mainly used to control the memory usage.
+# This value multiplied by the log size is about the amount of memory used to read logs from the disk at one time.
+max_number_of_logs_per_fetch_on_disk=1000
\ No newline at end of file
diff --git a/cluster/src/assembly/resources/sbin/add-node.bat b/cluster/src/assembly/resources/sbin/add-node.bat
new file mode 100755
index 0000000..cdaf94a
--- /dev/null
+++ b/cluster/src/assembly/resources/sbin/add-node.bat
@@ -0,0 +1,108 @@
+@REM
+@REM Licensed to the Apache Software Foundation (ASF) under one
+@REM or more contributor license agreements.  See the NOTICE file
+@REM distributed with this work for additional information
+@REM regarding copyright ownership.  The ASF licenses this file
+@REM to you under the Apache License, Version 2.0 (the
+@REM "License"); you may not use this file except in compliance
+@REM with the License.  You may obtain a copy of the License at
+@REM
+@REM     http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM Unless required by applicable law or agreed to in writing,
+@REM software distributed under the License is distributed on an
+@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+@REM KIND, either express or implied.  See the License for the
+@REM specific language governing permissions and limitations
+@REM under the License.
+@REM
+
+@echo off
+echo ````````````````````````
+echo Starting IoTDB
+echo ````````````````````````
+
+PATH %PATH%;%JAVA_HOME%\bin\
+set "FULL_VERSION="
+set "MAJOR_VERSION="
+set "MINOR_VERSION="
+
+
+for /f tokens^=2-5^ delims^=.-_+^" %%j in ('java -fullversion 2^>^&1') do (
+	set "FULL_VERSION=%%j-%%k-%%l-%%m"
+	IF "%%j" == "1" (
+	    set "MAJOR_VERSION=%%k"
+	    set "MINOR_VERSION=%%l"
+	) else (
+	    set "MAJOR_VERSION=%%j"
+	    set "MINOR_VERSION=%%k"
+	)
+)
+
+set JAVA_VERSION=%MAJOR_VERSION%
+
+IF NOT %JAVA_VERSION% == 8 (
+	IF NOT %JAVA_VERSION% == 11 (
+		echo IoTDB only supports jdk8 or jdk11, please check your java version.
+		goto finally
+	)
+)
+
+if "%OS%" == "Windows_NT" setlocal
+
+pushd %~dp0..
+if NOT DEFINED IOTDB_HOME set IOTDB_HOME=%cd%
+popd
+
+set IOTDB_CONF=%IOTDB_HOME%\conf
+set IOTDB_LOGS=%IOTDB_HOME%\logs
+
+IF EXIST "%IOTDB_CONF%\cluster-env.bat" (
+    CALL "%IOTDB_CONF%\cluster-env.bat"
+    ) ELSE (
+    echo "can't find %IOTDB_CONF%\cluster-env.bat"
+    )
+
+if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.cluster.ClusterMain
+if NOT DEFINED JAVA_HOME goto :err
+
+@REM -----------------------------------------------------------------------------
+@REM JVM Opts we'll use in legacy run or installation
+set JAVA_OPTS=-ea^
+ -Dlogback.configurationFile="%IOTDB_CONF%\logback.xml"^
+ -DIOTDB_HOME="%IOTDB_HOME%"^
+ -DTSFILE_HOME="%IOTDB_HOME%"^
+ -DIOTDB_CONF="%IOTDB_CONF%"
+
+@REM ***** CLASSPATH library setting *****
+@REM Ensure that any user defined CLASSPATH variables are not used on startup
+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
+goto okClasspath
+
+:append
+set CLASSPATH=%CLASSPATH%;%1
+goto :eof
+
+@REM -----------------------------------------------------------------------------
+:okClasspath
+
+rem echo CLASSPATH: %CLASSPATH%
+
+"%JAVA_HOME%\bin\java" %JAVA_OPTS% %IOTDB_HEAP_OPTS% -cp %CLASSPATH% %IOTDB_JMX_OPTS% %MAIN_CLASS% -a
+goto finally
+
+:err
+echo JAVA_HOME environment variable must be set!
+pause
+
+
+@REM -----------------------------------------------------------------------------
+:finally
+
+pause
+
+ENDLOCAL
\ No newline at end of file
diff --git a/cluster/src/assembly/resources/sbin/add-node.sh b/cluster/src/assembly/resources/sbin/add-node.sh
new file mode 100755
index 0000000..46d9edc
--- /dev/null
+++ b/cluster/src/assembly/resources/sbin/add-node.sh
@@ -0,0 +1,76 @@
+#!/bin/bash
+#
+# 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.
+#
+
+
+echo ---------------------
+echo Starting IoTDB
+echo ---------------------
+
+if [ -z "${IOTDB_HOME}" ]; then
+  export IOTDB_HOME="`dirname "$0"`/.."
+fi
+
+IOTDB_CONF=${IOTDB_HOME}/conf
+# IOTDB_LOGS=${IOTDB_HOME}/logs
+
+if [ -f "$IOTDB_CONF/cluster-env.sh" ]; then
+    . "$IOTDB_CONF/cluster-env.sh"
+else
+    echo "can't find $IOTDB_CONF/cluster-env.sh"
+fi
+
+if [ -n "$JAVA_HOME" ]; then
+    for java in "$JAVA_HOME"/bin/amd64/java "$JAVA_HOME"/bin/java; do
+        if [ -x "$java" ]; then
+            JAVA="$java"
+            break
+        fi
+    done
+else
+    JAVA=java
+fi
+
+if [ -z $JAVA ] ; then
+    echo Unable to find java executable. Check JAVA_HOME and PATH environment variables.  > /dev/stderr
+    exit 1;
+fi
+
+CLASSPATH=""
+for f in ${IOTDB_HOME}/lib/*.jar; do
+  CLASSPATH=${CLASSPATH}":"$f
+done
+classname=org.apache.iotdb.cluster.ClusterMain
+
+launch_service()
+{
+	class="$1"
+	iotdb_parms="-Dlogback.configurationFile=${IOTDB_CONF}/logback.xml"
+	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 -Dname=iotdb\.IoTDB"
+	exec "$JAVA" $iotdb_parms $IOTDB_JMX_OPTS $iotdb_parms -cp "$CLASSPATH"  "$class" -a
+	return $?
+}
+
+# Start up the service
+launch_service "$classname"
+
+exit $?
diff --git a/cluster/src/assembly/resources/sbin/nodetool.bat b/cluster/src/assembly/resources/sbin/nodetool.bat
new file mode 100755
index 0000000..7dd9111
--- /dev/null
+++ b/cluster/src/assembly/resources/sbin/nodetool.bat
@@ -0,0 +1,58 @@
+@REM
+@REM Licensed to the Apache Software Foundation (ASF) under one
+@REM or more contributor license agreements.  See the NOTICE file
+@REM distributed with this work for additional information
+@REM regarding copyright ownership.  The ASF licenses this file
+@REM to you under the Apache License, Version 2.0 (the
+@REM "License"); you may not use this file except in compliance
+@REM with the License.  You may obtain a copy of the License at
+@REM
+@REM     http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM Unless required by applicable law or agreed to in writing,
+@REM software distributed under the License is distributed on an
+@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+@REM KIND, either express or implied.  See the License for the
+@REM specific language governing permissions and limitations
+@REM under the License.
+@REM
+
+if "%OS%" == "Windows_NT" setlocal
+
+pushd %~dp0..
+if NOT DEFINED IOTDB_HOME set IOTDB_HOME=%CD%
+popd
+
+if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.cluster.utils.nodetool.NodeTool
+if NOT DEFINED JAVA_HOME goto :err
+
+@REM -----------------------------------------------------------------------------
+@REM JVM Opts we'll use in legacy run or installation
+set JAVA_OPTS=-ea^
+ -DIOTDB_HOME=%IOTDB_HOME%
+
+REM For each jar in the IOTDB_HOME lib directory call append to build the CLASSPATH variable.
+for %%i in ("%IOTDB_HOME%\lib\*.jar") do call :append "%%i"
+goto okClasspath
+
+:append
+set CLASSPATH=%CLASSPATH%;%1
+goto :eof
+
+REM -----------------------------------------------------------------------------
+:okClasspath
+
+"%JAVA_HOME%\bin\java" %JAVA_OPTS% -cp "%CLASSPATH%" %MAIN_CLASS% %*
+
+goto finally
+
+
+:err
+echo JAVA_HOME environment variable must be set!
+pause
+
+
+@REM -----------------------------------------------------------------------------
+:finally
+
+ENDLOCAL
\ No newline at end of file
diff --git a/cluster/src/assembly/resources/sbin/nodetool.sh b/cluster/src/assembly/resources/sbin/nodetool.sh
new file mode 100755
index 0000000..ab005a7
--- /dev/null
+++ b/cluster/src/assembly/resources/sbin/nodetool.sh
@@ -0,0 +1,48 @@
+#!/bin/sh
+#
+# 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.
+#
+
+if [ -z "${IOTDB_HOME}" ]; then
+  export IOTDB_HOME="$(cd "`dirname "$0"`"/..; pwd)"
+fi
+
+
+MAIN_CLASS=org.apache.iotdb.cluster.utils.nodetool.NodeTool
+
+
+CLASSPATH=""
+for f in ${IOTDB_HOME}/lib/*.jar; do
+  CLASSPATH=${CLASSPATH}":"$f
+done
+
+
+if [ -n "$JAVA_HOME" ]; then
+    for java in "$JAVA_HOME"/bin/amd64/java "$JAVA_HOME"/bin/java; do
+        if [ -x "$java" ]; then
+            JAVA="$java"
+            break
+        fi
+    done
+else
+    JAVA=java
+fi
+
+exec "$JAVA" -cp "$CLASSPATH" "$MAIN_CLASS" "$@"
+
+exit $?
diff --git a/cluster/src/assembly/resources/sbin/start-node.bat b/cluster/src/assembly/resources/sbin/start-node.bat
new file mode 100755
index 0000000..006567f
--- /dev/null
+++ b/cluster/src/assembly/resources/sbin/start-node.bat
@@ -0,0 +1,129 @@
+@REM
+@REM Licensed to the Apache Software Foundation (ASF) under one
+@REM or more contributor license agreements.  See the NOTICE file
+@REM distributed with this work for additional information
+@REM regarding copyright ownership.  The ASF licenses this file
+@REM to you under the Apache License, Version 2.0 (the
+@REM "License"); you may not use this file except in compliance
+@REM with the License.  You may obtain a copy of the License at
+@REM
+@REM     http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM Unless required by applicable law or agreed to in writing,
+@REM software distributed under the License is distributed on an
+@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+@REM KIND, either express or implied.  See the License for the
+@REM specific language governing permissions and limitations
+@REM under the License.
+@REM
+
+@echo off
+echo ````````````````````````
+echo Starting IoTDB
+echo ````````````````````````
+
+PATH %PATH%;%JAVA_HOME%\bin\
+set "FULL_VERSION="
+set "MAJOR_VERSION="
+set "MINOR_VERSION="
+
+
+for /f tokens^=2-5^ delims^=.-_+^" %%j in ('java -fullversion 2^>^&1') do (
+	set "FULL_VERSION=%%j-%%k-%%l-%%m"
+	IF "%%j" == "1" (
+	    set "MAJOR_VERSION=%%k"
+	    set "MINOR_VERSION=%%l"
+	) else (
+	    set "MAJOR_VERSION=%%j"
+	    set "MINOR_VERSION=%%k"
+	)
+)
+
+set JAVA_VERSION=%MAJOR_VERSION%
+
+IF NOT %JAVA_VERSION% == 8 (
+	IF NOT %JAVA_VERSION% == 11 (
+		echo IoTDB only supports jdk8 or jdk11, please check your java version.
+		goto finally
+	)
+)
+
+if "%OS%" == "Windows_NT" setlocal
+
+pushd %~dp0..
+if NOT DEFINED IOTDB_HOME set IOTDB_HOME=%cd%
+popd
+
+set IOTDB_CONF=%IOTDB_HOME%\conf
+set IOTDB_LOGS=%IOTDB_HOME%\logs
+
+
+IF EXIST "%IOTDB_CONF%\cluster-env.bat" (
+    IF "%1" == "printgc" (
+      CALL "%IOTDB_CONF%\cluster-env.bat" printgc
+      SHIFT
+    ) ELSE (
+      CALL "%IOTDB_CONF%\cluster-env.bat"
+    )
+) ELSE (
+    echo "can't find %IOTDB_CONF%\cluster-env.bat"
+)
+
+@setlocal ENABLEDELAYEDEXPANSION ENABLEEXTENSIONS
+set CONF_PARAMS=-s
+set is_conf_path=false
+for %%i in (%*) do (
+	IF "%%i" == "-c" (
+		set is_conf_path=true
+	) ELSE IF "!is_conf_path!" == "true" (
+		set is_conf_path=false
+		set IOTDB_CONF=%%i
+	) ELSE (
+		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
+
+@REM -----------------------------------------------------------------------------
+@REM JVM Opts we'll use in legacy run or installation
+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 *****
+@REM Ensure that any user defined CLASSPATH variables are not used on startup
+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.ClusterMain
+goto okClasspath
+
+:append
+set CLASSPATH=%CLASSPATH%;%1
+goto :eof
+
+@REM -----------------------------------------------------------------------------
+:okClasspath
+
+rem echo CLASSPATH: %CLASSPATH%
+
+"%JAVA_HOME%\bin\java" %JAVA_OPTS% %IOTDB_HEAP_OPTS% -cp %CLASSPATH% %IOTDB_JMX_OPTS% %MAIN_CLASS% %CONF_PARAMS%
+goto finally
+
+:err
+echo JAVA_HOME environment variable must be set!
+pause
+
+
+@REM -----------------------------------------------------------------------------
+:finally
+
+pause
+
+ENDLOCAL
\ No newline at end of file
diff --git a/cluster/src/assembly/resources/sbin/start-node.sh b/cluster/src/assembly/resources/sbin/start-node.sh
new file mode 100755
index 0000000..f8ff515
--- /dev/null
+++ b/cluster/src/assembly/resources/sbin/start-node.sh
@@ -0,0 +1,100 @@
+#!/bin/bash
+#
+# 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.
+#
+
+
+echo ---------------------
+echo "Starting IoTDB (Cluster Mode)"
+echo ---------------------
+
+if [ -z "${IOTDB_HOME}" ]; then
+  export IOTDB_HOME="`dirname "$0"`/.."
+fi
+
+IOTDB_CONF=${IOTDB_HOME}/conf
+# IOTDB_LOGS=${IOTDB_HOME}/logs
+
+if [ -f "$IOTDB_CONF/cluster-env.sh" ]; then
+    if [ "$#" -ge "1" -a "$1" == "printgc" ]; then
+      . "$IOTDB_CONF/cluster-env.sh" "printgc"
+      shift
+    else
+        . "$IOTDB_CONF/cluster-env.sh"
+    fi
+else
+    echo "can't find $IOTDB_CONF/iotdb-env.sh"
+fi
+
+is_conf_path=false
+for arg do
+    shift
+    if [ "$arg" == "-c" ]; then
+        is_conf_path=true
+        continue
+    fi
+
+    if [ $is_conf_path == true ]; then
+        IOTDB_CONF=$arg
+        is_conf_path=false
+        continue
+    fi
+    set -- "$@" "$arg"
+done
+
+CONF_PARAMS="-s "$*
+
+if [ -n "$JAVA_HOME" ]; then
+    for java in "$JAVA_HOME"/bin/amd64/java "$JAVA_HOME"/bin/java; do
+        if [ -x "$java" ]; then
+            JAVA="$java"
+            break
+        fi
+    done
+else
+    JAVA=java
+fi
+
+if [ -z $JAVA ] ; then
+    echo Unable to find java executable. Check JAVA_HOME and PATH environment variables.  > /dev/stderr
+    exit 1;
+fi
+
+CLASSPATH=""
+for f in ${IOTDB_HOME}/lib/*.jar; do
+  CLASSPATH=${CLASSPATH}":"$f
+done
+classname=org.apache.iotdb.cluster.ClusterMain
+
+launch_service()
+{
+	class="$1"
+	iotdb_parms="-Dlogback.configurationFile=${IOTDB_CONF}/logback.xml"
+	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 $iotdb_parms -cp "$CLASSPATH"  "$class" $CONF_PARAMS
+	return $?
+}
+
+# Start up the service
+launch_service "$classname"
+
+exit $?
diff --git a/cluster/src/assembly/resources/sbin/stop-node.bat b/cluster/src/assembly/resources/sbin/stop-node.bat
new file mode 100644
index 0000000..f6338b6
--- /dev/null
+++ b/cluster/src/assembly/resources/sbin/stop-node.bat
@@ -0,0 +1,27 @@
+@REM
+@REM Licensed to the Apache Software Foundation (ASF) under one
+@REM or more contributor license agreements.  See the NOTICE file
+@REM distributed with this work for additional information
+@REM regarding copyright ownership.  The ASF licenses this file
+@REM to you under the Apache License, Version 2.0 (the
+@REM "License"); you may not use this file except in compliance
+@REM with the License.  You may obtain a copy of the License at
+@REM
+@REM     http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM Unless required by applicable law or agreed to in writing,
+@REM software distributed under the License is distributed on an
+@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+@REM KIND, either express or implied.  See the License for the
+@REM specific language governing permissions and limitations
+@REM under the License.
+@REM
+
+@echo off
+
+pushd..
+set exec_dir=%cd%
+popd
+set exec_dir=%exec_dir:\=\\%
+wmic process where (commandline like "%%iotdb.ClusterMain%%" and not name="wmic.exe" and  commandline  like "%%%exec_dir%%%") delete
+
diff --git a/cluster/src/assembly/resources/sbin/stop-node.sh b/cluster/src/assembly/resources/sbin/stop-node.sh
new file mode 100644
index 0000000..55961c0
--- /dev/null
+++ b/cluster/src/assembly/resources/sbin/stop-node.sh
@@ -0,0 +1,39 @@
+#!/bin/sh
+#
+# 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.
+#
+
+
+PIDS=$(ps ax | grep -i 'ClusterMain' | grep java | grep -v grep | awk '{print $1}')
+sig=0
+for evry_pid in ${PIDS}
+do
+  cwd_path=$(ls -l /proc/$evry_pid | grep "cwd ->" | grep -v grep | awk '{print $NF}')
+  pwd_path=$(/bin/pwd)
+  if [[ $pwd_path =~ $cwd_path ]]; then
+    kill -s TERM $evry_pid
+    echo "close IoTDB"
+    sig=1
+  fi
+done
+
+if [ $sig -eq 0 ]; then
+  echo "No IoTDB server to stop"
+  exit 1
+fi
+
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/ClientMain.java b/cluster/src/main/java/org/apache/iotdb/cluster/ClientMain.java
new file mode 100644
index 0000000..5da4e52
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/ClientMain.java
@@ -0,0 +1,461 @@
+/*
+ * 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;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.jdbc.Config;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RpcTransportFactory;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.service.rpc.thrift.TSCloseOperationReq;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSIService.Client;
+import org.apache.iotdb.service.rpc.thrift.TSIService.Client.Factory;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSProtocolVersion;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.iotdb.session.SessionDataSet;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("java:S106")
+public class ClientMain {
+
+  private static final Logger logger = LoggerFactory.getLogger(ClientMain.class);
+
+  private static final String PARAM_INSERTION = "i";
+  private static final String PARAM_QUERY = "q";
+  private static final String PARAM_DELETE_STORAGE_GROUP = "dsg";
+  private static final String PARAM_DELETE_SERIES = "ds";
+  private static final String PARAM_QUERY_PORTS = "qp";
+  private static final String PARAM_INSERT_PORT = "ip";
+  private static final String PARAM_BATCH = "b";
+  private static Options options = new Options();
+
+  private static String ip = "127.0.0.1";
+  private static int port = 55560;
+
+  static {
+    options.addOption(new Option(PARAM_INSERTION, "Perform insertion"));
+    options.addOption(new Option(PARAM_QUERY, "Perform query"));
+    options.addOption(new Option(PARAM_DELETE_SERIES, "Perform deleting timeseries"));
+    options.addOption(new Option(PARAM_DELETE_STORAGE_GROUP, "Perform deleting storage group"));
+    options.addOption(new Option(PARAM_QUERY_PORTS, true, "Ports to query (ip is currently "
+        + "localhost)"));
+    options.addOption(new Option(PARAM_INSERT_PORT, true, "Port to perform insertion"));
+    options.addOption(new Option(PARAM_BATCH, "Test batch statement"));
+  }
+
+  private static Map<String, TSStatus> failedQueries;
+
+  private static final String[] STORAGE_GROUPS = new String[]{
+      "root.beijing",
+      "root.shanghai",
+      "root.guangzhou",
+      "root.shenzhen",
+  };
+
+  private static final String[] DEVICES = new String[]{
+      "root.beijing.d1",
+      "root.shanghai.d1",
+      "root.guangzhou.d1",
+      "root.shenzhen.d1",
+  };
+
+  private static final String[] MEASUREMENTS = new String[]{
+      "s1"
+  };
+
+  private static final TSDataType[] DATA_TYPES = new TSDataType[]{
+      TSDataType.DOUBLE
+  };
+
+  private static List<MeasurementSchema> schemas;
+
+  private static final String[] DATA_QUERIES = new String[]{
+      // raw data multi series
+      "SELECT * FROM root",
+      "SELECT * FROM root WHERE time <= 691200000",
+      "SELECT * FROM root WHERE time >= 391200000 and time <= 691200000",
+      "SELECT * FROM root.*.* WHERE s1 <= 0.7",
+      // raw data single series
+      "SELECT s1 FROM root.beijing.d1",
+      "SELECT s1 FROM root.shanghai.d1",
+      "SELECT s1 FROM root.guangzhou.d1",
+      "SELECT s1 FROM root.shenzhen.d1",
+      // aggregation
+      "SELECT count(s1) FROM root.*.*",
+      "SELECT avg(s1) FROM root.*.*",
+      "SELECT sum(s1) FROM root.*.*",
+      "SELECT max_value(s1) FROM root.*.*",
+      "SELECT count(s1) FROM root.*.* where time <= 691200000",
+      "SELECT count(s1) FROM root.*.* where s1 <= 0.7",
+      // group by device
+      "SELECT * FROM root GROUP BY DEVICE",
+      // fill
+      "SELECT s1 FROM root.beijing.d1 WHERE time = 86400000 FILL (DOUBLE[PREVIOUS,1d])",
+      "SELECT s1 FROM root.shanghai.d1 WHERE time = 86400000 FILL (DOUBLE[LINEAR,1d,1d])",
+      "SELECT s1 FROM root.guangzhou.d1 WHERE time = 126400000 FILL (DOUBLE[PREVIOUS,1d])",
+      "SELECT s1 FROM root.shenzhen.d1 WHERE time = 126400000 FILL (DOUBLE[LINEAR,1d,1d])",
+      // group by
+      "SELECT COUNT(*) FROM root.*.* GROUP BY ([0, 864000000), 3d, 3d)",
+      "SELECT AVG(*) FROM root.*.* WHERE s1 <= 0.7 GROUP BY ([0, 864000000), 3d, 3d)",
+      // last
+      "SELECT LAST s1 FROM root.*.*",
+  };
+
+  private static final String[] META_QUERY = new String[]{
+      "SHOW STORAGE GROUP",
+      "SHOW TIMESERIES root",
+      "COUNT TIMESERIES root",
+      "COUNT TIMESERIES root GROUP BY LEVEL=2",
+      "SHOW DEVICES",
+      "SHOW TIMESERIES root limit 1 offset 1",
+  };
+
+  public static void main(String[] args)
+      throws TException, StatementExecutionException, IoTDBConnectionException, ParseException, SQLException, ClassNotFoundException {
+    CommandLineParser parser = new DefaultParser();
+    CommandLine commandLine = parser.parse(options, args);
+    boolean noOption = args.length == 0;
+
+    failedQueries = new HashMap<>();
+    prepareSchema();
+
+
+    if (commandLine.hasOption(PARAM_INSERT_PORT)){
+      port = Integer.parseInt(commandLine.getOptionValue(PARAM_INSERT_PORT));
+    }
+
+    doInsertion(noOption, commandLine);
+
+    doQuery(noOption, commandLine);
+
+    doDeleteSeries(noOption, commandLine);
+
+    doDeleteSG(noOption, commandLine);
+
+    doBatchStmt(noOption, commandLine);
+  }
+
+  private static void doInsertion(boolean noOption, CommandLine commandLine) throws TException {
+    if (noOption || commandLine.hasOption(PARAM_INSERTION)) {
+      System.out.println("Test insertion");
+      Client client = getClient(ip, port);
+      long sessionId = connectClient(client);
+      testInsertion(client, sessionId);
+      client.closeSession(new TSCloseSessionReq(sessionId));
+    }
+  }
+
+  private static void doQuery(boolean noOption, CommandLine commandLine)
+      throws StatementExecutionException, TException, IoTDBConnectionException {
+    if (noOption || commandLine.hasOption(PARAM_QUERY)) {
+      int[] queryPorts = null;
+      if (commandLine.hasOption(PARAM_QUERY_PORTS)) {
+        queryPorts = parseIntArray(commandLine.getOptionValue(PARAM_QUERY_PORTS));
+      }
+      if (queryPorts == null) {
+        queryPorts = new int[]{55560, 55561, 55562};
+      }
+      for (int queryPort : queryPorts) {
+        System.out.println("Test port: " + queryPort);
+
+        Client client = getClient(ip, queryPort);
+        long sessionId = connectClient(client);
+        System.out.println("Test data queries");
+        testQuery(client, sessionId, DATA_QUERIES);
+
+        System.out.println("Test metadata queries");
+        testQuery(client, sessionId, META_QUERY);
+
+        logger.info("Failed queries: {}", failedQueries);
+        client.closeSession(new TSCloseSessionReq(sessionId));
+      }
+    }
+  }
+
+  private static void doDeleteSeries(boolean noOption, CommandLine commandLine)
+      throws TException {
+    if (noOption || commandLine.hasOption(PARAM_DELETE_SERIES)) {
+      System.out.println("Test delete timeseries");
+      Client client = getClient(ip, port);
+      long sessionId = connectClient(client);
+      testDeleteTimeseries(client, sessionId);
+      client.closeSession(new TSCloseSessionReq(sessionId));
+    }
+  }
+
+  private static void doDeleteSG(boolean noOption, CommandLine commandLine)
+      throws StatementExecutionException, TException, IoTDBConnectionException {
+    if (noOption || commandLine.hasOption(PARAM_DELETE_STORAGE_GROUP)) {
+      System.out.println("Test delete storage group");
+      Client client = getClient(ip, port);
+      long sessionId = connectClient(client);
+      testDeleteStorageGroup(client, sessionId);
+      client.closeSession(new TSCloseSessionReq(sessionId));
+    }
+  }
+
+  private static void doBatchStmt(boolean noOption, CommandLine commandLine)
+      throws SQLException, ClassNotFoundException {
+    if (noOption || commandLine.hasOption(PARAM_BATCH)) {
+      System.out.println("Test batch create sgs");
+      testBatch(ip, port);
+    }
+  }
+
+  private static int[] parseIntArray(String str) {
+    if (str == null) {
+      return new int[0];
+    }
+    String[] split = str.split(",");
+    int[] ret = new int[split.length];
+    for (int i = 0; i < split.length; i++) {
+      ret[i] = Integer.parseInt(split[i]);
+    }
+    return ret;
+  }
+
+  private static long connectClient(Client client) throws TException {
+    TSOpenSessionReq openReq = new TSOpenSessionReq(TSProtocolVersion.IOTDB_SERVICE_PROTOCOL_V3,
+        ZoneId.systemDefault().getId());
+    openReq.setUsername("root");
+    openReq.setPassword("root");
+    TSOpenSessionResp openResp = client.openSession(openReq);
+    return openResp.getSessionId();
+  }
+
+  @SuppressWarnings({"java:S2095"}) // the transport is used later
+  private static Client getClient(String ip, int port) throws TTransportException {
+    TSIService.Client.Factory factory = new Factory();
+    TTransport transport = RpcTransportFactory.INSTANCE.getTransport(new TSocket(ip, port));
+    transport.open();
+    TProtocol protocol =
+        IoTDBDescriptor.getInstance().getConfig().isRpcThriftCompressionEnable() ?
+            new TCompactProtocol(transport) : new TBinaryProtocol(transport);
+    return factory.getClient(protocol);
+  }
+
+  private static void prepareSchema() {
+    schemas = new ArrayList<>();
+    for (String device : DEVICES) {
+      for (int i = 0; i < MEASUREMENTS.length; i++) {
+        String measurement = MEASUREMENTS[i];
+        schemas.add(new MeasurementSchema(device + IoTDBConstant.PATH_SEPARATOR + measurement,
+            DATA_TYPES[i]));
+      }
+    }
+  }
+
+  private static void testQuery(Client client, long sessionId, String[] queries)
+      throws TException, StatementExecutionException, IoTDBConnectionException {
+    long statementId = client.requestStatementId(sessionId);
+    for (String dataQuery : queries) {
+      executeQuery(client, sessionId, dataQuery, statementId);
+    }
+
+    TSCloseOperationReq tsCloseOperationReq = new TSCloseOperationReq(sessionId);
+    tsCloseOperationReq.setStatementId(statementId);
+    client.closeOperation(tsCloseOperationReq);
+  }
+
+  private static void executeQuery(Client client, long sessionId, String query, long statementId)
+      throws TException, StatementExecutionException, IoTDBConnectionException {
+    if (logger.isInfoEnabled()) {
+      logger.info("{ {} }", query);
+    }
+    TSExecuteStatementResp resp = client
+        .executeQueryStatement(new TSExecuteStatementReq(sessionId, query, statementId).setFetchSize(1000));
+    if (resp.status.code != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      failedQueries.put(query, resp.status);
+      return;
+    }
+
+    long queryId = resp.getQueryId();
+    if (logger.isInfoEnabled()) {
+      logger.info(resp.columns.toString());
+    }
+
+    SessionDataSet dataSet = new SessionDataSet(query, resp.getColumns(),
+        resp.getDataTypeList(), resp.columnNameIndexMap, queryId, client, sessionId,
+        resp.queryDataSet, false);
+
+    while (dataSet.hasNext()) {
+      if (logger.isInfoEnabled()) {
+        logger.info(dataSet.next().toString());
+      }
+    }
+    System.out.println();
+
+    TSCloseOperationReq tsCloseOperationReq = new TSCloseOperationReq(sessionId);
+    tsCloseOperationReq.setQueryId(queryId);
+    client.closeOperation(tsCloseOperationReq);
+  }
+
+  private static void testDeleteStorageGroup(Client client, long sessionId)
+      throws TException, StatementExecutionException, IoTDBConnectionException {
+    if (logger.isInfoEnabled()) {
+      logger.info(client.deleteStorageGroups(sessionId, Arrays.asList(STORAGE_GROUPS)).toString());
+    }
+
+    testQuery(client, sessionId, new String[]{"SELECT * FROM root"});
+  }
+
+  private static void registerTimeseries(long sessionId, Client client) throws TException {
+    TSCreateTimeseriesReq req = new TSCreateTimeseriesReq();
+    req.setSessionId(sessionId);
+    for (MeasurementSchema schema : schemas) {
+      req.setDataType(schema.getType().ordinal());
+      req.setEncoding(schema.getEncodingType().ordinal());
+      req.setCompressor(schema.getCompressor().ordinal());
+      req.setPath(schema.getMeasurementId());
+      if (logger.isInfoEnabled()) {
+        logger.info(client.createTimeseries(req).toString());
+      }
+    }
+  }
+
+  @SuppressWarnings("ConstantConditions")
+  private static void testInsertion(Client client, long sessionId) throws TException {
+    for (String storageGroup : STORAGE_GROUPS) {
+      if (logger.isInfoEnabled()) {
+        logger.info(client.setStorageGroup(sessionId, storageGroup).toString());
+      }
+    }
+
+    registerTimeseries(sessionId, client);
+
+    TSInsertStringRecordReq insertReq = new TSInsertStringRecordReq();
+    insertReq.setMeasurements(Arrays.asList(MEASUREMENTS));
+    insertReq.setSessionId(sessionId);
+
+    for (int i = 0; i < 10; i++) {
+      List<String> values = new ArrayList<>(MEASUREMENTS.length);
+      insertReq.setTimestamp(i * 24 * 3600 * 1000L);
+      for (int i1 = 0; i1 < MEASUREMENTS.length; i1++) {
+        switch (DATA_TYPES[i1]) {
+          case DOUBLE:
+            values.add(Double.toString(i * 0.1));
+            break;
+          case BOOLEAN:
+            values.add(Boolean.toString(i % 2 == 0));
+            break;
+          case INT64:
+            values.add(Long.toString(i));
+            break;
+          case INT32:
+            values.add(Integer.toString(i));
+            break;
+          case FLOAT:
+            values.add(Float.toString(i * 0.1f));
+            break;
+          case TEXT:
+            values.add("S" + i);
+            break;
+        }
+      }
+
+      insertReq.setValues(values);
+
+      for (String device : DEVICES) {
+        insertReq.setDeviceId(device);
+        if (logger.isInfoEnabled()) {
+          logger.info(insertReq.toString());
+          logger.info(client.insertStringRecord(insertReq).toString());
+        }
+      }
+    }
+  }
+
+  private static void testDeleteTimeseries(Client client, long sessionId) throws TException {
+    List<String> paths = new ArrayList<>();
+    for (String measurement : MEASUREMENTS) {
+      for (String device : DEVICES) {
+        paths.add(device + "." + measurement);
+      }
+    }
+    if (logger.isInfoEnabled()) {
+      logger.info(client.deleteTimeseries(sessionId, paths).toString());
+    }
+  }
+
+  private static void testBatch(String ip, int port) throws ClassNotFoundException, SQLException {
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + String.format("%s:%d/", ip, port), "root",
+            "root");
+        Statement statement = connection.createStatement()) {
+
+      statement.addBatch("SET STORAGE GROUP TO root.batch1");
+      statement.addBatch("SET STORAGE GROUP TO root.batch2");
+      statement.addBatch("SET STORAGE GROUP TO root.batch3");
+      statement.addBatch("SET STORAGE GROUP TO root.batch4");
+
+      statement.executeBatch();
+      statement.clearBatch();
+
+      try (ResultSet set = statement.executeQuery("SHOW STORAGE GROUP")) {
+        int colNum = set.getMetaData().getColumnCount();
+        while (set.next()) {
+          StringBuilder stringBuilder = new StringBuilder();
+          for (int i = 0; i < colNum; i++) {
+            stringBuilder.append(set.getString(i + 1)).append(",");
+          }
+          System.out.println(stringBuilder.toString());
+        }
+      }
+    }
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/ClusterFileFlushPolicy.java b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterFileFlushPolicy.java
new file mode 100644
index 0000000..cca5c2a
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterFileFlushPolicy.java
@@ -0,0 +1,67 @@
+/*
+ * 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;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.iotdb.cluster.server.member.MetaGroupMember;
+import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy;
+import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileProcessor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClusterFileFlushPolicy implements TsFileFlushPolicy {
+
+  private static final Logger logger = LoggerFactory.getLogger(ClusterFileFlushPolicy.class);
+
+  private ExecutorService closePartitionExecutor;
+  private MetaGroupMember metaGroupMember;
+
+  public ClusterFileFlushPolicy(
+      MetaGroupMember metaGroupMember) {
+    this.metaGroupMember = metaGroupMember;
+    this.closePartitionExecutor = new ThreadPoolExecutor(16, 1024, 0, TimeUnit.SECONDS,
+        new LinkedBlockingDeque<>(), r -> {
+      Thread thread = new Thread(r);
+      thread.setName("ClusterFileFlushPolicy-" + thread.getId());
+      return thread;
+    });
+  }
+
+  @Override
+  public void apply(StorageGroupProcessor storageGroupProcessor, TsFileProcessor processor,
+      boolean isSeq) {
+    logger.info("The memtable size reaches the threshold, async flush it to tsfile: {}",
+        processor.getTsFileResource().getTsFile().getAbsolutePath());
+
+    if (processor.shouldClose()) {
+      // find the related DataGroupMember and close the processor through it
+      // we execute it in another thread to avoid deadlocks
+      closePartitionExecutor
+          .submit(() -> metaGroupMember.closePartition(storageGroupProcessor.getStorageGroupName(),
+              processor.getTimeRangeId(), isSeq));
+    }
+    // flush the memtable anyway to avoid the insertion trigger the policy again
+    processor.asyncFlush();
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java
new file mode 100644
index 0000000..b382242
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java
@@ -0,0 +1,319 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.iotdb.cluster.client.async.AsyncMetaClient;
+import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.exception.ConfigInconsistentException;
+import org.apache.iotdb.cluster.exception.StartUpCheckFailureException;
+import org.apache.iotdb.cluster.partition.slot.SlotPartitionTable;
+import org.apache.iotdb.cluster.partition.slot.SlotStrategy;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.server.MetaClusterServer;
+import org.apache.iotdb.cluster.server.Response;
+import org.apache.iotdb.cluster.utils.ClusterUtils;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.StartupException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.TAsyncClientManager;
+import org.apache.thrift.protocol.TBinaryProtocol.Factory;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClusterMain {
+
+  private static final Logger logger = LoggerFactory.getLogger(ClusterMain.class);
+
+  // establish the cluster as a seed
+  private static final String MODE_START = "-s";
+  // join an established cluster
+  private static final String MODE_ADD = "-a";
+  // send a request to remove a node, more arguments: ip-of-removed-node
+  // metaport-of-removed-node
+  private static final String MODE_REMOVE = "-r";
+  // the separator between the cluster configuration and the single-server configuration
+  private static final String SERVER_CONF_SEPARATOR = "-sc";
+  private static MetaClusterServer metaServer;
+
+  public static void main(String[] args) {
+    if (args.length < 1) {
+      logger.error("Usage: <-s|-a|-r> [-internal_meta_port <internal meta port>] "
+          + "[-internal_data_port <internal data port>] "
+          + "[-cluster_rpc_port <cluster rpc port>] "
+          + "[-seed_nodes <node1:meta_port:data_port:cluster_rpc_port,"
+          +               "node2:meta_port:data_port:cluster_rpc_port,"
+          +           "...,noden:meta_port:data_port:cluster_rpc_port,>] "
+          + "[-sc] "
+          + "[-rpc_port <rpc port>]");
+      return;
+    }
+    String mode = args[0];
+    if (args.length > 1) {
+      String[] params = Arrays.copyOfRange(args, 1, args.length);
+      replaceDefaultPrams(params);
+    }
+
+    // params check
+    if (!checkConfig()) {
+      return;
+    }
+
+    IoTDBDescriptor.getInstance().getConfig().setSyncEnable(false);
+    IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(false);
+    logger.info("Running mode {}", mode);
+    if (MODE_START.equals(mode)) {
+      try {
+        metaServer = new MetaClusterServer();
+        startServerCheck();
+        preStartCustomize();
+        metaServer.start();
+        metaServer.buildCluster();
+      } catch (TTransportException | StartupException | QueryProcessException |
+          StartUpCheckFailureException | ConfigInconsistentException e) {
+        metaServer.stop();
+        logger.error("Fail to start meta server", e);
+      }
+    } else if (MODE_ADD.equals(mode)) {
+      try {
+        metaServer = new MetaClusterServer();
+        preStartCustomize();
+        metaServer.start();
+        metaServer.joinCluster();
+      } catch (TTransportException | StartupException | QueryProcessException | StartUpCheckFailureException | ConfigInconsistentException e) {
+        metaServer.stop();
+        logger.error("Fail to join cluster", e);
+      }
+    } else if (MODE_REMOVE.equals(mode)) {
+      try {
+        doRemoveNode(args);
+      } catch (IOException e) {
+        logger.error("Fail to remove node in cluster", e);
+      }
+    } else {
+      logger.error("Unrecognized mode {}", mode);
+    }
+  }
+
+  private static void startServerCheck() throws StartupException {
+    ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
+    // check the initial replicateNum and refuse to start when the replicateNum <= 0
+    if (config.getReplicationNum() <= 0) {
+      String message = String.format("ReplicateNum should be greater than 0 instead of %d.",
+          config.getReplicationNum());
+      throw new StartupException(metaServer.getMember().getName(), message);
+    }
+    // check the initial cluster size and refuse to start when the size < quorum
+    int quorum = config.getReplicationNum() / 2 + 1;
+    if (config.getSeedNodeUrls().size() < quorum) {
+      String message = String.format("Seed number less than quorum, seed number: %s, quorum: "
+              + "%s.",
+          config.getSeedNodeUrls().size(), quorum);
+      throw new StartupException(metaServer.getMember().getName(), message);
+    }
+    // assert not duplicated nodes
+    Set<Node> seedNodes = new HashSet<>();
+    for (String url : config.getSeedNodeUrls()) {
+      Node node = ClusterUtils.parseNode(url);
+      if (seedNodes.contains(node)) {
+        String message = String.format(
+            "SeedNodes must not repeat each other. SeedNodes: %s", config.getSeedNodeUrls());
+        throw new StartupException(metaServer.getMember().getName(), message);
+      }
+      seedNodes.add(node);
+    }
+    // assert this node is in NodeList
+    Node localNode = new Node();
+    localNode.setIp(config.getClusterRpcIp()).setMetaPort(config.getInternalMetaPort())
+        .setDataPort(config.getInternalDataPort()).setClientPort(config.getClusterRpcPort());
+    if (!seedNodes.contains(localNode)) {
+      String message = String.format(
+          "SeedNodes must contains local node in start-server mode. LocalNode: %s ,SeedNodes: %s",
+          localNode.toString(), config.getSeedNodeUrls());
+      throw new StartupException(metaServer.getMember().getName(), message);
+    }
+  }
+
+  private static void replaceDefaultPrams(String[] args) {
+    int index;
+    String[] clusterParams;
+    String[] serverParams = null;
+    for (index = 0; index < args.length; index++) {
+      //find where -sc is
+      if (SERVER_CONF_SEPARATOR.equals(args[index])) {
+        break;
+      }
+    }
+    //parameters from 0 to "-sc" are for clusters
+    clusterParams = Arrays.copyOfRange(args, 0, index);
+
+    if (index < args.length) {
+      serverParams = Arrays.copyOfRange(args, index + 1, args.length);
+    }
+
+    if (clusterParams.length > 0) {
+      // replace the cluster default conf params
+      ClusterDescriptor.getInstance().replaceProps(clusterParams);
+    }
+
+    if (serverParams != null && serverParams.length > 0) {
+      // replace the server default conf params
+      IoTDBDescriptor.getInstance().replaceProps(serverParams);
+    }
+  }
+
+  /**
+   * check the configuration is legal or not
+   */
+  private static boolean checkConfig() {
+    // 0. first replace all hostname with ip
+    try {
+      ClusterDescriptor.getInstance().replaceHostnameWithIp();
+    } catch (Exception e) {
+      logger.error("replace hostname with ip failed, {}", e.getMessage());
+      return false;
+    }
+
+    // 1. check the cluster_rpc_ip and seed_nodes consistent or not
+    ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
+    String clusterRpcIp = "127.0.0.1";
+    String configClusterRpcIp = config.getClusterRpcIp();
+    List<String> seedNodes = config.getSeedNodeUrls();
+    boolean isClusterRpcIp = clusterRpcIp.equals(configClusterRpcIp);
+    for (String seedNodeIP : seedNodes) {
+      if ((isClusterRpcIp && !seedNodeIP.contains(clusterRpcIp)) ||
+          (!isClusterRpcIp && seedNodeIP.contains(clusterRpcIp))) {
+        logger.error(
+            "cluster_rpc_ip={} and seed_nodes={} should be consistent, both use local ip or real ip please",
+            configClusterRpcIp, seedNodes);
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static void doRemoveNode(String[] args) throws IOException {
+    if (args.length != 3) {
+      logger.error("Usage: -r <ip> <metaPort>");
+      return;
+    }
+    String ip = args[1];
+    int metaPort = Integer.parseInt(args[2]);
+    ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
+    TProtocolFactory factory = config
+        .isRpcThriftCompressionEnabled() ? new TCompactProtocol.Factory() : new Factory();
+    Node nodeToRemove = new Node();
+    nodeToRemove.setIp(ip).setMetaPort(metaPort);
+    // try sending the request to each seed node
+    for (String url : config.getSeedNodeUrls()) {
+      Node node = ClusterUtils.parseNode(url);
+      if (node == null) {
+        continue;
+      }
+      AsyncMetaClient client = new AsyncMetaClient(factory, new TAsyncClientManager(), node, null);
+      Long response = null;
+      try {
+        logger.info("Start removing node {} with the help of node {}", nodeToRemove, node);
+        response = SyncClientAdaptor.removeNode(client, nodeToRemove);
+      } catch (TException e) {
+        logger.warn("Cannot send remove node request through {}, try next node", node);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        logger.warn("Cannot send remove node request through {}, try next node", node);
+      }
+      if (response != null) {
+        handleNodeRemovalResp(response, nodeToRemove);
+        return;
+      }
+    }
+  }
+
+  private static void handleNodeRemovalResp(Long response, Node nodeToRemove) {
+    if (response == Response.RESPONSE_AGREE) {
+      logger.info("Node {} is successfully removed", nodeToRemove);
+    } 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 {
+      logger.error("Unexpected response {}", response);
+    }
+  }
+
+  public static MetaClusterServer getMetaServer() {
+    return metaServer;
+  }
+
+  /**
+   * Developers may perform pre-start customizations here for debugging or experiments.
+   *
+   */
+  @SuppressWarnings("java:S125") // leaving examples
+  private static void preStartCustomize() {
+    // customize data distribution
+    // The given example tries to divide storage groups like "root.sg_0", "root.sg_1"... into k
+    // nodes evenly, and use default strategy for other groups
+    SlotPartitionTable.setSlotStrategy(new SlotStrategy() {
+      SlotStrategy defaultStrategy = new SlotStrategy.DefaultStrategy();
+      int k = 3;
+      @Override
+      public int calculateSlotByTime(String storageGroupName, long timestamp, int maxSlotNum) {
+        int sgSerialNum = extractSerialNumInSGName(storageGroupName) % k;
+        if (sgSerialNum >= 0) {
+          return maxSlotNum / k * sgSerialNum;
+        } else {
+          return defaultStrategy.calculateSlotByTime(storageGroupName, timestamp, maxSlotNum);
+        }
+      }
+
+      @Override
+      public int calculateSlotByPartitionNum(String storageGroupName, long partitionId,
+          int maxSlotNum) {
+        int sgSerialNum = extractSerialNumInSGName(storageGroupName) % k;
+        if (sgSerialNum >= 0) {
+          return maxSlotNum / k * sgSerialNum;
+        } else {
+          return defaultStrategy.calculateSlotByPartitionNum(storageGroupName, partitionId, maxSlotNum);
+        }
+      }
+
+      private int extractSerialNumInSGName(String storageGroupName) {
+        String[] s = storageGroupName.split("_");
+        if (s.length != 2) {
+          return -1;
+        }
+        try {
+          return Integer.parseInt(s[1]);
+        } catch (NumberFormatException e) {
+          return -1;
+        }
+      }
+    });
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java b/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java
new file mode 100644
index 0000000..9870f8c
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java
@@ -0,0 +1,174 @@
+/*
+ * 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;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.utils.SerializeUtils;
+
+public class RemoteTsFileResource extends TsFileResource {
+
+  private Node source;
+  private boolean isRemote = false;
+  private boolean withModification = false;
+
+  /**
+   * Whether the plan range ([minPlanIndex, maxPlanIndex]) overlaps with another TsFile in the same
+   * time partition. If not (unique = true), we shall have confidence that the file has all data
+   * whose plan indexes are within [minPlanIndex, maxPlanIndex], so we can remove other local
+   * files that overlaps with it.
+   */
+  private boolean isPlanRangeUnique = false;
+
+  public RemoteTsFileResource() {
+    setClosed(true);
+    this.deviceToIndex = new ConcurrentHashMap<>();
+    this.startTimes = new long[INIT_ARRAY_SIZE];
+    this.endTimes = new long[INIT_ARRAY_SIZE];
+    initTimes(startTimes, Long.MAX_VALUE);
+    initTimes(endTimes, Long.MIN_VALUE);
+  }
+
+  private RemoteTsFileResource(TsFileResource other) throws IOException {
+    super(other);
+    withModification = new File(getModFile().getFilePath()).exists();
+    setClosed(true);
+  }
+
+  public RemoteTsFileResource(TsFileResource other, Node source) throws IOException {
+    this(other);
+    this.source = source;
+    this.isRemote = true;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    if (!super.equals(o)) {
+      return false;
+    }
+    RemoteTsFileResource that = (RemoteTsFileResource) o;
+    return Objects.equals(source, that.source);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), source);
+  }
+
+  public void serialize(DataOutputStream dataOutputStream) {
+    SerializeUtils.serialize(source, dataOutputStream);
+    try {
+      // the path here is only for the remote node to get a download link, so it does not matter
+      // if it is absolute
+      SerializeUtils.serialize(getTsFile().getPath(), dataOutputStream);
+
+      int deviceNum = deviceToIndex.size();
+      dataOutputStream.writeInt(deviceNum);
+      for (int i = 0; i < deviceNum; i++) {
+        dataOutputStream.writeLong(startTimes[i]);
+        dataOutputStream.writeLong(endTimes[i]);
+      }
+
+      for (Entry<String, Integer> stringIntegerEntry : deviceToIndex.entrySet()) {
+        String deviceName = stringIntegerEntry.getKey();
+        int index = stringIntegerEntry.getValue();
+        SerializeUtils.serialize(deviceName, dataOutputStream);
+        dataOutputStream.writeInt(index);
+      }
+
+      dataOutputStream.writeBoolean(withModification);
+
+      dataOutputStream.writeLong(maxPlanIndex);
+      dataOutputStream.writeLong(minPlanIndex);
+
+      dataOutputStream.writeByte(isPlanRangeUnique ? 1 : 0);
+    } catch (IOException ignored) {
+      // unreachable
+    }
+  }
+
+  public void deserialize(ByteBuffer buffer) {
+    source = new Node();
+    SerializeUtils.deserialize(source, buffer);
+    setFile(new File(SerializeUtils.deserializeString(buffer)));
+
+    int deviceNum = buffer.getInt();
+    startTimes = new long[deviceNum];
+    endTimes = new long[deviceNum];
+    deviceToIndex = new ConcurrentHashMap<>(deviceNum);
+
+    for (int i = 0; i < deviceNum; i++) {
+      startTimes[i] = buffer.getLong();
+      endTimes[i] = buffer.getLong();
+    }
+
+    for (int i = 0; i < deviceNum; i++) {
+      String deviceName = SerializeUtils.deserializeString(buffer);
+      int index = buffer.getInt();
+      deviceToIndex.put(deviceName, index);
+    }
+
+    withModification = buffer.get() == 1;
+
+    maxPlanIndex = buffer.getLong();
+    minPlanIndex = buffer.getLong();
+
+    isPlanRangeUnique = buffer.get() == 1;
+
+    isRemote = true;
+  }
+
+  public Node getSource() {
+    return source;
+  }
+
+  public boolean isRemote() {
+    return isRemote;
+  }
+
+  public void setRemote(boolean remote) {
+    isRemote = remote;
+  }
+
+  public boolean isWithModification() {
+    return withModification;
+  }
+
+  public boolean isPlanRangeUnique() {
+    return isPlanRangeUnique;
+  }
+
+  public void setPlanRangeUnique(boolean planRangeUnique) {
+    isPlanRangeUnique = planRangeUnique;
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/DataClientProvider.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/DataClientProvider.java
new file mode 100644
index 0000000..cb8a05c
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/DataClientProvider.java
@@ -0,0 +1,79 @@
+/*
+ * 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.client;
+
+import java.io.IOException;
+import org.apache.iotdb.cluster.client.async.AsyncClientPool;
+import org.apache.iotdb.cluster.client.async.AsyncDataClient;
+import org.apache.iotdb.cluster.client.async.AsyncDataClient.FactoryAsync;
+import org.apache.iotdb.cluster.client.sync.SyncClientPool;
+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.thrift.protocol.TProtocolFactory;
+
+public class DataClientProvider {
+  /**
+   * dataClientPool provides reusable thrift clients to connect to the DataGroupMembers of other
+   * nodes
+   */
+  private AsyncClientPool dataAsyncClientPool;
+  private SyncClientPool dataSyncClientPool;
+
+  public DataClientProvider(TProtocolFactory factory) {
+    if (!ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      dataSyncClientPool = new SyncClientPool(new SyncDataClient.FactorySync(factory));
+    } else {
+      dataAsyncClientPool = new AsyncClientPool(new FactoryAsync(factory));
+    }
+  }
+
+  private AsyncClientPool getDataAsyncClientPool() {
+    return dataAsyncClientPool;
+  }
+
+  private SyncClientPool getDataSyncClientPool() {
+    return dataSyncClientPool;
+  }
+
+  /**
+   * Get a thrift client that will connect to "node" using the data port.
+   *
+   * @param node the node to be connected
+   * @param timeout timeout threshold of connection
+   */
+  public AsyncDataClient getAsyncDataClient(Node node, int timeout) throws IOException {
+    AsyncDataClient client = (AsyncDataClient) getDataAsyncClientPool().getClient(node);
+    client.setTimeout(timeout);
+    return client;
+  }
+
+  /**
+   * Get a thrift client that will connect to "node" using the data port.
+   *
+   * @param node the node to be connected
+   * @param timeout timeout threshold of connection
+   */
+  public SyncDataClient getSyncDataClient(Node node, int timeout) {
+    SyncDataClient client = (SyncDataClient) getDataSyncClientPool().getClient(node);
+    client.setTimeout(timeout);
+    return client;
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncClientFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncClientFactory.java
new file mode 100644
index 0000000..45f6d27
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncClientFactory.java
@@ -0,0 +1,62 @@
+/*
+ * 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.client.async;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService;
+import org.apache.thrift.async.TAsyncClientManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AsyncClientFactory {
+
+  private static final Logger logger = LoggerFactory.getLogger(AsyncClientFactory.class);
+  static TAsyncClientManager[] managers;
+  org.apache.thrift.protocol.TProtocolFactory protocolFactory;
+  AtomicInteger clientCnt = new AtomicInteger();
+
+  static {
+    managers =
+        new TAsyncClientManager[ClusterDescriptor.getInstance().getConfig()
+            .getSelectorNumOfClientPool()];
+    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      for (int i = 0; i < managers.length; i++) {
+        try {
+          managers[i] = new TAsyncClientManager();
+        } catch (IOException e) {
+          logger.error("Cannot create data heartbeat client manager for factory", e);
+        }
+      }
+    }
+  }
+
+  /**
+   * Get a client which will connect the given node and be cached in the given pool.
+   * @param node the cluster node the client will connect.
+   * @param pool the pool that will cache the client for reusing.
+   * @return
+   * @throws IOException
+   */
+  protected abstract RaftService.AsyncClient getAsyncClient(Node node, AsyncClientPool pool)
+      throws IOException;
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncClientPool.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncClientPool.java
new file mode 100644
index 0000000..9901840
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncClientPool.java
@@ -0,0 +1,234 @@
+/*
+ * 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.client.async;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.concurrent.BasicThreadFactory;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService.AsyncClient;
+import org.apache.iotdb.cluster.utils.ClusterNode;
+import org.apache.thrift.async.TAsyncMethodCall;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AsyncClientPool {
+
+  private static final Logger logger = LoggerFactory.getLogger(AsyncClientPool.class);
+  private static final long WAIT_CLIENT_TIMEOUT_MS = 5 * 1000L;
+  private int maxConnectionForEachNode;
+  private Map<ClusterNode, Deque<AsyncClient>> clientCaches = new ConcurrentHashMap<>();
+  private Map<ClusterNode, Integer> nodeClientNumMap = new ConcurrentHashMap<>();
+  private Map<ClusterNode, Integer> nodeErrorClientCountMap = new ConcurrentHashMap<>();
+  private AsyncClientFactory asyncClientFactory;
+  private ScheduledExecutorService cleanErrorClientExecutorService;
+  // when set to true, if MAX_ERROR_COUNT errors occurs continuously when connecting to node, any
+  // further requests to the node will be rejected for PROBE_NODE_STATUS_PERIOD_SECOND
+  // heartbeats should not be blocked
+  private boolean blockOnError;
+
+  private static final int MAX_ERROR_COUNT = 3;
+  private static final int PROBE_NODE_STATUS_PERIOD_SECOND = 60;
+
+  public AsyncClientPool(AsyncClientFactory asyncClientFactory) {
+    this(asyncClientFactory, true);
+  }
+
+  public AsyncClientPool(AsyncClientFactory asyncClientFactory, boolean blockOnError) {
+    this.asyncClientFactory = asyncClientFactory;
+    this.maxConnectionForEachNode =
+        ClusterDescriptor.getInstance().getConfig().getMaxClientPerNodePerMember();
+    this.blockOnError = blockOnError;
+    if (blockOnError) {
+      this.cleanErrorClientExecutorService = new ScheduledThreadPoolExecutor(1,
+          new BasicThreadFactory.Builder().namingPattern("clean-error-client-%d").daemon(true)
+              .build());
+      this.cleanErrorClientExecutorService
+          .scheduleAtFixedRate(this::cleanErrorClients, PROBE_NODE_STATUS_PERIOD_SECOND,
+              PROBE_NODE_STATUS_PERIOD_SECOND, TimeUnit.SECONDS);
+    }
+  }
+
+  /**
+   * Get a client of the given node from the cache if one is available, or create a new one.
+   *
+   * @param node
+   * @return
+   * @throws IOException
+   */
+  public AsyncClient getClient(Node node) throws IOException {
+    ClusterNode clusterNode = new ClusterNode(node);
+    if (blockOnError && nodeErrorClientCountMap.getOrDefault(clusterNode, 0) > MAX_ERROR_COUNT) {
+      throw new IOException(String.format("connect node failed, maybe the node is down, %s", node));
+    }
+
+    AsyncClient client;
+    synchronized (this) {
+      //As clientCaches is ConcurrentHashMap, computeIfAbsent is thread safety.
+      Deque<AsyncClient> clientStack = clientCaches.computeIfAbsent(clusterNode,
+          n -> new ArrayDeque<>());
+      if (clientStack.isEmpty()) {
+        int nodeClientNum = nodeClientNumMap.getOrDefault(clusterNode, 0);
+        if (nodeClientNum >= maxConnectionForEachNode) {
+          client = waitForClient(clientStack, clusterNode, nodeClientNum);
+        } else {
+          nodeClientNumMap.put(clusterNode, nodeClientNum + 1);
+          client = asyncClientFactory.getAsyncClient(clusterNode, this);
+        }
+      } else {
+        client = clientStack.pop();
+      }
+    }
+    return client;
+  }
+
+  /**
+   * Wait for a client to be returned for at most WAIT_CLIENT_TIMEOUT_MS milliseconds. If no client
+   * is returned beyond the timeout, a new client will be returned. WARNING: the caller must
+   * synchronize on the pool.
+   *
+   * @param clientStack
+   * @param node
+   * @param nodeClientNum
+   * @return
+   * @throws IOException
+   */
+  @SuppressWarnings({"squid:S2273"}) // synchronized outside
+  private AsyncClient waitForClient(Deque<AsyncClient> clientStack, ClusterNode node,
+      int nodeClientNum)
+      throws IOException {
+    // wait for an available client
+    long waitStart = System.currentTimeMillis();
+    while (clientStack.isEmpty()) {
+      try {
+        this.wait(WAIT_CLIENT_TIMEOUT_MS);
+        if (clientStack.isEmpty()
+            && System.currentTimeMillis() - waitStart >= WAIT_CLIENT_TIMEOUT_MS) {
+          logger.warn("Cannot get an available client after {}ms, create a new one, factory {} now is {}",
+              WAIT_CLIENT_TIMEOUT_MS, asyncClientFactory, nodeClientNum);
+          nodeClientNumMap.put(node, nodeClientNum + 1);
+          return asyncClientFactory.getAsyncClient(node, this);
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        logger.warn("Interrupted when waiting for an available client of {}", node);
+        return null;
+      }
+    }
+    return clientStack.pop();
+  }
+
+  /**
+   * Return a client of a node to the pool. Closed client should not be returned.
+   *
+   * @param node
+   * @param client
+   */
+  public void putClient(Node node, AsyncClient client) {
+    ClusterNode clusterNode = new ClusterNode(node);
+    TAsyncMethodCall<?> call = null;
+    if (client instanceof AsyncDataClient) {
+      call = ((AsyncDataClient) client).getCurrMethod();
+    } else if (client instanceof AsyncMetaClient) {
+      call = ((AsyncMetaClient) client).getCurrMethod();
+    }
+    if (call != null) {
+      logger.warn("A using client {} is put back while running {}", client.hashCode(), call);
+    }
+    synchronized (this) {
+      //As clientCaches is ConcurrentHashMap, computeIfAbsent is thread safety.
+      Deque<AsyncClient> clientStack = clientCaches
+          .computeIfAbsent(clusterNode, n -> new ArrayDeque<>());
+      clientStack.push(client);
+      this.notifyAll();
+    }
+  }
+
+  void onError(Node node) {
+    ClusterNode clusterNode = new ClusterNode(node);
+    // clean all cached clients when network fails
+    synchronized (this) {
+      Deque<AsyncClient> clientStack = clientCaches
+          .computeIfAbsent(clusterNode, n -> new ArrayDeque<>());
+      while (!clientStack.isEmpty()) {
+        AsyncClient client = clientStack.pop();
+        if (client instanceof AsyncDataClient) {
+          ((AsyncDataClient) client).close();
+        } else if (client instanceof AsyncMetaClient) {
+          ((AsyncMetaClient) client).close();
+        }
+      }
+      clientStack.clear();
+      nodeClientNumMap.put(clusterNode, 0);
+      this.notifyAll();
+    }
+    if (!blockOnError) {
+      return;
+    }
+    synchronized (this) {
+      if (nodeErrorClientCountMap.containsKey(clusterNode)) {
+        nodeErrorClientCountMap.put(clusterNode, nodeErrorClientCountMap.get(clusterNode) + 1);
+      } else {
+        nodeErrorClientCountMap.put(clusterNode, 1);
+      }
+    }
+    if (logger.isDebugEnabled()) {
+      logger.debug("the node={}, connect error times={}", clusterNode,
+          nodeErrorClientCountMap.get(clusterNode));
+    }
+  }
+
+  @SuppressWarnings("squid:S1135")
+  void onComplete(Node node) {
+    ClusterNode clusterNode = new ClusterNode(node);
+    // TODO: if the heartbeat client pool completes, also unblock another pool
+    nodeErrorClientCountMap.remove(clusterNode);
+  }
+
+  void cleanErrorClients() {
+    synchronized (this) {
+      nodeErrorClientCountMap.clear();
+      logger.debug("clean all error clients");
+    }
+  }
+
+  void recreateClient(Node node) {
+    ClusterNode clusterNode = new ClusterNode(node);
+    synchronized (this) {
+      Deque<AsyncClient> clientStack = clientCaches
+          .computeIfAbsent(clusterNode, n -> new ArrayDeque<>());
+      try {
+        AsyncClient asyncClient = asyncClientFactory.getAsyncClient(node, this);
+        clientStack.push(asyncClient);
+      } catch (IOException e) {
+        logger.error("Cannot create a new client for {}", node, e);
+        nodeClientNumMap.computeIfPresent(clusterNode, (n, cnt) -> cnt - 1);
+      }
+      this.notifyAll();
+    }
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncDataClient.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncDataClient.java
new file mode 100644
index 0000000..28a6a02
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncDataClient.java
@@ -0,0 +1,148 @@
+/*
+ * 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.client.async;
+
+import java.io.IOException;
+import java.util.Date;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService;
+import org.apache.iotdb.cluster.rpc.thrift.TSDataService.AsyncClient;
+import org.apache.iotdb.cluster.server.RaftServer;
+import org.apache.thrift.async.TAsyncClientManager;
+import org.apache.thrift.async.TAsyncMethodCall;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.transport.TNonblockingSocket;
+import org.apache.thrift.transport.TNonblockingTransport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Notice: Because a client will be returned to a pool immediately after a successful request, you
+ * should not cache it anywhere else or there may be conflicts.
+ */
+// the two classes does not share a common parent and Java does not allow multiple extension
+@SuppressWarnings("common-java:DuplicatedBlocks")
+public class AsyncDataClient extends AsyncClient {
+
+  private static final Logger logger = LoggerFactory.getLogger(AsyncDataClient.class);
+
+  Node node;
+  AsyncClientPool pool;
+
+  public AsyncDataClient(TProtocolFactory protocolFactory,
+      TAsyncClientManager clientManager,
+      TNonblockingTransport transport) {
+    super(protocolFactory, clientManager, transport);
+  }
+
+  public AsyncDataClient(TProtocolFactory protocolFactory,
+      TAsyncClientManager clientManager, Node node, AsyncClientPool pool) throws IOException {
+    // the difference of the two clients lies in the port
+    super(protocolFactory, clientManager, new TNonblockingSocket(node.getIp(), node.getDataPort()
+        , RaftServer.getConnectionTimeoutInMS()));
+    this.node = node;
+    this.pool = pool;
+  }
+
+  @Override
+  public void onComplete() {
+    super.onComplete();
+    // return itself to the pool if the job is done
+    if (pool != null) {
+      pool.putClient(node, this);
+      pool.onComplete(node);
+    }
+
+  }
+
+  @SuppressWarnings("squid:S1135")
+  @Override
+  public void onError(Exception e) {
+    super.onError(e);
+    if (pool != null) {
+      pool.recreateClient(node);
+      //TODO: if e instance of network failure
+      pool.onError(node);
+    }
+  }
+
+  public void close() {
+    ___transport.close();
+    ___currentMethod = null;
+  }
+
+  public static class FactoryAsync extends AsyncClientFactory {
+
+    public FactoryAsync(org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
+      this.protocolFactory = protocolFactory;
+    }
+
+    @Override
+    public RaftService.AsyncClient getAsyncClient(Node node, AsyncClientPool pool)
+        throws IOException {
+      TAsyncClientManager manager = managers[clientCnt.incrementAndGet() % managers.length];
+      manager = manager == null ? new TAsyncClientManager() : manager;
+      return new AsyncDataClient(protocolFactory, manager, node, pool);
+    }
+  }
+
+  public static class SingleManagerFactory extends AsyncClientFactory {
+
+    private TAsyncClientManager manager;
+
+    public SingleManagerFactory(org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
+      this.protocolFactory = protocolFactory;
+      try {
+        manager = new TAsyncClientManager();
+      } catch (IOException e) {
+        logger.error("Cannot init manager of SingleThreadFactoryAsync", e);
+      }
+    }
+
+    @Override
+    public RaftService.AsyncClient getAsyncClient(Node node, AsyncClientPool pool)
+        throws IOException {
+      return new AsyncDataClient(protocolFactory, manager, node, pool);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "DataClient{" +
+        "node=" + node +
+        '}';
+  }
+
+  public Node getNode() {
+    return node;
+  }
+
+  public boolean isReady() {
+    if (___currentMethod != null) {
+      logger.warn("Client {} is running {} and will timeout at {}", hashCode(), ___currentMethod,
+          new Date(___currentMethod.getTimeoutTimestamp()));
+    }
+    return ___currentMethod == null && !hasError();
+  }
+
+  TAsyncMethodCall<Object> getCurrMethod() {
+    return ___currentMethod;
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncDataHeartbeatClient.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncDataHeartbeatClient.java
new file mode 100644
index 0000000..5f82136
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncDataHeartbeatClient.java
@@ -0,0 +1,69 @@
+/*
+ * 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.client.async;
+
+import java.io.IOException;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService;
+import org.apache.iotdb.cluster.server.RaftServer;
+import org.apache.iotdb.cluster.utils.ClusterUtils;
+import org.apache.thrift.async.TAsyncClientManager;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.transport.TNonblockingSocket;
+
+/**
+ * Notice: Because a client will be returned to a pool immediately after a successful request, you
+ * should not cache it anywhere else or there may be conflicts.
+ */
+public class AsyncDataHeartbeatClient extends AsyncDataClient {
+
+  private AsyncDataHeartbeatClient(TProtocolFactory protocolFactory,
+      TAsyncClientManager clientManager, Node node, AsyncClientPool pool) throws IOException {
+    super(protocolFactory, clientManager, new TNonblockingSocket(node.getIp(),
+        node.getDataPort() + ClusterUtils.DATA_HEARTBEAT_PORT_OFFSET
+        , RaftServer.getConnectionTimeoutInMS()));
+    this.node = node;
+    this.pool = pool;
+  }
+
+  public static class FactoryAsync extends AsyncClientFactory {
+
+    public FactoryAsync(TProtocolFactory protocolFactory) {
+      this.protocolFactory = protocolFactory;
+    }
+
+    @Override
+    public RaftService.AsyncClient getAsyncClient(Node node, AsyncClientPool pool)
+        throws IOException {
+      TAsyncClientManager manager = managers[clientCnt.incrementAndGet() % managers.length];
+      manager = manager == null ? new TAsyncClientManager() : manager;
+      return new AsyncDataHeartbeatClient(protocolFactory, manager, node, pool);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "AsyncDataHeartbeatClient{" +
+        "node=" + super.getNode() + "," +
+        "dataHeartbeatPort=" + (super.getNode().getDataPort()
+        + ClusterUtils.DATA_HEARTBEAT_PORT_OFFSET) +
+        '}';
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncMetaClient.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncMetaClient.java
new file mode 100644
index 0000000..17ad8e3
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncMetaClient.java
@@ -0,0 +1,125 @@
+/*
+ * 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.client.async;
+
+import java.io.IOException;
+import java.util.Date;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService;
+import org.apache.iotdb.cluster.rpc.thrift.TSMetaService.AsyncClient;
+import org.apache.iotdb.cluster.server.RaftServer;
+import org.apache.thrift.async.TAsyncClientManager;
+import org.apache.thrift.async.TAsyncMethodCall;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.transport.TNonblockingSocket;
+import org.apache.thrift.transport.TNonblockingTransport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Notice: Because a client will be returned to a pool immediately after a successful request, you
+ * should not cache it anywhere else or there may be conflicts.
+ */
+// the two classes does not share a common parent and Java does not allow multiple extension
+@SuppressWarnings("common-java:DuplicatedBlocks")
+public class AsyncMetaClient extends AsyncClient {
+
+  private static final Logger logger = LoggerFactory.getLogger(AsyncMetaClient.class);
+  Node node;
+  AsyncClientPool pool;
+
+  public AsyncMetaClient(TProtocolFactory protocolFactory,
+      TAsyncClientManager clientManager,
+      TNonblockingTransport transport) {
+    super(protocolFactory, clientManager, transport);
+  }
+
+  public AsyncMetaClient(TProtocolFactory protocolFactory,
+      TAsyncClientManager clientManager, Node node, AsyncClientPool pool) throws IOException {
+    // the difference of the two clients lies in the port
+    super(protocolFactory, clientManager, new TNonblockingSocket(node.getIp(), node.getMetaPort(),
+        RaftServer.getConnectionTimeoutInMS()));
+    this.node = node;
+    this.pool = pool;
+  }
+
+  @Override
+  public void onComplete() {
+    super.onComplete();
+    // return itself to the pool if the job is done
+    if (pool != null) {
+      pool.putClient(node, this);
+      pool.onComplete(node);
+    }
+  }
+
+  @SuppressWarnings("squid:S1135")
+  @Override
+  public void onError(Exception e) {
+    super.onError(e);
+    pool.recreateClient(node);
+    //TODO: if e instance of network failure
+    pool.onError(node);
+  }
+
+  public static class FactoryAsync extends AsyncClientFactory {
+
+    public FactoryAsync(org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
+      this.protocolFactory = protocolFactory;
+    }
+
+    @Override
+    public RaftService.AsyncClient getAsyncClient(Node node, AsyncClientPool pool)
+        throws IOException {
+      TAsyncClientManager manager = managers[clientCnt.incrementAndGet() % managers.length];
+      manager = manager == null ? new TAsyncClientManager() : manager;
+      return new AsyncMetaClient(protocolFactory, manager, node, pool);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "MetaClient{" +
+        "node=" + node +
+        '}';
+  }
+
+
+  public void close() {
+    ___transport.close();
+    ___currentMethod = null;
+  }
+
+  public Node getNode() {
+    return node;
+  }
+
+  public boolean isReady() {
+    if (___currentMethod != null) {
+      logger.warn("Client {} is running {} and will timeout at {}", hashCode(), ___currentMethod,
+          new Date(___currentMethod.getTimeoutTimestamp()));
+    }
+    return ___currentMethod == null;
+  }
+
+  TAsyncMethodCall<Object> getCurrMethod() {
+    return ___currentMethod;
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncMetaHeartbeatClient.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncMetaHeartbeatClient.java
new file mode 100644
index 0000000..0cc1016
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncMetaHeartbeatClient.java
@@ -0,0 +1,70 @@
+/*
+ * 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.client.async;
+
+import java.io.IOException;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService;
+import org.apache.iotdb.cluster.server.RaftServer;
+import org.apache.iotdb.cluster.utils.ClusterUtils;
+import org.apache.thrift.async.TAsyncClientManager;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.transport.TNonblockingSocket;
+
+/**
+ * Notice: Because a client will be returned to a pool immediately after a successful request, you
+ * should not cache it anywhere else or there may be conflicts.
+ */
+public class AsyncMetaHeartbeatClient extends AsyncMetaClient {
+
+  private AsyncMetaHeartbeatClient(TProtocolFactory protocolFactory,
+      TAsyncClientManager clientManager, Node node, AsyncClientPool pool) throws IOException {
+    super(protocolFactory, clientManager, new TNonblockingSocket(node.getIp(),
+        node.getMetaPort() + ClusterUtils.DATA_HEARTBEAT_PORT_OFFSET
+        , RaftServer.getConnectionTimeoutInMS()));
+    this.node = node;
+    this.pool = pool;
+  }
+
+  public static class FactoryAsync extends AsyncClientFactory {
+
+    public FactoryAsync(TProtocolFactory protocolFactory) {
+      this.protocolFactory = protocolFactory;
+    }
+
+    @Override
+    public RaftService.AsyncClient getAsyncClient(Node node, AsyncClientPool pool)
+        throws IOException {
+      TAsyncClientManager manager = managers[clientCnt.incrementAndGet() % managers.length];
+      manager = manager == null ? new TAsyncClientManager() : manager;
+      return new AsyncMetaHeartbeatClient(protocolFactory, manager, node, pool);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "AsyncMetaHeartbeatClient{" +
+        "node=" + super.getNode() + "," +
+        "metaHeartbeatPort=" + (super.getNode().getMetaPort()
+        + ClusterUtils.META_HEARTBEAT_PORT_OFFSET) +
+        '}';
+  }
+
+}
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
new file mode 100644
index 0000000..b4436f5
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
@@ -0,0 +1,456 @@
+/*
+ * 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.client.sync;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.iotdb.cluster.client.async.AsyncDataClient;
+import org.apache.iotdb.cluster.client.async.AsyncMetaClient;
+import org.apache.iotdb.cluster.log.Snapshot;
+import org.apache.iotdb.cluster.log.snapshot.SnapshotFactory;
+import org.apache.iotdb.cluster.rpc.thrift.AddNodeResponse;
+import org.apache.iotdb.cluster.rpc.thrift.CheckStatusResponse;
+import org.apache.iotdb.cluster.rpc.thrift.ExecutNonQueryReq;
+import org.apache.iotdb.cluster.rpc.thrift.GetAggrResultRequest;
+import org.apache.iotdb.cluster.rpc.thrift.GetAllPathsResult;
+import org.apache.iotdb.cluster.rpc.thrift.GroupByRequest;
+import org.apache.iotdb.cluster.rpc.thrift.LastQueryRequest;
+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.RaftService.AsyncClient;
+import org.apache.iotdb.cluster.rpc.thrift.SingleSeriesQueryRequest;
+import org.apache.iotdb.cluster.rpc.thrift.StartUpStatus;
+import org.apache.iotdb.cluster.rpc.thrift.TNodeStatus;
+import org.apache.iotdb.cluster.server.RaftServer;
+import org.apache.iotdb.cluster.server.handlers.caller.GenericHandler;
+import org.apache.iotdb.cluster.server.handlers.caller.GetChildNodeNextLevelPathHandler;
+import org.apache.iotdb.cluster.server.handlers.caller.GetNodesListHandler;
+import org.apache.iotdb.cluster.server.handlers.caller.GetTimeseriesSchemaHandler;
+import org.apache.iotdb.cluster.server.handlers.caller.JoinClusterHandler;
+import org.apache.iotdb.cluster.server.handlers.caller.PullMeasurementSchemaHandler;
+import org.apache.iotdb.cluster.server.handlers.caller.PullSnapshotHandler;
+import org.apache.iotdb.cluster.server.handlers.caller.PullTimeseriesSchemaHandler;
+import org.apache.iotdb.cluster.server.handlers.forwarder.ForwardPlanHandler;
+import org.apache.iotdb.cluster.utils.PlanSerializer;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.utils.SerializeUtils;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.iotdb.tsfile.read.filter.TimeFilter;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
+import org.apache.iotdb.tsfile.read.filter.operator.AndFilter;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * SyncClientAdaptor convert the async of AsyncClient method call to a sync one by synchronizing on
+ * an AtomicReference of the return value of an RPC, and wait for at most connectionTimeoutInMS
+ * until the reference is set by the handler or the request timeouts.
+ */
+@SuppressWarnings("java:S2274") // enable timeout
+public class SyncClientAdaptor {
+
+  private static final Logger logger = LoggerFactory.getLogger(SyncClientAdaptor.class);
+
+  private SyncClientAdaptor() {
+    // static class
+  }
+
+  public static Long removeNode(AsyncMetaClient asyncMetaClient, Node nodeToRemove)
+      throws TException, InterruptedException {
+    AtomicReference<Long> responseRef = new AtomicReference<>();
+    GenericHandler<Long> handler = new GenericHandler<>(asyncMetaClient.getNode(), responseRef);
+    asyncMetaClient.removeNode(nodeToRemove, handler);
+    synchronized (responseRef) {
+      if (responseRef.get() == null) {
+        responseRef.wait(RaftServer.getConnectionTimeoutInMS());
+      }
+    }
+    return responseRef.get();
+  }
+
+  public static Boolean matchTerm(AsyncClient client, Node target, long prevLogIndex,
+      long prevLogTerm, Node header) throws TException, InterruptedException {
+    try {
+      AtomicReference<Boolean> resultRef = new AtomicReference<>(null);
+      GenericHandler<Boolean> matchTermHandler = new GenericHandler<>(target, resultRef);
+
+      client.matchTerm(prevLogIndex, prevLogTerm, header, matchTermHandler);
+      synchronized (resultRef) {
+        if (resultRef.get() == null) {
+          resultRef.wait(RaftServer.getConnectionTimeoutInMS());
+        }
+      }
+      return resultRef.get();
+    } catch (NullPointerException e) {
+      logger.error("match term null exception", e);
+      return false;
+    }
+  }
+
+  public static Long querySingleSeriesByTimestamp(AsyncDataClient client,
+      SingleSeriesQueryRequest request)
+      throws TException, InterruptedException {
+    AtomicReference<Long> result = new AtomicReference<>();
+    GenericHandler<Long> handler = new GenericHandler<>(client.getNode(), result);
+
+    client.querySingleSeriesByTimestamp(request, handler);
+    synchronized (result) {
+      if (result.get() == null && handler.getException() == null) {
+        result.wait(RaftServer.getReadOperationTimeoutMS());
+      }
+    }
+    return result.get();
+  }
+
+  public static Long querySingleSeries(AsyncDataClient client, SingleSeriesQueryRequest request,
+      long timeOffset) throws TException, InterruptedException {
+    AtomicReference<Long> result = new AtomicReference<>();
+    GenericHandler<Long> handler = new GenericHandler<>(client.getNode(), result);
+    Filter newFilter;
+    // add timestamp to as a timeFilter to skip the data which has been read
+    if (request.isSetTimeFilterBytes()) {
+      Filter timeFilter = FilterFactory.deserialize(request.timeFilterBytes);
+      newFilter = new AndFilter(timeFilter, TimeFilter.gt(timeOffset));
+    } else {
+      newFilter = TimeFilter.gt(timeOffset);
+    }
+    request.setTimeFilterBytes(SerializeUtils.serializeFilter(newFilter));
+
+    client.querySingleSeries(request, handler);
+    synchronized (result) {
+      if (result.get() == null && handler.getException() == null) {
+        result.wait(RaftServer.getReadOperationTimeoutMS());
+      }
+    }
+    return result.get();
+  }
+
+  public static List<String> getNodeList(AsyncDataClient client, Node header,
+      String schemaPattern, int level) throws TException, InterruptedException {
+    GetNodesListHandler handler = new GetNodesListHandler();
+    AtomicReference<List<String>> response = new AtomicReference<>(null);
+    handler.setResponse(response);
+    handler.setContact(client.getNode());
+
+    client.getNodeList(header, schemaPattern, level, handler);
+    synchronized (response) {
+      if (response.get() == null) {
+        response.wait(RaftServer.getReadOperationTimeoutMS());
+      }
+    }
+    return response.get();
+  }
+
+  public static Set<String> getNextChildren(AsyncDataClient client, Node header, String path)
+      throws TException, InterruptedException {
+    GetChildNodeNextLevelPathHandler handler = new GetChildNodeNextLevelPathHandler();
+    AtomicReference<Set<String>> response = new AtomicReference<>(null);
+    handler.setResponse(response);
+    handler.setContact(client.getNode());
+
+    client.getChildNodePathInNextLevel(header, path, handler);
+    synchronized (response) {
+      if (response.get() == null) {
+        response.wait(RaftServer.getReadOperationTimeoutMS());
+      }
+    }
+    return response.get();
+  }
+
+  public static ByteBuffer getAllMeasurementSchema(AsyncDataClient client,
+      Node header, ShowTimeSeriesPlan plan)
+      throws IOException, InterruptedException, TException {
+    GetTimeseriesSchemaHandler handler = new GetTimeseriesSchemaHandler();
+    AtomicReference<ByteBuffer> response = new AtomicReference<>(null);
+    handler.setResponse(response);
+    handler.setContact(client.getNode());
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+    DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
+    plan.serialize(dataOutputStream);
+
+    client.getAllMeasurementSchema(header, ByteBuffer.wrap(byteArrayOutputStream.toByteArray()),
+        handler);
+    synchronized (response) {
+      if (response.get() == null) {
+        response.wait(RaftServer.getReadOperationTimeoutMS());
+      }
+    }
+    return response.get();
+  }
+
+  public static TNodeStatus queryNodeStatus(AsyncMetaClient client)
+      throws TException, InterruptedException {
+    AtomicReference<TNodeStatus> resultRef = new AtomicReference<>();
+    GenericHandler<TNodeStatus> handler = new GenericHandler<>(client.getNode(), resultRef);
+
+    client.queryNodeStatus(handler);
+    synchronized (resultRef) {
+      if (resultRef.get() == null) {
+        resultRef.wait(RaftServer.getReadOperationTimeoutMS());
+      }
+    }
+    if (handler.getException() != null) {
+      throw new TException(handler.getException());
+    }
+    return resultRef.get();
+  }
+
+  public static CheckStatusResponse checkStatus(AsyncMetaClient client, StartUpStatus startUpStatus)
+      throws TException, InterruptedException {
+    AtomicReference<CheckStatusResponse> resultRef = new AtomicReference<>();
+    GenericHandler<CheckStatusResponse> handler = new GenericHandler<>(client.getNode(), resultRef);
+
+    client.checkStatus(startUpStatus, handler);
+    synchronized (resultRef) {
+      if (resultRef.get() == null) {
+        resultRef.wait(RaftServer.getReadOperationTimeoutMS());
+      }
+    }
+    if (handler.getException() != null) {
+      throw new TException(handler.getException());
+    }
+    return resultRef.get();
+  }
+
+  public static AddNodeResponse addNode(AsyncMetaClient client, Node thisNode,
+      StartUpStatus startUpStatus)
+      throws TException, InterruptedException {
+    JoinClusterHandler handler = new JoinClusterHandler();
+    AtomicReference<AddNodeResponse> response = new AtomicReference<>(null);
+    handler.setResponse(response);
+    handler.setContact(client.getNode());
+
+    client.addNode(thisNode, startUpStatus, handler);
+    synchronized (response) {
+      if (response.get() == null) {
+        response.wait(60 * 1000L);
+      }
+    }
+    return response.get();
+  }
+
+  public static List<MeasurementSchema> pullMeasurementSchema(AsyncDataClient client,
+      PullSchemaRequest pullSchemaRequest) throws TException, InterruptedException {
+    AtomicReference<List<MeasurementSchema>> measurementSchemas = new AtomicReference<>();
+
+    client.pullMeasurementSchema(pullSchemaRequest,
+        new PullMeasurementSchemaHandler(client.getNode(), pullSchemaRequest.getPrefixPaths(),
+            measurementSchemas));
+    synchronized (measurementSchemas) {
+      if (measurementSchemas.get() == null) {
+        measurementSchemas.wait(RaftServer.getReadOperationTimeoutMS());
+      }
+    }
+    return measurementSchemas.get();
+  }
+
+  public static List<TimeseriesSchema> pullTimeseriesSchema(AsyncDataClient client,
+      PullSchemaRequest pullSchemaRequest) throws TException, InterruptedException {
+    AtomicReference<List<TimeseriesSchema>> timeseriesSchemas = new AtomicReference<>();
+    client.pullTimeSeriesSchema(pullSchemaRequest,
+        new PullTimeseriesSchemaHandler(client.getNode(), pullSchemaRequest.getPrefixPaths(),
+            timeseriesSchemas));
+
+    synchronized (timeseriesSchemas) {
+      if (timeseriesSchemas.get() == null) {
+        timeseriesSchemas.wait(RaftServer.getReadOperationTimeoutMS());
+      }
+    }
+    return timeseriesSchemas.get();
+  }
+
+  public static List<ByteBuffer> getAggrResult(AsyncDataClient client, GetAggrResultRequest request)
+      throws TException, InterruptedException {
+    AtomicReference<List<ByteBuffer>> resultReference = new AtomicReference<>();
+    GenericHandler<List<ByteBuffer>> handler = new GenericHandler<>(client.getNode(),
+        resultReference);
+
+    client.getAggrResult(request, handler);
+    synchronized (resultReference) {
+      if (resultReference.get() == null) {
+        resultReference.wait(RaftServer.getReadOperationTimeoutMS());
+      }
+    }
+    if (handler.getException() != null) {
+      throw new TException(handler.getException());
+    }
+    return resultReference.get();
+  }
+
+  public static List<String> getUnregisteredMeasurements(AsyncDataClient client, Node header,
+      List<String> seriesPaths) throws TException, InterruptedException {
+    AtomicReference<List<String>> remoteResult = new AtomicReference<>();
+    GenericHandler<List<String>> handler = new GenericHandler<>(client.getNode(), remoteResult);
+
+    client.getUnregisteredTimeseries(header, seriesPaths, handler);
+    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+  }
+
+  public static GetAllPathsResult getAllPaths(AsyncDataClient client, Node header,
+      List<String> pathsToQuery, boolean withAlias)
+      throws InterruptedException, TException {
+    AtomicReference<GetAllPathsResult> remoteResult = new AtomicReference<>();
+    GenericHandler<GetAllPathsResult> handler = new GenericHandler<>(client.getNode(),
+        remoteResult);
+
+    client.getAllPaths(header, pathsToQuery, withAlias, handler);
+    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+  }
+
+  public static Integer getPathCount(AsyncDataClient client, Node header, List<String> pathsToQuery,
+      int level)
+      throws InterruptedException, TException {
+    AtomicReference<Integer> remoteResult = new AtomicReference<>(null);
+    GenericHandler<Integer> handler = new GenericHandler<>(client.getNode(), remoteResult);
+
+    client.getPathCount(header, pathsToQuery, level, handler);
+    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+  }
+
+  public static Set<String> getAllDevices(AsyncDataClient client, Node header,
+      List<String> pathsToQuery)
+      throws InterruptedException, TException {
+    AtomicReference<Set<String>> remoteResult = new AtomicReference<>();
+    GenericHandler<Set<String>> handler = new GenericHandler<>(client.getNode(), remoteResult);
+
+    client.getAllDevices(header, pathsToQuery, handler);
+    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+  }
+
+  public static Long getGroupByExecutor(AsyncDataClient client, GroupByRequest request)
+      throws TException, InterruptedException {
+    AtomicReference<Long> result = new AtomicReference<>();
+    GenericHandler<Long> handler = new GenericHandler<>(client.getNode(), result);
+
+    client.getGroupByExecutor(request, handler);
+    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+  }
+
+  public static ByteBuffer previousFill(AsyncDataClient client, PreviousFillRequest request)
+      throws TException, InterruptedException {
+    AtomicReference<ByteBuffer> resultRef = new AtomicReference<>();
+    GenericHandler<ByteBuffer> nodeHandler = new GenericHandler<>(client.getNode(), resultRef);
+
+    client.previousFill(request, nodeHandler);
+    return nodeHandler.getResult(RaftServer.getReadOperationTimeoutMS());
+  }
+
+  public static TSStatus executeNonQuery(AsyncClient client, PhysicalPlan plan, Node header,
+      Node receiver) throws IOException, TException, InterruptedException {
+    AtomicReference<TSStatus> status = new AtomicReference<>();
+    ExecutNonQueryReq req = new ExecutNonQueryReq();
+    req.planBytes = ByteBuffer.wrap(PlanSerializer.getInstance().serialize(plan));
+    if (header != null) {
+      req.setHeader(header);
+    }
+
+    client.executeNonQueryPlan(req, new ForwardPlanHandler(status, plan, receiver));
+    synchronized (status) {
+      if (status.get() == null) {
+        status.wait(RaftServer.getWriteOperationTimeoutMS());
+      }
+    }
+    return status.get();
+  }
+
+  public static ByteBuffer readFile(AsyncDataClient client, String remotePath, long offset,
+      int fetchSize)
+      throws InterruptedException, TException {
+    AtomicReference<ByteBuffer> result = new AtomicReference<>();
+    GenericHandler<ByteBuffer> handler = new GenericHandler<>(client.getNode(), result);
+
+    client.readFile(remotePath, offset, fetchSize, handler);
+    return handler.getResult(RaftServer.getWriteOperationTimeoutMS());
+  }
+
+  public static List<ByteBuffer> getGroupByResult(AsyncDataClient client, Node 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);
+
+    client.getGroupByResult(header, executorId, curStartTime, curEndTime, handler);
+    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+  }
+
+  public static ByteBuffer peekNextNotNullValue(AsyncDataClient client, Node header,
+      long executorId
+      , long curStartTime, long curEndTime) throws InterruptedException, TException {
+    AtomicReference<ByteBuffer> fetchResult = new AtomicReference<>();
+    GenericHandler<ByteBuffer> handler = new GenericHandler<>(client.getNode(), fetchResult);
+
+    client.peekNextNotNullValue(header, executorId, curStartTime, curEndTime, handler);
+    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+  }
+
+  public static <T extends Snapshot> Map<Integer, T> pullSnapshot(AsyncDataClient client,
+      PullSnapshotRequest request, List<Integer> slots, SnapshotFactory<T> factory)
+      throws TException, InterruptedException {
+    AtomicReference<Map<Integer, T>> snapshotRef = new AtomicReference<>();
+
+    client.pullSnapshot(request, new PullSnapshotHandler<>(snapshotRef,
+        client.getNode(), slots, factory));
+    synchronized (snapshotRef) {
+      if (snapshotRef.get() == null) {
+        snapshotRef.wait(RaftServer.getReadOperationTimeoutMS());
+      }
+    }
+    return snapshotRef.get();
+  }
+
+  public static ByteBuffer last(AsyncDataClient client, List<PartialPath> seriesPaths,
+      List<Integer> dataTypeOrdinals, QueryContext context,
+      Map<String, Set<String>> deviceMeasurements,
+      Node header)
+      throws TException, InterruptedException {
+    AtomicReference<ByteBuffer> result = new AtomicReference<>();
+    GenericHandler<ByteBuffer> handler = new GenericHandler<>(client.getNode(), result);
+    LastQueryRequest request = new LastQueryRequest(PartialPath.toStringList(seriesPaths),
+        dataTypeOrdinals,
+        context.getQueryId(), deviceMeasurements, header, client.getNode());
+
+    client.last(request, handler);
+    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+  }
+
+  public static boolean onSnapshotApplied(AsyncDataClient client, Node header, List<Integer> slots)
+      throws TException, InterruptedException {
+    AtomicReference<Boolean> result = new AtomicReference<>(false);
+    GenericHandler<Boolean> handler = new GenericHandler<>(client.getNode(), result);
+
+    client.onSnapshotApplied(header, slots, handler);
+    return handler.getResult(RaftServer.getWriteOperationTimeoutMS());
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java
similarity index 56%
copy from server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java
index 5ff1e3b..fc70b26 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java
@@ -16,19 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.exception;
 
-import org.apache.iotdb.rpc.TSStatusCode;
+package org.apache.iotdb.cluster.client.sync;
 
-public class TsFileProcessorException extends IoTDBException {
+import java.io.IOException;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService;
+import org.apache.thrift.transport.TTransportException;
 
-  private static final long serialVersionUID = 3749107630243950925L;
+public interface SyncClientFactory {
 
-  public TsFileProcessorException(String message) {
-    super(message, TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
-  }
-
-  public TsFileProcessorException(Exception exception) {
-    super(exception.getMessage(), TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
-  }
+  /**
+   * Get a client which will connect the given node and be cached in the given pool.
+   * @param node the cluster node the client will connect.
+   * @param pool the pool that will cache the client for reusing.
+   * @return
+   * @throws IOException
+   */
+  RaftService.Client getSyncClient(Node node, SyncClientPool pool) throws TTransportException;
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientPool.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientPool.java
new file mode 100644
index 0000000..535a815
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientPool.java
@@ -0,0 +1,137 @@
+/*
+ * 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.client.sync;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.SocketTimeoutException;
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService.Client;
+import org.apache.iotdb.cluster.utils.ClusterNode;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SyncClientPool {
+
+  private static final Logger logger = LoggerFactory.getLogger(
+      SyncClientPool.class);
+  private static final long WAIT_CLIENT_TIMEOUT_MS = 5 * 1000L;
+  private int maxConnectionForEachNode;
+  private Map<ClusterNode, Deque<Client>> clientCaches = new ConcurrentHashMap<>();
+  private Map<ClusterNode, Integer> nodeClientNumMap = new ConcurrentHashMap<>();
+  private SyncClientFactory syncClientFactory;
+
+  public SyncClientPool(SyncClientFactory syncClientFactory) {
+    this.syncClientFactory = syncClientFactory;
+    this.maxConnectionForEachNode =
+        ClusterDescriptor.getInstance().getConfig().getMaxClientPerNodePerMember();
+  }
+
+  /**
+   * Get a client of the given node from the cache if one is available, or create a new one.
+   * @param node
+   * @return
+   * @throws IOException
+   */
+  public Client getClient(Node node) {
+    ClusterNode clusterNode = new ClusterNode(node);
+    //As clientCaches is ConcurrentHashMap, computeIfAbsent is thread safety.
+    Deque<Client> clientStack = clientCaches.computeIfAbsent(clusterNode, n -> new ArrayDeque<>());
+    synchronized (this) {
+      if (clientStack.isEmpty()) {
+        int nodeClientNum = nodeClientNumMap.getOrDefault(clusterNode, 0);
+        if (nodeClientNum >= maxConnectionForEachNode) {
+          return waitForClient(clientStack, clusterNode, nodeClientNum);
+        } else {
+          nodeClientNumMap.put(clusterNode, nodeClientNum + 1);
+          return createClient(clusterNode, nodeClientNum);
+        }
+      } else {
+        return clientStack.pop();
+      }
+    }
+  }
+
+  @SuppressWarnings("squid:S2273") // synchronized outside
+  private Client waitForClient(Deque<Client> clientStack, ClusterNode node, int nodeClientNum) {
+    // wait for an available client
+    long waitStart = System.currentTimeMillis();
+    while (clientStack.isEmpty()) {
+      try {
+        this.wait(WAIT_CLIENT_TIMEOUT_MS);
+        if (clientStack.isEmpty() && System.currentTimeMillis() - waitStart >= WAIT_CLIENT_TIMEOUT_MS) {
+          logger.warn("Cannot get an available client after {}ms, create a new one",
+              WAIT_CLIENT_TIMEOUT_MS);
+          nodeClientNumMap.put(node, nodeClientNum + 1);
+          return createClient(node, nodeClientNum);
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        logger.warn("Interrupted when waiting for an available client of {}", node);
+        return null;
+      }
+    }
+    return clientStack.pop();
+  }
+
+  /**
+   * Return a client of a node to the pool. Closed client should not be returned.
+   * @param node
+   * @param client
+   */
+  void putClient(Node node, Client client) {
+    ClusterNode clusterNode = new ClusterNode(node);
+    //As clientCaches is ConcurrentHashMap, computeIfAbsent is thread safety.
+    Deque<Client> clientStack = clientCaches.computeIfAbsent(clusterNode, n -> new ArrayDeque<>());
+    synchronized (this) {
+      if (client.getInputProtocol() != null && client.getInputProtocol().getTransport().isOpen()) {
+        clientStack.push(client);
+      } else {
+        try {
+          clientStack.push(syncClientFactory.getSyncClient(node, this));
+        } catch (TTransportException e) {
+          logger.error("Cannot open transport for client", e);
+          nodeClientNumMap.computeIfPresent(clusterNode, (n, oldValue) -> oldValue - 1);
+        }
+      }
+      this.notifyAll();
+    }
+  }
+
+  private Client createClient(ClusterNode node, int nodeClientNum) {
+    try {
+      return syncClientFactory.getSyncClient(node, this);
+    } catch (TTransportException e) {
+      if (e.getCause() instanceof ConnectException || e.getCause() instanceof SocketTimeoutException) {
+        logger.debug("Cannot open transport for client {} : {}", node, e.getMessage());
+      } else {
+        logger.error("Cannot open transport for client {}", node, e);
+      }
+      nodeClientNumMap.put(node, nodeClientNum);
+      return null;
+    }
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncDataClient.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncDataClient.java
new file mode 100644
index 0000000..23e084c
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncDataClient.java
@@ -0,0 +1,105 @@
+/*
+ * 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.client.sync;
+
+import java.net.SocketException;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.TSDataService.Client;
+import org.apache.iotdb.cluster.server.RaftServer;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.rpc.RpcTransportFactory;
+import org.apache.iotdb.rpc.TimeoutChangeableTransport;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransportException;
+
+/**
+ * Notice: Because a client will be returned to a pool immediately after a successful request, you
+ * should not cache it anywhere else or there may be conflicts.
+ */
+// the two classes does not share a common parent and Java does not allow multiple extension
+@SuppressWarnings("common-java:DuplicatedBlocks")
+public class SyncDataClient extends Client {
+
+  Node node;
+  SyncClientPool pool;
+
+  public SyncDataClient(TProtocol prot) {
+    super(prot);
+  }
+
+  public SyncDataClient(TProtocolFactory protocolFactory, Node node, SyncClientPool pool)
+      throws TTransportException {
+    // the difference of the two clients lies in the port
+    super(protocolFactory.getProtocol(RpcTransportFactory.INSTANCE.getTransport(
+        new TSocket(node.getIp(), node.getDataPort(), RaftServer.getConnectionTimeoutInMS()))));
+    this.node = node;
+    this.pool = pool;
+    getInputProtocol().getTransport().open();
+  }
+
+  public void setTimeout(int timeout) {
+    // the same transport is used in both input and output
+    ((TimeoutChangeableTransport) (getInputProtocol().getTransport()))
+        .setTimeout(timeout);
+  }
+
+  @TestOnly
+  public int getTimeout() throws SocketException {
+    return ((TimeoutChangeableTransport) getInputProtocol().getTransport()).getTimeOut();
+  }
+
+  public void putBack() {
+    if (pool != null) {
+      pool.putClient(node, this);
+    } else {
+      TProtocol inputProtocol = getInputProtocol();
+      if (inputProtocol != null) {
+        inputProtocol.getTransport().close();
+      }
+    }
+  }
+
+  public static class FactorySync implements SyncClientFactory {
+
+    private TProtocolFactory protocolFactory;
+
+    public FactorySync(TProtocolFactory protocolFactory) {
+      this.protocolFactory = protocolFactory;
+    }
+
+    @Override
+    public SyncDataClient getSyncClient(Node node, SyncClientPool pool) throws TTransportException {
+      return new SyncDataClient(protocolFactory, node, pool);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "DataClient{" +
+        "node=" + node +
+        '}';
+  }
+
+  public Node getNode() {
+    return node;
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncDataHeartbeatClient.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncDataHeartbeatClient.java
new file mode 100644
index 0000000..134ec11
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncDataHeartbeatClient.java
@@ -0,0 +1,71 @@
+/*
+ * 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.client.sync;
+
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.server.RaftServer;
+import org.apache.iotdb.cluster.utils.ClusterUtils;
+import org.apache.iotdb.rpc.RpcTransportFactory;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransportException;
+
+/**
+ * Notice: Because a client will be returned to a pool immediately after a successful request, you
+ * should not cache it anywhere else or there may be conflicts.
+ */
+public class SyncDataHeartbeatClient extends SyncDataClient {
+
+  private SyncDataHeartbeatClient(TProtocolFactory protocolFactory, Node node, SyncClientPool pool)
+      throws TTransportException {
+    // the difference of the two clients lies in the port
+    super(protocolFactory.getProtocol(RpcTransportFactory.INSTANCE.getTransport(
+        new TSocket(node.getIp(), node.getDataPort() + ClusterUtils.DATA_HEARTBEAT_PORT_OFFSET,
+            RaftServer.getConnectionTimeoutInMS()))));
+    this.node = node;
+    this.pool = pool;
+    getInputProtocol().getTransport().open();
+  }
+
+
+  public static class FactorySync implements SyncClientFactory {
+
+    private TProtocolFactory protocolFactory;
+
+    public FactorySync(TProtocolFactory protocolFactory) {
+      this.protocolFactory = protocolFactory;
+    }
+
+    @Override
+    public SyncDataHeartbeatClient getSyncClient(Node node, SyncClientPool pool)
+        throws TTransportException {
+      return new SyncDataHeartbeatClient(protocolFactory, node, pool);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "SyncHeartbeatDataClient{" +
+        "node=" + super.getNode() + "," +
+        "dataHeartbeatPort=" + (super.getNode().getDataPort()
+        + ClusterUtils.DATA_HEARTBEAT_PORT_OFFSET) +
+        '}';
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncMetaClient.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncMetaClient.java
new file mode 100644
index 0000000..6e4d450
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncMetaClient.java
@@ -0,0 +1,80 @@
+/*
+ * 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.client.sync;
+
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.TSMetaService.Client;
+import org.apache.iotdb.cluster.server.RaftServer;
+import org.apache.iotdb.rpc.RpcTransportFactory;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransportException;
+
+/**
+ * Notice: Because a client will be returned to a pool immediately after a successful request, you
+ * should not cache it anywhere else or there may be conflicts.
+ */
+// the two classes does not share a common parent and Java does not allow multiple extension
+@SuppressWarnings("common-java:DuplicatedBlocks")
+public class SyncMetaClient extends Client {
+
+  Node node;
+  SyncClientPool pool;
+
+  SyncMetaClient(TProtocol prot) {
+    super(prot);
+  }
+
+  public SyncMetaClient(TProtocolFactory protocolFactory, Node node, SyncClientPool pool)
+      throws TTransportException {
+    super(protocolFactory.getProtocol(RpcTransportFactory.INSTANCE.getTransport(
+        new TSocket(node.getIp(), node.getMetaPort(), RaftServer.getConnectionTimeoutInMS()))));
+    this.node = node;
+    this.pool = pool;
+    getInputProtocol().getTransport().open();
+  }
+
+  public void putBack() {
+    if (pool != null) {
+      pool.putClient(node, this);
+    } else {
+      getInputProtocol().getTransport().close();
+    }
+  }
+
+  public static class FactorySync implements SyncClientFactory {
+
+    private TProtocolFactory protocolFactory;
+
+    public FactorySync(TProtocolFactory protocolFactory) {
+      this.protocolFactory = protocolFactory;
+    }
+
+    @Override
+    public SyncMetaClient getSyncClient(Node node, SyncClientPool pool) throws TTransportException {
+      return new SyncMetaClient(protocolFactory, node, pool);
+    }
+  }
+
+  public Node getNode() {
+    return node;
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncMetaHeartbeatClient.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncMetaHeartbeatClient.java
new file mode 100644
index 0000000..dddc66f
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncMetaHeartbeatClient.java
@@ -0,0 +1,71 @@
+/*
+ * 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.client.sync;
+
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.server.RaftServer;
+import org.apache.iotdb.cluster.utils.ClusterUtils;
+import org.apache.iotdb.rpc.RpcTransportFactory;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransportException;
+
+/**
+ * Notice: Because a client will be returned to a pool immediately after a successful request, you
+ * should not cache it anywhere else or there may be conflicts.
+ */
+public class SyncMetaHeartbeatClient extends SyncMetaClient {
+
+  private SyncMetaHeartbeatClient(TProtocolFactory protocolFactory, Node node, SyncClientPool pool)
+      throws TTransportException {
+    // the difference of the two clients lies in the port
+    super(protocolFactory.getProtocol(RpcTransportFactory.INSTANCE.getTransport(
+        new TSocket(node.getIp(), node.getMetaPort() + ClusterUtils.META_HEARTBEAT_PORT_OFFSET,
+            RaftServer.getConnectionTimeoutInMS()))));
+    this.node = node;
+    this.pool = pool;
+    getInputProtocol().getTransport().open();
+  }
+
+
+  public static class FactorySync implements SyncClientFactory {
+
+    private TProtocolFactory protocolFactory;
+
+    public FactorySync(TProtocolFactory protocolFactory) {
+      this.protocolFactory = protocolFactory;
+    }
+
+    @Override
+    public SyncMetaHeartbeatClient getSyncClient(Node node, SyncClientPool pool)
+        throws TTransportException {
+      return new SyncMetaHeartbeatClient(protocolFactory, node, pool);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "SyncMetaHeartbeatClient{" +
+        "node=" + super.getNode() + "," +
+        "metaHeartbeatPort=" + (super.getNode().getMetaPort()
+        + ClusterUtils.META_HEARTBEAT_PORT_OFFSET) +
+        '}';
+  }
+}
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
new file mode 100644
index 0000000..4d5d05b
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
@@ -0,0 +1,444 @@
+/*
+ * 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.config;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iotdb.cluster.utils.ClusterConsistent;
+
+public class ClusterConfig {
+
+  static final String CONFIG_NAME = "iotdb-cluster.properties";
+
+  private String clusterRpcIp = "127.0.0.1";
+  private int internalMetaPort = 9003;
+  private int internalDataPort = 40010;
+  private int clusterRpcPort = 55560;
+
+  /**
+   * each one is a "<IP | domain name>:<meta port>:<data port>:<client port></>" string tuple
+   */
+  private List<String> seedNodeUrls = Arrays
+      .asList("127.0.0.1:9003:40010:55560", "127.0.0.1:9005:40012:55561",
+          "127.0.0.1:9007:40014:55562");
+
+  @ClusterConsistent
+  private boolean isRpcThriftCompressionEnabled = false;
+  private int maxConcurrentClientNum = 10000;
+
+  @ClusterConsistent
+  private int replicationNum = 2;
+
+  @ClusterConsistent
+  private String clusterName = "default";
+
+  @ClusterConsistent
+  private boolean useAsyncServer = true;
+
+  private boolean useAsyncApplier = true;
+
+  private int connectionTimeoutInMS = 20 * 1000;
+
+  private int readOperationTimeoutMS = 30_1000;
+
+  private int writeOperationTimeoutMS = 30_1000;
+
+  private int catchUpTimeoutMS = 60_000;
+
+  private boolean useBatchInLogCatchUp = true;
+
+  /**
+   * max number of committed logs to be saved
+   */
+  private int minNumOfLogsInMem = 100;
+
+  /**
+   * max number of committed logs in memory
+   */
+  private int maxNumOfLogsInMem = 1000;
+
+  /**
+   * deletion check period of the submitted log
+   */
+  private int logDeleteCheckIntervalSecond = -1;
+
+  /**
+   * max number of clients in a ClientPool of a member for one node.
+   */
+  private int maxClientPerNodePerMember = 1000;
+
+  /**
+   * ClientPool will have so many selector threads (TAsyncClientManager) to distribute to its
+   * clients.
+   */
+  private int selectorNumOfClientPool = Runtime.getRuntime().availableProcessors() / 3 > 0 ?
+      Runtime.getRuntime().availableProcessors() / 3 : 1;
+
+  /**
+   * Whether creating schema automatically is enabled, this will replace the one in
+   * iotdb-engine.properties
+   */
+  private boolean enableAutoCreateSchema = true;
+
+  private boolean enableRaftLogPersistence = true;
+
+
+  private int flushRaftLogThreshold = 10000;
+
+  /**
+   * Size of log buffer. If raft log persistence is enabled and the size of a insert plan is smaller
+   * than this parameter, then the insert plan will be rejected by WAL.
+   */
+  private int raftLogBufferSize = 16 * 1024 * 1024;
+
+  /**
+   * consistency level, now three consistency levels are supported: strong, mid and weak. Strong
+   * consistency means the server will first try to synchronize with the leader to get the newest
+   * meta data, if failed(timeout), directly report an error to the user; While mid consistency
+   * means the server will first try to synchronize with the leader, but if failed(timeout), it will
+   * give up and just use current data it has cached before; Weak consistency do not synchronize
+   * with the leader and simply use the local data
+   */
+  private ConsistencyLevel consistencyLevel = ConsistencyLevel.MID_CONSISTENCY;
+
+  private long joinClusterTimeOutMs = 5000L;
+
+  private int pullSnapshotRetryIntervalMs = 5 * 1000;
+
+  /**
+   * The maximum value of the raft log index stored in the memory per raft group, These indexes are
+   * used to index the location of the log on the disk
+   */
+  private int maxRaftLogIndexSizeInMemory = 10000;
+
+  /**
+   * The maximum size of the raft log saved on disk for each file (in bytes) of each raft group. The
+   * default size is 1GB
+   */
+  private int maxRaftLogPersistDataSizePerFile = 1073741824;
+
+  /**
+   * The maximum number of persistent raft log files on disk per raft group, So each raft group's
+   * log takes up disk space approximately equals max_raft_log_persist_data_size_per_file *
+   * max_number_of_persist_raft_log_files
+   */
+  private int maxNumberOfPersistRaftLogFiles = 5;
+
+  /**
+   * The maximum number of logs saved on the disk
+   */
+  private int maxPersistRaftLogNumberOnDisk = 1_000_000;
+
+
+  private boolean enableUsePersistLogOnDiskToCatchUp = false;
+
+  /**
+   * The number of logs read on the disk at one time, which is mainly used to control the memory
+   * usage.This value multiplied by the log size is about the amount of memory used to read logs
+   * from the disk at one time.
+   */
+  private int maxNumberOfLogsPerFetchOnDisk = 1000;
+
+  /**
+   * When set to true, if the log queue of a follower fills up, LogDispatcher will wait for a
+   * while until the queue becomes available, otherwise LogDispatcher will just ignore that slow
+   * node.
+   */
+  private boolean waitForSlowNode = true;
+
+  public int getSelectorNumOfClientPool() {
+    return selectorNumOfClientPool;
+  }
+
+  public void setSelectorNumOfClientPool(int selectorNumOfClientPool) {
+    this.selectorNumOfClientPool = selectorNumOfClientPool;
+  }
+
+  public int getMaxClientPerNodePerMember() {
+    return maxClientPerNodePerMember;
+  }
+
+  public void setMaxClientPerNodePerMember(int maxClientPerNodePerMember) {
+    this.maxClientPerNodePerMember = maxClientPerNodePerMember;
+  }
+
+  public boolean isUseBatchInLogCatchUp() {
+    return useBatchInLogCatchUp;
+  }
+
+  public void setUseBatchInLogCatchUp(boolean useBatchInLogCatchUp) {
+    this.useBatchInLogCatchUp = useBatchInLogCatchUp;
+  }
+
+  public String getClusterRpcIp() {
+    return clusterRpcIp;
+  }
+
+  void setClusterRpcIp(String clusterRpcIp) {
+    this.clusterRpcIp = clusterRpcIp;
+  }
+
+  public int getInternalMetaPort() {
+    return internalMetaPort;
+  }
+
+  void setInternalMetaPort(int internalMetaPort) {
+    this.internalMetaPort = internalMetaPort;
+  }
+
+  public boolean isRpcThriftCompressionEnabled() {
+    return isRpcThriftCompressionEnabled;
+  }
+
+  void setRpcThriftCompressionEnabled(boolean rpcThriftCompressionEnabled) {
+    isRpcThriftCompressionEnabled = rpcThriftCompressionEnabled;
+  }
+
+  public int getMaxConcurrentClientNum() {
+    return maxConcurrentClientNum;
+  }
+
+  void setMaxConcurrentClientNum(int maxConcurrentClientNum) {
+    this.maxConcurrentClientNum = maxConcurrentClientNum;
+  }
+
+  public List<String> getSeedNodeUrls() {
+    return seedNodeUrls;
+  }
+
+  public void setSeedNodeUrls(List<String> seedNodeUrls) {
+    this.seedNodeUrls = seedNodeUrls;
+  }
+
+  public int getReplicationNum() {
+    return replicationNum;
+  }
+
+  public void setReplicationNum(int replicationNum) {
+    this.replicationNum = replicationNum;
+  }
+
+  void setClusterName(String clusterName) {
+    this.clusterName = clusterName;
+  }
+
+  public String getClusterName() {
+    return clusterName;
+  }
+
+  public int getInternalDataPort() {
+    return internalDataPort;
+  }
+
+  void setInternalDataPort(int internalDataPort) {
+    this.internalDataPort = internalDataPort;
+  }
+
+  public int getClusterRpcPort() {
+    return clusterRpcPort;
+  }
+
+  void setClusterRpcPort(int clusterRpcPort) {
+    this.clusterRpcPort = clusterRpcPort;
+  }
+
+  public int getConnectionTimeoutInMS() {
+    return connectionTimeoutInMS;
+  }
+
+  void setConnectionTimeoutInMS(int connectionTimeoutInMS) {
+    this.connectionTimeoutInMS = connectionTimeoutInMS;
+  }
+
+  public int getCatchUpTimeoutMS() {
+    return catchUpTimeoutMS;
+  }
+
+  public void setCatchUpTimeoutMS(int catchUpTimeoutMS) {
+    this.catchUpTimeoutMS = catchUpTimeoutMS;
+  }
+
+  public int getReadOperationTimeoutMS() {
+    return readOperationTimeoutMS;
+  }
+
+  void setReadOperationTimeoutMS(int readOperationTimeoutMS) {
+    this.readOperationTimeoutMS = readOperationTimeoutMS;
+  }
+
+  public int getWriteOperationTimeoutMS() {
+    return writeOperationTimeoutMS;
+  }
+
+  public void setWriteOperationTimeoutMS(int writeOperationTimeoutMS) {
+    this.writeOperationTimeoutMS = writeOperationTimeoutMS;
+  }
+
+  public int getMinNumOfLogsInMem() {
+    return minNumOfLogsInMem;
+  }
+
+  public void setMinNumOfLogsInMem(int minNumOfLogsInMem) {
+    this.minNumOfLogsInMem = minNumOfLogsInMem;
+  }
+
+  public int getLogDeleteCheckIntervalSecond() {
+    return logDeleteCheckIntervalSecond;
+  }
+
+  void setLogDeleteCheckIntervalSecond(int logDeleteCheckIntervalSecond) {
+    this.logDeleteCheckIntervalSecond = logDeleteCheckIntervalSecond;
+  }
+
+  public ConsistencyLevel getConsistencyLevel() {
+    return consistencyLevel;
+  }
+
+  public void setConsistencyLevel(ConsistencyLevel consistencyLevel) {
+    this.consistencyLevel = consistencyLevel;
+  }
+
+  public boolean isEnableAutoCreateSchema() {
+    return enableAutoCreateSchema;
+  }
+
+  public void setEnableAutoCreateSchema(boolean enableAutoCreateSchema) {
+    this.enableAutoCreateSchema = enableAutoCreateSchema;
+  }
+
+  public boolean isUseAsyncServer() {
+    return useAsyncServer;
+  }
+
+  public void setUseAsyncServer(boolean useAsyncServer) {
+    this.useAsyncServer = useAsyncServer;
+  }
+
+  public boolean isEnableRaftLogPersistence() {
+    return enableRaftLogPersistence;
+  }
+
+  public void setEnableRaftLogPersistence(boolean enableRaftLogPersistence) {
+    this.enableRaftLogPersistence = enableRaftLogPersistence;
+  }
+
+  public boolean isUseAsyncApplier() {
+    return useAsyncApplier;
+  }
+
+  public void setUseAsyncApplier(boolean useAsyncApplier) {
+    this.useAsyncApplier = useAsyncApplier;
+  }
+
+  public int getMaxNumOfLogsInMem() {
+    return maxNumOfLogsInMem;
+  }
+
+  public void setMaxNumOfLogsInMem(int maxNumOfLogsInMem) {
+    this.maxNumOfLogsInMem = maxNumOfLogsInMem;
+  }
+
+  public int getRaftLogBufferSize() {
+    return raftLogBufferSize;
+  }
+
+  public void setRaftLogBufferSize(int raftLogBufferSize) {
+    this.raftLogBufferSize = raftLogBufferSize;
+  }
+
+  public int getFlushRaftLogThreshold() {
+    return flushRaftLogThreshold;
+  }
+
+  void setFlushRaftLogThreshold(int flushRaftLogThreshold) {
+    this.flushRaftLogThreshold = flushRaftLogThreshold;
+  }
+
+  public long getJoinClusterTimeOutMs() {
+    return joinClusterTimeOutMs;
+  }
+
+  public void setJoinClusterTimeOutMs(long joinClusterTimeOutMs) {
+    this.joinClusterTimeOutMs = joinClusterTimeOutMs;
+  }
+
+  public int getPullSnapshotRetryIntervalMs() {
+    return pullSnapshotRetryIntervalMs;
+  }
+
+  public void setPullSnapshotRetryIntervalMs(int pullSnapshotRetryIntervalMs) {
+    this.pullSnapshotRetryIntervalMs = pullSnapshotRetryIntervalMs;
+  }
+
+  public int getMaxRaftLogIndexSizeInMemory() {
+    return maxRaftLogIndexSizeInMemory;
+  }
+
+  public void setMaxRaftLogIndexSizeInMemory(int maxRaftLogIndexSizeInMemory) {
+    this.maxRaftLogIndexSizeInMemory = maxRaftLogIndexSizeInMemory;
+  }
+
+  public int getMaxRaftLogPersistDataSizePerFile() {
+    return maxRaftLogPersistDataSizePerFile;
+  }
+
+  public void setMaxRaftLogPersistDataSizePerFile(int maxRaftLogPersistDataSizePerFile) {
+    this.maxRaftLogPersistDataSizePerFile = maxRaftLogPersistDataSizePerFile;
+  }
+
+  public int getMaxNumberOfPersistRaftLogFiles() {
+    return maxNumberOfPersistRaftLogFiles;
+  }
+
+  public void setMaxNumberOfPersistRaftLogFiles(int maxNumberOfPersistRaftLogFiles) {
+    this.maxNumberOfPersistRaftLogFiles = maxNumberOfPersistRaftLogFiles;
+  }
+
+  public int getMaxPersistRaftLogNumberOnDisk() {
+    return maxPersistRaftLogNumberOnDisk;
+  }
+
+  public void setMaxPersistRaftLogNumberOnDisk(int maxPersistRaftLogNumberOnDisk) {
+    this.maxPersistRaftLogNumberOnDisk = maxPersistRaftLogNumberOnDisk;
+  }
+
+  public boolean isEnableUsePersistLogOnDiskToCatchUp() {
+    return enableUsePersistLogOnDiskToCatchUp;
+  }
+
+  public void setEnableUsePersistLogOnDiskToCatchUp(boolean enableUsePersistLogOnDiskToCatchUp) {
+    this.enableUsePersistLogOnDiskToCatchUp = enableUsePersistLogOnDiskToCatchUp;
+  }
+
+  public int getMaxNumberOfLogsPerFetchOnDisk() {
+    return maxNumberOfLogsPerFetchOnDisk;
+  }
+
+  public void setMaxNumberOfLogsPerFetchOnDisk(int maxNumberOfLogsPerFetchOnDisk) {
+    this.maxNumberOfLogsPerFetchOnDisk = maxNumberOfLogsPerFetchOnDisk;
+  }
+
+  public boolean isWaitForSlowNode() {
+    return waitForSlowNode;
+  }
+
+  public void setWaitForSlowNode(boolean waitForSlowNode) {
+    this.waitForSlowNode = waitForSlowNode;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java
similarity index 50%
copy from server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java
index f8c007c..184d0dd 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java
@@ -16,25 +16,27 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.cluster.config;
 
-package org.apache.iotdb.db.exception.query;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
 
-import org.apache.iotdb.db.exception.IoTDBException;
-import org.apache.iotdb.rpc.TSStatusCode;
+public class ClusterConstant {
 
-public class QueryProcessException extends IoTDBException {
+  // a failed election will restart in 2s~5s, this should be at least as long as a heartbeat
+  // interval, or a stale node may frequently issue elections and thus makes the leader step down
+  public static final long ELECTION_LEAST_TIME_OUT_MS = 2 * 1000L;
+  public static final long ELECTION_RANDOM_TIME_OUT_MS = 3 * 1000L;
+  public static final int SLOT_NUM = 10000;
+  public static final int HASH_SALT = 2333;
+  public static final int CHECK_ALIVE_TIME_OUT_MS = 1000;
 
-  private static final long serialVersionUID = -683191083844850054L;
+  public static final int LOG_NUM_IN_BATCH = 100;
 
-  public QueryProcessException(String message) {
-    super(message, TSStatusCode.QUERY_PROCESS_ERROR.getStatusCode());
-  }
+  public static final Node EMPTY_NODE = new Node();
 
-  public QueryProcessException(String message, int errorCode) {
-    super(message, errorCode);
+  private ClusterConstant() {
+    // constant class
   }
 
-  public QueryProcessException(IoTDBException e) {
-    super(e, e.getErrorCode());
-  }
+  static final String CLUSTER_CONF = "CLUSTER_CONF";
 }
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
new file mode 100644
index 0000000..3c9e8ec
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java
@@ -0,0 +1,371 @@
+/*
+ * 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.config;
+
+import com.google.common.net.InetAddresses;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.iotdb.cluster.exception.BadSeedUrlFormatException;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClusterDescriptor {
+
+  private static final Logger logger = LoggerFactory.getLogger(ClusterDescriptor.class);
+  private static final ClusterDescriptor INSTANCE = new ClusterDescriptor();
+
+  private static final String OPTION_INTERVAL_META_PORT = "internal_meta_port";
+  private static final String OPTION_INTERVAL_DATA_PORT = "internal_data_port";
+  private static final String OPTION_CLUSTER_RPC_PORT = "cluster_rpc_port";
+  private static final String OPTION_SEED_NODES = "seed_nodes";
+
+
+  private ClusterConfig config = new ClusterConfig();
+  private static CommandLine commandLine;
+
+  private ClusterDescriptor() {
+    loadProps();
+  }
+
+  public ClusterConfig getConfig() {
+    return config;
+  }
+
+  public static ClusterDescriptor getInstance() {
+    return INSTANCE;
+  }
+
+  public String getPropsUrl() {
+    String url = System.getProperty(ClusterConstant.CLUSTER_CONF, null);
+    if (url == null) {
+      url = System.getProperty(IoTDBConstant.IOTDB_HOME, null);
+      if (url != null) {
+        url = url + File.separatorChar + "conf" + File.separatorChar + ClusterConfig.CONFIG_NAME;
+      } else {
+        logger.warn(
+            "Cannot find IOTDB_HOME or CLUSTER_CONF environment variable when loading "
+                + "config file {}, use default configuration",
+            ClusterConfig.CONFIG_NAME);
+        // update all data seriesPath
+        return null;
+      }
+    } else {
+      url += (File.separatorChar + ClusterConfig.CONFIG_NAME);
+    }
+    return url;
+  }
+
+  public void replaceProps(String[] params) {
+    Options options = new Options();
+
+    Option metaPort = new Option(OPTION_INTERVAL_META_PORT, OPTION_INTERVAL_META_PORT, true,
+        "port for metadata service");
+    metaPort.setRequired(false);
+    options.addOption(metaPort);
+
+    Option dataPort = new Option(OPTION_INTERVAL_DATA_PORT, OPTION_INTERVAL_DATA_PORT, true,
+        "port for data service");
+    dataPort.setRequired(false);
+    options.addOption(dataPort);
+
+    Option clusterRpcPort = new Option(OPTION_CLUSTER_RPC_PORT, OPTION_CLUSTER_RPC_PORT, true,
+        "port for client service");
+    clusterRpcPort.setRequired(false);
+    options.addOption(clusterRpcPort);
+
+    Option seedNodes = new Option(OPTION_SEED_NODES, OPTION_SEED_NODES, true,
+        "comma-separated {IP/DOMAIN}:meta_port:data_port:client_port pairs");
+    seedNodes.setRequired(false);
+    options.addOption(seedNodes);
+
+    boolean ok = parseCommandLine(options, params);
+    if (!ok) {
+      logger.error("replaces properties failed, use default conf params");
+    } else {
+      if (commandLine.hasOption(OPTION_INTERVAL_META_PORT)) {
+        config.setInternalMetaPort(Integer.parseInt(commandLine.getOptionValue(
+            OPTION_INTERVAL_META_PORT)));
+        logger.debug("replace local meta port with={}", config.getInternalMetaPort());
+      }
+
+      if (commandLine.hasOption(OPTION_INTERVAL_DATA_PORT)) {
+        config.setInternalDataPort(Integer.parseInt(commandLine.getOptionValue(
+            OPTION_INTERVAL_DATA_PORT)));
+        logger.debug("replace local data port with={}", config.getInternalDataPort());
+      }
+
+      if (commandLine.hasOption(OPTION_CLUSTER_RPC_PORT)) {
+        config.setClusterRpcPort(Integer.parseInt(commandLine.getOptionValue(
+            OPTION_CLUSTER_RPC_PORT)));
+        logger.debug("replace local cluster rpc port with={}", config.getClusterRpcPort());
+      }
+
+      if (commandLine.hasOption(OPTION_SEED_NODES)) {
+        String seedNodeUrls = commandLine.getOptionValue(OPTION_SEED_NODES);
+        config.setSeedNodeUrls(getSeedUrlList(seedNodeUrls));
+        logger.debug("replace seed nodes with={}", config.getSeedNodeUrls());
+      }
+    }
+  }
+
+  public void replaceHostnameWithIp() throws UnknownHostException, BadSeedUrlFormatException {
+    boolean isInvalidClusterRpcIp = InetAddresses.isInetAddress(config.getClusterRpcIp());
+    if (!isInvalidClusterRpcIp) {
+      String clusterRpcIp = hostnameToIP(config.getClusterRpcIp());
+      config.setClusterRpcIp(clusterRpcIp);
+    }
+
+    List<String> newSeedUrls = new ArrayList<>();
+    for (String seedUrl : config.getSeedNodeUrls()) {
+      String[] splits = seedUrl.split(":");
+      if (splits.length != 4) {
+        throw new BadSeedUrlFormatException(seedUrl);
+      }
+      String seedIP = splits[0];
+      boolean isInvalidSeedIp = InetAddresses.isInetAddress(seedIP);
+      if (!isInvalidSeedIp) {
+        String newSeedIP = hostnameToIP(seedIP);
+        newSeedUrls.add(newSeedIP + ":" + splits[1] + ":" + splits[2] + ":" + splits[3]);
+      } else {
+        newSeedUrls.add(seedUrl);
+      }
+    }
+    config.setSeedNodeUrls(newSeedUrls);
+    logger.debug("after replace, the clusterRpcIP={}, seedUrls={}", config.getClusterRpcIp(),
+        config.getSeedNodeUrls());
+  }
+
+  private static boolean parseCommandLine(Options options, String[] params) {
+    try {
+      CommandLineParser parser = new DefaultParser();
+      commandLine = parser.parse(options, params);
+    } catch (ParseException e) {
+      logger.error("parse conf params failed", e);
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * load an property file and set TsfileDBConfig variables.
+   */
+  private void loadProps() {
+
+    String url = getPropsUrl();
+    Properties properties = System.getProperties();
+    if (url != null) {
+      try (InputStream inputStream = new FileInputStream(new File(url))) {
+        logger.info("Start to read config file {}", url);
+        properties.load(inputStream);
+      } catch (IOException e) {
+        logger.warn("Fail to find config file {}", url, e);
+      }
+    }
+    config.setClusterRpcIp(properties.getProperty("cluster_rpc_ip", config.getClusterRpcIp()));
+
+    config.setInternalMetaPort(Integer.parseInt(properties.getProperty(OPTION_INTERVAL_META_PORT,
+        String.valueOf(config.getInternalMetaPort()))));
+
+    config.setInternalDataPort(Integer.parseInt(properties.getProperty(OPTION_INTERVAL_DATA_PORT,
+        Integer.toString(config.getInternalDataPort()))));
+
+    config.setClusterRpcPort(Integer.parseInt(properties.getProperty(OPTION_CLUSTER_RPC_PORT,
+        Integer.toString(config.getClusterRpcPort()))));
+
+    config.setMaxConcurrentClientNum(Integer.parseInt(properties.getProperty(
+        "max_concurrent_client_num", String.valueOf(config.getMaxConcurrentClientNum()))));
+
+    config.setReplicationNum(Integer.parseInt(properties.getProperty(
+        "default_replica_num", String.valueOf(config.getReplicationNum()))));
+
+    config.setClusterName(properties.getProperty("cluster_name", config.getClusterName()));
+
+    config.setRpcThriftCompressionEnabled(Boolean.parseBoolean(properties.getProperty(
+        "rpc_thrift_compression_enable", String.valueOf(config.isRpcThriftCompressionEnabled()))));
+
+    config.setConnectionTimeoutInMS(Integer.parseInt(properties
+        .getProperty("connection_timeout_ms", String.valueOf(config.getConnectionTimeoutInMS()))));
+
+    config.setReadOperationTimeoutMS(Integer.parseInt(properties
+        .getProperty("read_operation_timeout_ms",
+            String.valueOf(config.getReadOperationTimeoutMS()))));
+
+    config.setCatchUpTimeoutMS(Integer.parseInt(properties
+        .getProperty("catch_up_timeout_ms",
+            String.valueOf(config.getCatchUpTimeoutMS()))));
+
+    config.setWriteOperationTimeoutMS(Integer.parseInt(properties
+        .getProperty("write_operation_timeout_ms",
+            String.valueOf(config.getWriteOperationTimeoutMS()))));
+
+    config.setUseBatchInLogCatchUp(Boolean.parseBoolean(properties.getProperty(
+        "use_batch_in_catch_up", String.valueOf(config.isUseBatchInLogCatchUp()))));
+
+    config.setMinNumOfLogsInMem(Integer.parseInt(properties
+        .getProperty("min_num_of_logs_in_mem", String.valueOf(config.getMinNumOfLogsInMem()))));
+
+    config.setMaxNumOfLogsInMem(Integer.parseInt(properties
+        .getProperty("max_num_of_logs_in_mem", String.valueOf(config.getMaxNumOfLogsInMem()))));
+
+    config.setLogDeleteCheckIntervalSecond(Integer.parseInt(properties
+        .getProperty("log_deletion_check_interval_second",
+            String.valueOf(config.getLogDeleteCheckIntervalSecond()))));
+
+    config.setEnableAutoCreateSchema(Boolean.parseBoolean(properties
+        .getProperty("enable_auto_create_schema",
+            String.valueOf(config.isEnableAutoCreateSchema()))));
+
+    config.setUseAsyncServer(
+        Boolean.parseBoolean(properties.getProperty("is_use_async_server",
+            String.valueOf(config.isUseAsyncServer()))));
+
+    config.setUseAsyncApplier(
+        Boolean.parseBoolean(properties.getProperty("is_use_async_applier",
+            String.valueOf(config.isUseAsyncApplier()))));
+
+    config.setEnableRaftLogPersistence(
+        Boolean.parseBoolean(properties.getProperty("is_enable_raft_log_persistence",
+            String.valueOf(config.isEnableRaftLogPersistence()))));
+
+    config.setFlushRaftLogThreshold(Integer.parseInt(properties
+        .getProperty("flush_raft_log_threshold", String.valueOf(config.getFlushRaftLogThreshold())))
+    );
+
+    config.setRaftLogBufferSize(Integer.parseInt(properties
+        .getProperty("raft_log_buffer_size", String.valueOf(config.getRaftLogBufferSize())))
+    );
+
+    config.setMaxRaftLogIndexSizeInMemory(Integer
+        .parseInt(properties.getProperty("max_raft_log_index_size_in_memory",
+            String.valueOf(config.getMaxRaftLogIndexSizeInMemory()))));
+
+    config.setMaxRaftLogPersistDataSizePerFile(Integer
+        .parseInt(properties.getProperty("max_raft_log_persist_data_size_per_file",
+            String.valueOf(config.getMaxRaftLogPersistDataSizePerFile()))));
+
+    config.setMaxNumberOfPersistRaftLogFiles(Integer
+        .parseInt(properties.getProperty("max_number_of_persist_raft_log_files",
+            String.valueOf(config.getMaxNumberOfPersistRaftLogFiles()))));
+
+    config.setMaxPersistRaftLogNumberOnDisk(
+        Integer.parseInt(properties.getProperty("max_persist_raft_log_number_on_disk",
+            String.valueOf(config.getMaxPersistRaftLogNumberOnDisk()))));
+
+    config.setMaxNumberOfLogsPerFetchOnDisk(
+        Integer.parseInt(properties.getProperty("max_number_of_logs_per_fetch_on_disk",
+            String.valueOf(config.getMaxNumberOfLogsPerFetchOnDisk()))));
+
+    config.setEnableUsePersistLogOnDiskToCatchUp(
+        Boolean.parseBoolean(properties.getProperty("enable_use_persist_log_on_disk_to_catch_up",
+            String.valueOf(config.isEnableUsePersistLogOnDiskToCatchUp()))));
+
+    String consistencyLevel = properties.getProperty("consistency_level");
+    if (consistencyLevel != null) {
+      config.setConsistencyLevel(ConsistencyLevel.getConsistencyLevel(consistencyLevel));
+    }
+
+    String seedUrls = properties.getProperty(OPTION_SEED_NODES);
+    if (seedUrls != null) {
+      List<String> urlList = getSeedUrlList(seedUrls);
+      config.setSeedNodeUrls(urlList);
+    }
+  }
+
+  private List<String> getSeedUrlList(String seedUrls) {
+    if (seedUrls == null) {
+      return Collections.emptyList();
+    }
+    List<String> urlList = new ArrayList<>();
+    String[] split = seedUrls.split(",");
+    for (String nodeUrl : split) {
+      nodeUrl = nodeUrl.trim();
+      if ("".equals(nodeUrl)) {
+        continue;
+      }
+      urlList.add(nodeUrl);
+    }
+    return urlList;
+  }
+
+  public void loadHotModifiedProps() throws QueryProcessException {
+    Properties properties = getProperties();
+    if (properties != null) {
+      loadHotModifiedProps(properties);
+    }
+  }
+
+  private Properties getProperties() throws QueryProcessException {
+    String url = getPropsUrl();
+    if (url == null) {
+      return null;
+    }
+    Properties properties;
+    try (InputStream inputStream = new FileInputStream(new File(url))) {
+      logger.info("Start to reload config file {}", url);
+      properties = new Properties();
+      properties.load(inputStream);
+    } catch (Exception e) {
+      throw new QueryProcessException(
+          String.format("Fail to reload config file %s because %s", url, e.getMessage()));
+    }
+    return properties;
+  }
+
+  /**
+   * This method is for setting hot modified properties of the cluster. Currently, we support
+   * max_concurrent_client_num, connection_timeout_ms, max_resolved_log_size
+   *
+   * @param properties
+   * @throws QueryProcessException
+   */
+  public void loadHotModifiedProps(Properties properties) {
+
+    config.setMaxConcurrentClientNum(Integer.parseInt(properties
+        .getProperty("max_concurrent_client_num",
+            String.valueOf(config.getMaxConcurrentClientNum()))));
+
+    config.setConnectionTimeoutInMS(Integer.parseInt(properties
+        .getProperty("connection_timeout_ms", String.valueOf(config.getConnectionTimeoutInMS()))));
+
+    logger.info("Set cluster configuration {}", properties);
+  }
+
+  private String hostnameToIP(String hostname) throws UnknownHostException {
+    InetAddress address = InetAddress.getByName(hostname);
+    return address.getHostAddress();
+  }
+
+}
\ No newline at end of file
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/config/ConsistencyLevel.java b/cluster/src/main/java/org/apache/iotdb/cluster/config/ConsistencyLevel.java
new file mode 100644
index 0000000..b7d0303
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ConsistencyLevel.java
@@ -0,0 +1,69 @@
+/*
+ * 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.config;
+
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public enum ConsistencyLevel {
+  /**
+   * Strong consistency means the server will first try to synchronize with the leader to get the
+   * newest meta data, if failed(timeout), directly report an error to the user;
+   */
+  STRONG_CONSISTENCY("strong"),
+
+  /**
+   * mid consistency means the server will first try to synchronize with the leader, but if
+   * failed(timeout), it will give up and just use current data it has cached before;
+   */
+  MID_CONSISTENCY("mid"),
+
+  /**
+   * weak consistency do not synchronize with the leader and simply use the local data
+   */
+  WEAK_CONSISTENCY("weak"),
+  ;
+
+  private String consistencyLevelName;
+  private static final Logger logger = LoggerFactory.getLogger(ConsistencyLevel.class);
+
+  ConsistencyLevel(String consistencyLevelName) {
+    this.consistencyLevelName = consistencyLevelName;
+  }
+
+  public static ConsistencyLevel getConsistencyLevel(String consistencyLevel) {
+    if (consistencyLevel == null) {
+      return ConsistencyLevel.MID_CONSISTENCY;
+    }
+    switch (consistencyLevel.toLowerCase()) {
+      case "strong":
+        return ConsistencyLevel.STRONG_CONSISTENCY;
+      case "mid":
+        return ConsistencyLevel.MID_CONSISTENCY;
+      case "weak":
+        return ConsistencyLevel.WEAK_CONSISTENCY;
+      default:
+        logger.warn("Unsupported consistency level={}, use default consistency level={}",
+            consistencyLevel, ConsistencyLevel.MID_CONSISTENCY.consistencyLevelName);
+        return ConsistencyLevel.MID_CONSISTENCY;
+    }
+  }
+
+}
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/AddSelfException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/AddSelfException.java
index c680701..f6fe674 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/AddSelfException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+/**
+ * Raised when a node received an AddNodeRequest of adding itself.
+ */
+public class AddSelfException extends Exception {
+
+  public AddSelfException() {
+    super("Cannot add one itself");
+  }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/BadSeedUrlFormatException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/BadSeedUrlFormatException.java
index c680701..813b924 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/BadSeedUrlFormatException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+public class BadSeedUrlFormatException extends Exception {
+
+  public BadSeedUrlFormatException(String seedUrl) {
+    super(String.format("Seed url %s has bad format, which should be "
+        + "{IP/DomainName}:{metaPort}:{dataPort}:{clientPort}", seedUrl));
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/CheckConsistencyException.java
similarity index 60%
copy from server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/CheckConsistencyException.java
index 5ff1e3b..12ac407 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/CheckConsistencyException.java
@@ -16,19 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.exception;
+package org.apache.iotdb.cluster.exception;
 
-import org.apache.iotdb.rpc.TSStatusCode;
-
-public class TsFileProcessorException extends IoTDBException {
-
-  private static final long serialVersionUID = 3749107630243950925L;
+/**
+ * Raised when check consistency failed, now only happens if there is a strong-consistency and
+ * syncLeader failed
+ */
+public class CheckConsistencyException extends Exception {
 
-  public TsFileProcessorException(String message) {
-    super(message, TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
+  public CheckConsistencyException(String errMag) {
+    super(String.format("check consistency failed, error message=%s ", errMag));
   }
 
-  public TsFileProcessorException(Exception exception) {
-    super(exception.getMessage(), TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
-  }
+  public static final CheckConsistencyException CHECK_STRONG_CONSISTENCY_EXCEPTION =
+      new CheckConsistencyException(
+      "strong consistency, sync with leader failed");
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/ConfigInconsistentException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/ConfigInconsistentException.java
index c680701..31c7150 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/ConfigInconsistentException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+public class ConfigInconsistentException extends Exception {
+
+  public ConfigInconsistentException() {
+    super("The configuration of this node is inconsistent with the cluster. See previous logs for "
+        + "explanation");
+  }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/EmptyIntervalException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/EmptyIntervalException.java
index c680701..829ba15 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/EmptyIntervalException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
+package org.apache.iotdb.cluster.exception;
 
-public class Config {
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+public class EmptyIntervalException extends Exception {
+
+  public EmptyIntervalException(Filter filter) {
+    super(String.format("The interval of the filter %s is empty.", filter));
+  }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/EntryCompactedException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/EntryCompactedException.java
index c680701..a5b2888 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/EntryCompactedException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+public class EntryCompactedException extends Exception {
+
+    public EntryCompactedException(long index, long boundary) {
+        super(String
+            .format("Entry index %d is unavailable due to compaction, and the lower bound is %d",
+                index, boundary));
+    }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/EntryUnavailableException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/EntryUnavailableException.java
index c680701..f4b6984 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/EntryUnavailableException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+public class EntryUnavailableException extends Exception {
+
+    public EntryUnavailableException(long index, long boundary) {
+        super(String
+            .format("Entry index %d is unavailable, and the upper bound is %d", index, boundary));
+    }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/GetEntriesWrongParametersException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/GetEntriesWrongParametersException.java
index c680701..d6903b9 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/GetEntriesWrongParametersException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+public class GetEntriesWrongParametersException extends Exception {
+
+  public GetEntriesWrongParametersException(long low, long high) {
+    super(String.format("invalid getEntries: parameter: %d >= %d", low, high));
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/LeaderUnknownException.java
similarity index 62%
copy from server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/LeaderUnknownException.java
index 5ff1e3b..bf6c3c6 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/LeaderUnknownException.java
@@ -16,19 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.exception;
+package org.apache.iotdb.cluster.exception;
 
-import org.apache.iotdb.rpc.TSStatusCode;
+import java.util.Collection;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
 
-public class TsFileProcessorException extends IoTDBException {
-
-  private static final long serialVersionUID = 3749107630243950925L;
-
-  public TsFileProcessorException(String message) {
-    super(message, TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
-  }
+/**
+ * Raised when a request should be forwarded to the leader but the leader cannot be found.
+ */
+public class LeaderUnknownException extends Exception {
 
-  public TsFileProcessorException(Exception exception) {
-    super(exception.getMessage(), TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
+  public LeaderUnknownException(Collection<Node> nodes) {
+    super(String.format("The leader is unknown in this group %s", nodes));
   }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/LogExecutionException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/LogExecutionException.java
index c680701..609c5f9 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/LogExecutionException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+public class LogExecutionException extends Exception {
+
+  public LogExecutionException(Throwable cause) {
+    super(cause);
+  }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/MemberReadOnlyException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/MemberReadOnlyException.java
index c680701..63d7f55 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/MemberReadOnlyException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+
+public class MemberReadOnlyException extends Exception{
+
+  public MemberReadOnlyException(Node node) {
+    super(String.format("The node %s has been set readonly for the partitions, please retry to find "
+        + "a new node", node));
+  }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/NoHeaderNodeException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/NoHeaderNodeException.java
index c680701..d634089 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/NoHeaderNodeException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+/**
+ * Raised when a DataClusterServer receives a request without a header node indicating which
+ * group it belongs to.
+ */
+public class NoHeaderNodeException extends Exception {
+
+  public NoHeaderNodeException() {
+    super("Header Node is required in data group communication!");
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/NotInSameGroupException.java
similarity index 62%
copy from server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/NotInSameGroupException.java
index 5ff1e3b..61314b2 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/NotInSameGroupException.java
@@ -16,19 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.exception;
 
-import org.apache.iotdb.rpc.TSStatusCode;
+package org.apache.iotdb.cluster.exception;
 
-public class TsFileProcessorException extends IoTDBException {
+import java.util.List;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
 
-  private static final long serialVersionUID = 3749107630243950925L;
-
-  public TsFileProcessorException(String message) {
-    super(message, TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
-  }
+/**
+ * Raised when a DataClusterServer receives a request but the node is not in the target group.
+ */
+public class NotInSameGroupException extends Exception {
 
-  public TsFileProcessorException(Exception exception) {
-    super(exception.getMessage(), TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
+  public NotInSameGroupException(List<Node> group, Node thisNode) {
+    super(String.format("This node %s is not in the data group %s", thisNode,
+        group));
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/NotManagedSlotException.java
similarity index 62%
copy from server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/NotManagedSlotException.java
index 5ff1e3b..ec78451 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/NotManagedSlotException.java
@@ -16,19 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.exception;
 
-import org.apache.iotdb.rpc.TSStatusCode;
+package org.apache.iotdb.cluster.exception;
 
-public class TsFileProcessorException extends IoTDBException {
+import java.util.List;
 
-  private static final long serialVersionUID = 3749107630243950925L;
-
-  public TsFileProcessorException(String message) {
-    super(message, TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
-  }
+/**
+ * Raised when a data group leader receives a PullSnapshotRequest but finds it does not manage
+ * the requested slot.
+ */
+public class NotManagedSlotException extends Exception {
 
-  public TsFileProcessorException(Exception exception) {
-    super(exception.getMessage(), TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
+  public NotManagedSlotException(int requiredSlots, List<Integer> slots) {
+    super(String.format("%d is not in the managed slots %s", requiredSlots, slots));
   }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/PartitionTableUnavailableException.java
similarity index 63%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/PartitionTableUnavailableException.java
index c680701..f658dfc 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/PartitionTableUnavailableException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+
+/**
+ * Raised when a node receives requests before the its partition table is set up.
+ */
+public class PartitionTableUnavailableException extends Exception {
+
+  public PartitionTableUnavailableException(Node node) {
+    super(String.format("Partition table of %s is not ready, cannot serve", node));
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/PullFileException.java
similarity index 63%
copy from server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/PullFileException.java
index 5ff1e3b..c4d25d2 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/PullFileException.java
@@ -16,19 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.exception;
 
-import org.apache.iotdb.rpc.TSStatusCode;
+package org.apache.iotdb.cluster.exception;
 
-public class TsFileProcessorException extends IoTDBException {
+import org.apache.iotdb.cluster.rpc.thrift.Node;
 
-  private static final long serialVersionUID = 3749107630243950925L;
+public class PullFileException extends Exception{
 
-  public TsFileProcessorException(String message) {
-    super(message, TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
+  public PullFileException(String fileName, Node node) {
+    super(String.format("Cannot pull file %s from %s due to network condition", fileName, node));
   }
 
-  public TsFileProcessorException(Exception exception) {
-    super(exception.getMessage(), TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
+  public PullFileException(String fileName, Node node, Exception e) {
+    super(String.format("Cannot pull file %s from %s because %s", fileName, node, e.getMessage()), e);
   }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/QueryTimeOutException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/QueryTimeOutException.java
index c680701..ea42fdf 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/QueryTimeOutException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+public class QueryTimeOutException extends Exception {
+
+  public QueryTimeOutException(String query) {
+    super("Query: " + query + " timed out");
+  }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/ReaderNotFoundException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/ReaderNotFoundException.java
index c680701..48a65d4 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/ReaderNotFoundException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+public class ReaderNotFoundException extends Exception {
+
+  public ReaderNotFoundException(long readerId) {
+    super(String.format("The requested reader %d is not found", readerId));
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/RequestTimeOutException.java
similarity index 63%
copy from server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/RequestTimeOutException.java
index 5ff1e3b..792b343 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/RequestTimeOutException.java
@@ -16,19 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.exception;
 
-import org.apache.iotdb.rpc.TSStatusCode;
+package org.apache.iotdb.cluster.exception;
 
-public class TsFileProcessorException extends IoTDBException {
+import org.apache.iotdb.cluster.log.Log;
 
-  private static final long serialVersionUID = 3749107630243950925L;
+/**
+ * Raised when a request times out before reaching an agreement in a group.
+ */
+public class RequestTimeOutException extends Exception {
 
-  public TsFileProcessorException(String message) {
-    super(message, TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
+  public RequestTimeOutException(Log log) {
+    super("Request \"" + log + "\" timeout");
   }
 
-  public TsFileProcessorException(Exception exception) {
-    super(exception.getMessage(), TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
+  public RequestTimeOutException(String request) {
+    super("Request \"" + request + "\" timeout");
   }
 }
diff --git a/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBConnectionException.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/SnapshotInstallationException.java
similarity index 66%
copy from service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBConnectionException.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/SnapshotInstallationException.java
index 90272ad..2866c3e 100644
--- a/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBConnectionException.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/SnapshotInstallationException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,21 +16,23 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.rpc;
 
-public class IoTDBConnectionException extends Exception{
+package org.apache.iotdb.cluster.exception;
 
-  private static final long serialVersionUID = -1268775292265203036L;
+public class SnapshotInstallationException extends Exception{
 
-  public IoTDBConnectionException(String reason) {
-    super(reason);
+  public SnapshotInstallationException(String message) {
+    super(message);
   }
 
-  public IoTDBConnectionException(Throwable cause) {
-    super(cause);
+  public SnapshotInstallationException(String message, Throwable cause) {
+    super(message, cause);
   }
 
-  public IoTDBConnectionException(String message, Throwable cause) {
-    super(message, cause);
+  public SnapshotInstallationException() {
+  }
+
+  public SnapshotInstallationException(Throwable cause) {
+    super(cause);
   }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/StartUpCheckFailureException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/StartUpCheckFailureException.java
index c680701..d84e741 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/StartUpCheckFailureException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+public class StartUpCheckFailureException extends Exception {
+
+  public StartUpCheckFailureException() {
+    super("The start up check cannot finish timely, please check the network");
+  }
+
+  public StartUpCheckFailureException(Throwable e) {
+    super(e);
+  }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/TruncateCommittedEntryException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/TruncateCommittedEntryException.java
index c680701..46b4ff5 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/TruncateCommittedEntryException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+public class TruncateCommittedEntryException extends Exception {
+
+  public TruncateCommittedEntryException(long index, long committed) {
+    super(String
+        .format("The committed entries cannot be truncated: parameter: %d, commitIndex : %d", index,
+            committed));
+  }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/UnknownLogTypeException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/UnknownLogTypeException.java
index c680701..149f623 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/UnknownLogTypeException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+/**
+ * Raised when the type of a raft log is unknown.
+ */
+public class UnknownLogTypeException extends Exception {
+
+  public UnknownLogTypeException(int logType) {
+    super("Unknown log type: " + logType);
+  }
 }
diff --git a/session/src/main/java/org/apache/iotdb/session/Config.java b/cluster/src/main/java/org/apache/iotdb/cluster/exception/UnsupportedPlanException.java
similarity index 64%
copy from session/src/main/java/org/apache/iotdb/session/Config.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/exception/UnsupportedPlanException.java
index c680701..479db4c 100644
--- a/session/src/main/java/org/apache/iotdb/session/Config.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/exception/UnsupportedPlanException.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,14 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.session;
 
-public class Config {
+package org.apache.iotdb.cluster.exception;
 
-  public static final String DEFAULT_USER = "root";
-  public static final String DEFAULT_PASSWORD = "root";
-  public static final int DEFAULT_FETCH_SIZE = 10000;
-  public static final int DEFAULT_TIMEOUT_MS = 0;
-  public static final int RETRY_NUM = 3;
-  public static final long RETRY_INTERVAL_MS = 1000;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+
+public class UnsupportedPlanException extends Exception {
+
+  public UnsupportedPlanException(PhysicalPlan plan) {
+    super(String.format("Plan %s is not supported", plan));
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/CommitLogCallback.java
similarity index 53%
copy from server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/log/CommitLogCallback.java
index f8c007c..63219fc 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/CommitLogCallback.java
@@ -17,24 +17,31 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.exception.query;
+package org.apache.iotdb.cluster.log;
 
-import org.apache.iotdb.db.exception.IoTDBException;
-import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.cluster.server.member.RaftMember;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-public class QueryProcessException extends IoTDBException {
+public class CommitLogCallback implements AsyncMethodCallback<Void> {
 
-  private static final long serialVersionUID = -683191083844850054L;
+  private static final Logger logger = LoggerFactory.getLogger(CommitLogCallback.class);
+  private final RaftMember raftMember;
 
-  public QueryProcessException(String message) {
-    super(message, TSStatusCode.QUERY_PROCESS_ERROR.getStatusCode());
+  public CommitLogCallback(RaftMember raftMember) {
+    this.raftMember = raftMember;
   }
 
-  public QueryProcessException(String message, int errorCode) {
-    super(message, errorCode);
+  @Override
+  public void onComplete(Void v) {
+    synchronized (raftMember.getSyncLock()) {
+      raftMember.getSyncLock().notifyAll();
+    }
   }
 
-  public QueryProcessException(IoTDBException e) {
-    super(e, e.getErrorCode());
+  @Override
+  public void onError(Exception e) {
+    logger.error("async commit log failed", e);
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/CommitLogTask.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/CommitLogTask.java
new file mode 100644
index 0000000..4b71abf
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/CommitLogTask.java
@@ -0,0 +1,68 @@
+/*
+ * 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.log;
+
+import org.apache.iotdb.cluster.log.manage.RaftLogManager;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CommitLogTask implements Runnable {
+
+  private static final Logger logger = LoggerFactory.getLogger(CommitLogTask.class);
+  private RaftLogManager logManager;
+  private long leaderCommit;
+  private long term;
+
+  public CommitLogTask(RaftLogManager logManager, long leaderCommit, long term) {
+    this.logManager = logManager;
+    this.leaderCommit = leaderCommit;
+    this.term = term;
+  }
+
+  /**
+   * listener field
+   */
+  private AsyncMethodCallback<Void> callback;
+
+  /**
+   * @param callback the event listener
+   */
+  public void registerCallback(AsyncMethodCallback<Void> callback) {
+    this.callback = callback;
+  }
+
+  private void doCommitLog() {
+    if (callback == null) {
+      logger.error("callback is not registered");
+      return;
+    }
+
+    boolean success = logManager.maybeCommit(leaderCommit, term);
+    if (success) {
+      callback.onComplete(null);
+    }
+  }
+
+  @Override
+  public void run() {
+    doCommitLog();
+  }
+}
\ No newline at end of file
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/HardState.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/HardState.java
new file mode 100644
index 0000000..46bf2dc
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/HardState.java
@@ -0,0 +1,133 @@
+/*
+ * 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.log;
+
+import java.nio.ByteBuffer;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.db.utils.SerializeUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+
+public class HardState {
+
+  private long currentTerm;
+  private Node voteFor;
+
+  public HardState() {
+    this.voteFor = null;
+  }
+
+  public static HardState deserialize(ByteBuffer buffer) {
+    HardState res = new HardState();
+    res.setCurrentTerm(ReadWriteIOUtils.readLong(buffer));
+    // marker is previously read, remaining fields:
+    // currentTerm(long), marker(byte)
+    // (optional)ipLength(int), ipBytes(byte[]), port(int), identifier(int), dataPort(int)
+    int isNull = buffer.get();
+    if (isNull == 1) {
+      Node node = new Node();
+      SerializeUtils.deserialize(node, buffer);
+      res.setVoteFor(node);
+    } else {
+      res.setVoteFor(null);
+    }
+    return res;
+  }
+
+  @SuppressWarnings("java:S125") // not commented code
+  public ByteBuffer serialize() {
+    int totalSize = Long.BYTES + Byte.BYTES;
+    // currentTerm(long), marker(byte)
+    // (optional) ipLength(int), ipBytes(byte[]), port(int), identifier(int), dataPort(int), clientPort(int)
+    if (voteFor != null) {
+      byte[] ipBytes = voteFor.getIp().getBytes();
+      totalSize +=
+          Integer.BYTES + ipBytes.length + Integer.BYTES + Integer.BYTES + Integer.BYTES + Integer.BYTES;
+      byte[] buffer = new byte[totalSize];
+      ByteBuffer byteBuffer = ByteBuffer.wrap(buffer);
+      byteBuffer.putLong(currentTerm);
+      byteBuffer.put((byte) 1);
+      byteBuffer.putInt(ipBytes.length);
+      byteBuffer.put(ipBytes);
+      byteBuffer.putInt(voteFor.getMetaPort());
+      byteBuffer.putInt(voteFor.getNodeIdentifier());
+      byteBuffer.putInt(voteFor.getDataPort());
+      byteBuffer.putInt(voteFor.getClientPort());
+      byteBuffer.flip();
+      return byteBuffer;
+    }
+    byte[] buffer = new byte[totalSize];
+    ByteBuffer byteBuffer = ByteBuffer.wrap(buffer);
+    byteBuffer.putLong(currentTerm);
+    byteBuffer.put((byte) 0);
+    byteBuffer.flip();
+
+    return byteBuffer;
+  }
+
+  public long getCurrentTerm() {
+    return currentTerm;
+  }
+
+  public void setCurrentTerm(long currentTerm) {
+    this.currentTerm = currentTerm;
+  }
+
+  public Node getVoteFor() {
+    return voteFor;
+  }
+
+  public void setVoteFor(Node voteFor) {
+    this.voteFor = voteFor;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof HardState)) {
+      return false;
+    }
+    HardState that = (HardState) o;
+    return new EqualsBuilder()
+        .append(currentTerm, that.currentTerm)
+        .append(voteFor, that.voteFor)
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+        .append(currentTerm)
+        .append(voteFor)
+        .toHashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "HardState{" +
+        "currentTerm=" + currentTerm +
+        ", voteFor=" + voteFor +
+        '}';
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/Log.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/Log.java
new file mode 100644
index 0000000..0c236b2
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/Log.java
@@ -0,0 +1,130 @@
+/*
+ * 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.log;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+import java.util.Objects;
+
+/**
+ * Log records operations that are made on this cluster. Each log records 2 longs: currLogIndex,
+ * currLogTerm, so that the logs in a cluster will form a log chain and abnormal operations can thus
+ * be distinguished and removed.
+ */
+public abstract class Log implements Comparable<Log> {
+
+  private static final Comparator<Log> COMPARATOR =
+      Comparator.comparingLong(Log::getCurrLogIndex).thenComparing(Log::getCurrLogTerm);
+
+  protected static final int DEFAULT_BUFFER_SIZE = 4096;
+  private long currLogIndex;
+  private long currLogTerm;
+
+  // for async application
+  private volatile boolean applied;
+  @SuppressWarnings("java:S3077")
+  private volatile Exception exception;
+
+  private long createTime;
+  private long enqueueTime;
+
+  public abstract ByteBuffer serialize();
+
+  public abstract void deserialize(ByteBuffer buffer);
+
+  public enum Types {
+    // DO CHECK LogParser when you add a new type of log
+    ADD_NODE, PHYSICAL_PLAN, CLOSE_FILE, REMOVE_NODE, EMPTY_CONTENT, TEST_LARGE_CONTENT
+  }
+
+  public long getCurrLogIndex() {
+    return currLogIndex;
+  }
+
+  public void setCurrLogIndex(long currLogIndex) {
+    this.currLogIndex = currLogIndex;
+  }
+
+  public long getCurrLogTerm() {
+    return currLogTerm;
+  }
+
+  public void setCurrLogTerm(long currLogTerm) {
+    this.currLogTerm = currLogTerm;
+  }
+
+  @SuppressWarnings("java:S2886") // synchronized outside
+  public boolean isApplied() {
+    return applied;
+  }
+
+  public void setApplied(boolean applied) {
+    synchronized (this) {
+      this.applied = applied;
+      this.notifyAll();
+    }
+  }
+
+  public Exception getException() {
+    return exception;
+  }
+
+  public void setException(Exception exception) {
+    this.exception = exception;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Log log = (Log) o;
+    return currLogIndex == log.currLogIndex &&
+        currLogTerm == log.currLogTerm;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(currLogIndex, currLogTerm);
+  }
+
+  @Override
+  public int compareTo(Log o) {
+    return COMPARATOR.compare(this, o);
+  }
+
+  public long getCreateTime() {
+    return createTime;
+  }
+
+  public void setCreateTime(long createTime) {
+    this.createTime = createTime;
+  }
+
+  public long getEnqueueTime() {
+    return enqueueTime;
+  }
+
+  public void setEnqueueTime(long enqueueTime) {
+    this.enqueueTime = enqueueTime;
+  }
+}
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/LogApplier.java
similarity index 62%
copy from server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/log/LogApplier.java
index 5ff1e3b..6c768f6 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/TsFileProcessorException.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/LogApplier.java
@@ -16,19 +16,23 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.exception;
 
-import org.apache.iotdb.rpc.TSStatusCode;
+package org.apache.iotdb.cluster.log;
 
-public class TsFileProcessorException extends IoTDBException {
+/**
+ * LogApplier applies the log to the local node to make it take effect.
+ */
+public interface LogApplier {
 
-  private static final long serialVersionUID = 3749107630243950925L;
+  /**
+   * Apply the given log, if any exception is thrown during the execution, the exception will be
+   * recorded in the log. Either an exception is thrown or the log is executed successfully, log
+   * .setApplied(true) must be called.
+   * @param log
+   */
+  void apply(Log log);
 
-  public TsFileProcessorException(String message) {
-    super(message, TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
-  }
+  default void close() {
 
-  public TsFileProcessorException(Exception exception) {
-    super(exception.getMessage(), TSStatusCode.TSFILE_PROCESSOR_ERROR.getStatusCode());
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/LogDispatcher.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/LogDispatcher.java
new file mode 100644
index 0000000..751d3e8
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/LogDispatcher.java
@@ -0,0 +1,424 @@
+/*
+ * 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.log;
+
+
+import java.util.concurrent.Future;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.rpc.thrift.AppendEntriesRequest;
+import org.apache.iotdb.cluster.rpc.thrift.AppendEntryRequest;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService.AsyncClient;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService.Client;
+import org.apache.iotdb.cluster.server.Peer;
+import org.apache.iotdb.cluster.server.Timer;
+import org.apache.iotdb.cluster.server.handlers.caller.AppendNodeEntryHandler;
+import org.apache.iotdb.cluster.server.member.RaftMember;
+import org.apache.iotdb.cluster.utils.ClientUtils;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A LogDispatcher servers a raft leader by queuing logs that the leader wants to send to the
+ * follower and send the logs in an ordered manner so that the followers will not wait for previous
+ * logs for too long. For example: if the leader send 3 logs, log1, log2, log3, concurrently to
+ * follower A, the actual reach order may be log3, log2, and log1. According to the protocol, log3
+ * and log2 must halt until log1 reaches, as a result, the total delay may increase significantly.
+ */
+public class LogDispatcher {
+
+  private static final Logger logger = LoggerFactory.getLogger(LogDispatcher.class);
+  private RaftMember member;
+  private boolean useBatchInLogCatchUp = ClusterDescriptor.getInstance().getConfig()
+      .isUseBatchInLogCatchUp();
+  private List<BlockingQueue<SendLogRequest>> nodeLogQueues =
+      new ArrayList<>();
+  private ExecutorService executorService;
+  private ExecutorService serializationService;
+
+  public LogDispatcher(RaftMember member) {
+    this.member = member;
+    executorService = Executors.newCachedThreadPool();
+    serializationService = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
+    for (Node node : member.getAllNodes()) {
+      if (!node.equals(member.getThisNode())) {
+        nodeLogQueues.add(createQueueAndBindingThread(node));
+      }
+    }
+  }
+
+  @TestOnly
+  public void close() throws InterruptedException {
+    executorService.shutdownNow();
+    executorService.awaitTermination(10, TimeUnit.SECONDS);
+    serializationService.shutdownNow();
+    serializationService.awaitTermination(10, TimeUnit.SECONDS);
+  }
+
+  public void offer(SendLogRequest log) {
+    // do serialization here to avoid taking LogManager for too long
+    log.serializedLogFuture = serializationService.submit(() -> log.getLog().serialize());
+    for (int i = 0; i < nodeLogQueues.size(); i++) {
+      BlockingQueue<SendLogRequest> nodeLogQueue = nodeLogQueues.get(i);
+      try {
+        boolean addSucceeded;
+        if (ClusterDescriptor.getInstance().getConfig().isWaitForSlowNode()) {
+          addSucceeded = nodeLogQueue.offer(log,
+              ClusterDescriptor.getInstance().getConfig().getWriteOperationTimeoutMS(),
+              TimeUnit.MILLISECONDS);
+        } else {
+          addSucceeded = nodeLogQueue.add(log);
+        }
+
+        if (!addSucceeded) {
+          logger.debug("Log queue[{}] of {} is full, ignore the log to this node", i,
+              member.getName());
+        } else {
+          log.setEnqueueTime(System.nanoTime());
+        }
+      } catch (IllegalStateException e) {
+        logger.debug("Log queue[{}] of {} is full, ignore the log to this node", i,
+            member.getName());
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      }
+    }
+  }
+
+  private BlockingQueue<SendLogRequest> createQueueAndBindingThread(Node node) {
+    BlockingQueue<SendLogRequest> logBlockingQueue =
+        new ArrayBlockingQueue<>(
+            ClusterDescriptor.getInstance().getConfig().getMaxNumOfLogsInMem());
+    int bindingThreadNum = 1;
+    for (int i = 0; i < bindingThreadNum; i++) {
+      executorService.submit(new DispatcherThread(node, logBlockingQueue));
+    }
+    return logBlockingQueue;
+  }
+
+  public static class SendLogRequest {
+
+    private Log log;
+    private AtomicInteger voteCounter;
+    private AtomicBoolean leaderShipStale;
+    private AtomicLong newLeaderTerm;
+    private AppendEntryRequest appendEntryRequest;
+    private long enqueueTime;
+    private Future<ByteBuffer> serializedLogFuture;
+
+    public SendLogRequest(Log log, AtomicInteger voteCounter,
+        AtomicBoolean leaderShipStale, AtomicLong newLeaderTerm,
+        AppendEntryRequest appendEntryRequest) {
+      this.setLog(log);
+      this.setVoteCounter(voteCounter);
+      this.setLeaderShipStale(leaderShipStale);
+      this.setNewLeaderTerm(newLeaderTerm);
+      this.setAppendEntryRequest(appendEntryRequest);
+    }
+
+    public AtomicInteger getVoteCounter() {
+      return voteCounter;
+    }
+
+    public void setVoteCounter(AtomicInteger voteCounter) {
+      this.voteCounter = voteCounter;
+    }
+
+    public Log getLog() {
+      return log;
+    }
+
+    public void setLog(Log log) {
+      this.log = log;
+    }
+
+    public long getEnqueueTime() {
+      return enqueueTime;
+    }
+
+    public void setEnqueueTime(long enqueueTime) {
+      this.enqueueTime = enqueueTime;
+    }
+
+    public AtomicBoolean getLeaderShipStale() {
+      return leaderShipStale;
+    }
+
+    public void setLeaderShipStale(AtomicBoolean leaderShipStale) {
+      this.leaderShipStale = leaderShipStale;
+    }
+
+    public AtomicLong getNewLeaderTerm() {
+      return newLeaderTerm;
+    }
+
+    void setNewLeaderTerm(AtomicLong newLeaderTerm) {
+      this.newLeaderTerm = newLeaderTerm;
+    }
+
+    public AppendEntryRequest getAppendEntryRequest() {
+      return appendEntryRequest;
+    }
+
+    public void setAppendEntryRequest(
+        AppendEntryRequest appendEntryRequest) {
+      this.appendEntryRequest = appendEntryRequest;
+    }
+
+    @Override
+    public String toString() {
+      return "SendLogRequest{" +
+          "log=" + log +
+          '}';
+    }
+  }
+
+  class DispatcherThread implements Runnable {
+
+    private Node receiver;
+    private BlockingQueue<SendLogRequest> logBlockingDeque;
+    private List<SendLogRequest> currBatch = new ArrayList<>();
+    private Peer peer;
+
+    DispatcherThread(Node receiver,
+        BlockingQueue<SendLogRequest> logBlockingDeque) {
+      this.receiver = receiver;
+      this.logBlockingDeque = logBlockingDeque;
+      this.peer = member.getPeerMap().computeIfAbsent(receiver,
+          r -> new Peer(member.getLogManager().getLastLogIndex()));
+    }
+
+    @Override
+    public void run() {
+      Thread.currentThread().setName("LogDispatcher-" + member.getName() + "-" + receiver);
+      try {
+        while (!Thread.interrupted()) {
+          SendLogRequest poll = logBlockingDeque.take();
+          currBatch.add(poll);
+          logBlockingDeque.drainTo(currBatch);
+          if (logger.isDebugEnabled()) {
+            logger.debug("Sending {} logs to {}", currBatch.size(), receiver);
+          }
+          for (SendLogRequest request : currBatch) {
+            request.getAppendEntryRequest().entry = request.serializedLogFuture.get();
+          }
+          sendBatchLogs(currBatch);
+          currBatch.clear();
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      } catch (Exception e) {
+        logger.error("Unexpected error in log dispatcher", e);
+      }
+      logger.info("Dispatcher exits");
+    }
+
+    private void appendEntriesAsync(List<ByteBuffer> logList, AppendEntriesRequest request,
+        List<SendLogRequest> currBatch)
+        throws TException {
+      AsyncMethodCallback<Long> handler = new AppendEntriesHandler(currBatch);
+      AsyncClient client = member.getSendLogAsyncClient(receiver);
+      if (logger.isDebugEnabled()) {
+        logger.debug("{}: append entries {} with {} logs", member.getName(), receiver,
+            logList.size());
+      }
+      if (client != null) {
+        client.appendEntries(request, handler);
+      }
+    }
+
+    private void appendEntriesSync(List<ByteBuffer> logList, AppendEntriesRequest request,
+        List<SendLogRequest> currBatch) {
+
+      long startTime = Timer.Statistic.RAFT_SENDER_WAIT_FOR_PREV_LOG.getOperationStartTime();
+      if (!member.waitForPrevLog(peer, currBatch.get(0).getLog())) {
+        logger.warn("{}: node {} timed out when appending {}", member.getName(), receiver,
+            currBatch.get(0).getLog());
+        return;
+      }
+      Timer.Statistic.RAFT_SENDER_WAIT_FOR_PREV_LOG.calOperationCostTimeFromStart(startTime);
+
+      Client client = member.getSyncClient(receiver);
+      if (client == null) {
+        logger.error("No available client for {}", receiver);
+        return;
+      }
+      AsyncMethodCallback<Long> handler = new AppendEntriesHandler(currBatch);
+      startTime = Timer.Statistic.RAFT_SENDER_SEND_LOG.getOperationStartTime();
+      try {
+        long result = client.appendEntries(request);
+        Timer.Statistic.RAFT_SENDER_SEND_LOG.calOperationCostTimeFromStart(startTime);
+        if (result != -1 && logger.isInfoEnabled()) {
+          logger.info("{}: Append {} logs to {}, resp: {}", member.getName(), logList.size(),
+              receiver, result);
+        }
+        handler.onComplete(result);
+      } catch (TException e) {
+        client.getInputProtocol().getTransport().close();
+        handler.onError(e);
+        logger.warn("Failed logs: {}, first index: {}", logList, request.prevLogIndex + 1);
+      } finally {
+        ClientUtils.putBackSyncClient(client);
+      }
+    }
+
+    private AppendEntriesRequest prepareRequest(List<ByteBuffer> logList,
+        List<SendLogRequest> currBatch, int firstIndex) {
+      AppendEntriesRequest request = new AppendEntriesRequest();
+
+      if (member.getHeader() != null) {
+        request.setHeader(member.getHeader());
+      }
+      request.setLeader(member.getThisNode());
+      request.setLeaderCommit(member.getLogManager().getCommitLogIndex());
+
+      synchronized (member.getTerm()) {
+        request.setTerm(member.getTerm().get());
+      }
+
+      request.setEntries(logList);
+      // set index for raft
+      request.setPrevLogIndex(currBatch.get(firstIndex).getLog().getCurrLogIndex() - 1);
+      try {
+        request.setPrevLogTerm(currBatch.get(firstIndex).getAppendEntryRequest().prevLogTerm);
+      } catch (Exception e) {
+        logger.error("getTerm failed for newly append entries", e);
+      }
+      return request;
+    }
+
+    private void sendLogs(List<SendLogRequest> currBatch) throws TException {
+      int logIndex = 0;
+      logger.debug("send logs from index {} to {}", currBatch.get(0).getLog().getCurrLogIndex(),
+          currBatch.get(currBatch.size() - 1).getLog().getCurrLogIndex());
+      while (logIndex < currBatch.size()) {
+        long logSize = IoTDBDescriptor.getInstance().getConfig().getThriftMaxFrameSize();
+        List<ByteBuffer> logList = new ArrayList<>();
+        int prevIndex = logIndex;
+
+        for (; logIndex < currBatch.size(); logIndex++) {
+          long curSize = currBatch.get(logIndex).getAppendEntryRequest().entry.array().length;
+          if (logSize - curSize <= IoTDBConstant.LEFT_SIZE_IN_REQUEST) {
+            break;
+          }
+          logSize -= curSize;
+          Timer.Statistic.LOG_DISPATCHER_LOG_IN_QUEUE
+              .calOperationCostTimeFromStart(currBatch.get(logIndex).getLog().getCreateTime());
+          logList.add(currBatch.get(logIndex).getAppendEntryRequest().entry);
+        }
+
+        AppendEntriesRequest appendEntriesRequest = prepareRequest(logList, currBatch, prevIndex);
+        if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+          appendEntriesAsync(logList, appendEntriesRequest, currBatch.subList(prevIndex, logIndex));
+        } else {
+          appendEntriesSync(logList, appendEntriesRequest, currBatch.subList(prevIndex, logIndex));
+        }
+        for (; prevIndex < logIndex; prevIndex++) {
+          Timer.Statistic.LOG_DISPATCHER_FROM_CREATE_TO_END
+              .calOperationCostTimeFromStart(currBatch.get(prevIndex).getLog().getCreateTime());
+        }
+      }
+    }
+
+    private void sendBatchLogs(List<SendLogRequest> currBatch) throws TException {
+      if (currBatch.size() > 1) {
+        if (useBatchInLogCatchUp) {
+          sendLogs(currBatch);
+        } else {
+          for (SendLogRequest batch : currBatch) {
+            sendLog(batch);
+          }
+        }
+      } else {
+        sendLog(currBatch.get(0));
+      }
+    }
+
+    private void sendLog(SendLogRequest logRequest) {
+      Timer.Statistic.LOG_DISPATCHER_LOG_IN_QUEUE
+          .calOperationCostTimeFromStart(logRequest.getLog().getCreateTime());
+      member.sendLogToFollower(logRequest.getLog(), logRequest.getVoteCounter(), receiver,
+          logRequest.getLeaderShipStale(), logRequest.getNewLeaderTerm(),
+          logRequest.getAppendEntryRequest());
+      Timer.Statistic.LOG_DISPATCHER_FROM_CREATE_TO_END
+          .calOperationCostTimeFromStart(logRequest.getLog().getCreateTime());
+    }
+
+    class AppendEntriesHandler implements AsyncMethodCallback<Long> {
+
+      private final List<AsyncMethodCallback<Long>> singleEntryHandlers;
+
+      private AppendEntriesHandler(List<SendLogRequest> batch) {
+        singleEntryHandlers = new ArrayList<>(batch.size());
+        for (SendLogRequest sendLogRequest : batch) {
+          AppendNodeEntryHandler handler = getAppendNodeEntryHandler(sendLogRequest.getLog(),
+              sendLogRequest.getVoteCounter()
+              , receiver,
+              sendLogRequest.getLeaderShipStale(), sendLogRequest.getNewLeaderTerm(), peer);
+          singleEntryHandlers.add(handler);
+        }
+      }
+
+      @Override
+      public void onComplete(Long aLong) {
+        for (AsyncMethodCallback<Long> singleEntryHandler : singleEntryHandlers) {
+          singleEntryHandler.onComplete(aLong);
+        }
+      }
+
+      @Override
+      public void onError(Exception e) {
+        for (AsyncMethodCallback<Long> singleEntryHandler : singleEntryHandlers) {
+          singleEntryHandler.onError(e);
+        }
+      }
+
+      private AppendNodeEntryHandler getAppendNodeEntryHandler(Log log, AtomicInteger voteCounter,
+          Node node, AtomicBoolean leaderShipStale, AtomicLong newLeaderTerm, Peer peer) {
+        AppendNodeEntryHandler handler = new AppendNodeEntryHandler();
+        handler.setReceiver(node);
+        handler.setVoteCounter(voteCounter);
+        handler.setLeaderShipStale(leaderShipStale);
+        handler.setLog(log);
+        handler.setMember(member);
+        handler.setPeer(peer);
+        handler.setReceiverTerm(newLeaderTerm);
+        return handler;
+      }
+    }
+  }
+}
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
new file mode 100644
index 0000000..4a7afc4
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/LogParser.java
@@ -0,0 +1,100 @@
+/*
+ * 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.log;
+
+import java.nio.ByteBuffer;
+import org.apache.iotdb.cluster.exception.UnknownLogTypeException;
+import org.apache.iotdb.cluster.log.Log.Types;
+import org.apache.iotdb.cluster.log.logtypes.AddNodeLog;
+import org.apache.iotdb.cluster.log.logtypes.CloseFileLog;
+import org.apache.iotdb.cluster.log.logtypes.EmptyContentLog;
+import org.apache.iotdb.cluster.log.logtypes.LargeTestLog;
+import org.apache.iotdb.cluster.log.logtypes.PhysicalPlanLog;
+import org.apache.iotdb.cluster.log.logtypes.RemoveNodeLog;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * LogParser transform a ByteBuffer into a Log.
+ */
+public class LogParser {
+
+  private static final Logger logger = LoggerFactory.getLogger(LogParser.class);
+  private static final LogParser INSTANCE = new LogParser();
+
+  private LogParser() {
+    // singleton class
+  }
+
+  public static LogParser getINSTANCE() {
+    return INSTANCE;
+  }
+
+  public Log parse(ByteBuffer buffer) throws UnknownLogTypeException {
+    if (logger.isDebugEnabled()) {
+      logger.debug("Received a log buffer, pos:{}, limit:{}", buffer.position(), buffer.limit());
+    }
+    int typeInt = buffer.get();
+    Types type;
+    try {
+      type = Types.values()[typeInt];
+    } catch (ArrayIndexOutOfBoundsException e) {
+      throw new UnknownLogTypeException(typeInt);
+    }
+    logger.debug("The log type is {}", type);
+    Log log;
+    switch (type) {
+      case ADD_NODE:
+        AddNodeLog addNodeLog = new AddNodeLog();
+        addNodeLog.deserialize(buffer);
+        log = addNodeLog;
+        break;
+      case PHYSICAL_PLAN:
+        PhysicalPlanLog physicalPlanLog = new PhysicalPlanLog();
+        physicalPlanLog.deserialize(buffer);
+        log = physicalPlanLog;
+        break;
+      case CLOSE_FILE:
+        CloseFileLog closeFileLog = new CloseFileLog();
+        closeFileLog.deserialize(buffer);
+        log = closeFileLog;
+        break;
+      case REMOVE_NODE:
+        RemoveNodeLog removeNodeLog = new RemoveNodeLog();
+        removeNodeLog.deserialize(buffer);
+        log = removeNodeLog;
+        break;
+      case EMPTY_CONTENT:
+        EmptyContentLog emptyLog = new EmptyContentLog();
+        emptyLog.deserialize(buffer);
+        log = emptyLog;
+        break;
+      case TEST_LARGE_CONTENT:
+        LargeTestLog largeLog = new LargeTestLog();
+        largeLog.deserialize(buffer);
+        log = largeLog;
+        break;
+      default:
+        throw new IllegalArgumentException(type.toString());
+    }
+    logger.debug("Parsed a log {}", log);
+    return log;
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/Snapshot.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/Snapshot.java
new file mode 100644
index 0000000..2d15dd1
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/Snapshot.java
@@ -0,0 +1,71 @@
+/*
+ * 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.log;
+
+
+import java.nio.ByteBuffer;
+import org.apache.iotdb.cluster.log.snapshot.SnapshotInstaller;
+import org.apache.iotdb.cluster.server.member.RaftMember;
+
+/**
+ * As we can only hold a certain amount of logs in memory, when the logs' size exceed the memory
+ * capacity, they will be deleted or compressed, and a snapshot of current system will be generated.
+ * If a node need to catch up but its next log is deleted, it will catch up based on the latest
+ * snapshot and the logs still in memory. The snapshot could be a file recording the list of current
+ * system files, or the compressed all historical logs, depending on the implementation.
+ */
+public abstract class Snapshot {
+
+  protected long lastLogIndex;
+  protected long lastLogTerm;
+
+  public abstract ByteBuffer serialize();
+
+  public abstract void deserialize(ByteBuffer buffer);
+
+  public void setLastLogIndex(long lastLogIndex) {
+    this.lastLogIndex = lastLogIndex;
+  }
+
+  public void setLastLogTerm(long lastLogTerm) {
+    this.lastLogTerm = lastLogTerm;
+  }
+
+  public long getLastLogIndex() {
+    return lastLogIndex;
+  }
+
+  public long getLastLogTerm() {
+    return lastLogTerm;
+  }
+
+  @SuppressWarnings("java:S1452") // is it possible not to use wildcard?
+  public abstract SnapshotInstaller<? extends Snapshot> getDefaultInstaller(RaftMember member);
+
+  /**
+   * Discard contents which is generated by logs whose index <= 'minIndex' if possible.
+   * This method is a best-effort one without guarantee that the result will absolutely not
+   * contain contents before 'minIndex'.
+   * @param minIndex
+   */
+  public void truncateBefore(long minIndex) {
+
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/StableEntryManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/StableEntryManager.java
new file mode 100644
index 0000000..1b406d9
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/StableEntryManager.java
@@ -0,0 +1,55 @@
+/*
+ * 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.log;
+
+import java.io.IOException;
+import java.util.List;
+
+public interface StableEntryManager {
+
+  List<Log> getAllEntriesAfterAppliedIndex();
+
+  void append(List<Log> entries, long maxHaveAppliedCommitIndex) throws IOException;
+
+  void flushLogBuffer();
+
+  void forceFlushLogBuffer();
+
+  void removeCompactedEntries(long index);
+
+  void setHardStateAndFlush(HardState state);
+
+  HardState getHardState();
+
+  /**
+   * @param startIndex (inclusive) the log start index
+   * @param endIndex   (inclusive) the log end index
+   * @return the raft log which index between [startIndex, endIndex] or empty if not found
+   */
+  List<Log> getLogs(long startIndex, long endIndex);
+
+  void close();
+
+  /**
+   * clear all logs, this method mainly used for after a follower accept a snapshot, all the logs
+   * should be cleaned
+   */
+  void clearAllLogs(long commitIndex);
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/AsyncDataLogApplier.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/AsyncDataLogApplier.java
new file mode 100644
index 0000000..0958432
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/AsyncDataLogApplier.java
@@ -0,0 +1,270 @@
+/*
+ * 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.log.applier;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import org.apache.iotdb.cluster.log.Log;
+import org.apache.iotdb.cluster.log.LogApplier;
+import org.apache.iotdb.cluster.log.logtypes.CloseFileLog;
+import org.apache.iotdb.cluster.log.logtypes.PhysicalPlanLog;
+import org.apache.iotdb.cluster.server.Timer;
+import org.apache.iotdb.cluster.server.Timer.Statistic;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.PartialPath;
+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.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.service.IoTDB;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AsyncDataLogApplier implements LogApplier {
+
+  private static final Logger logger = LoggerFactory.getLogger(AsyncDataLogApplier.class);
+  private static final int CONCURRENT_CONSUMER_NUM = Runtime.getRuntime().availableProcessors();
+  private LogApplier embeddedApplier;
+  private Map<PartialPath, DataLogConsumer> consumerMap;
+  private ExecutorService consumerPool;
+  private String name;
+
+  // a plan that affects multiple sgs should wait until all consumers become empty to assure all
+  // previous logs are applied, such a plan will wait on this condition if it finds any
+  // consumers nonempty, and each time a consumer becomes empty, this will be notified so the
+  // waiting log can start another round of check
+  private final Object consumerEmptyCondition = new Object();
+
+  public AsyncDataLogApplier(LogApplier embeddedApplier, String name) {
+    this.embeddedApplier = embeddedApplier;
+    consumerMap = new ConcurrentHashMap<>();
+    consumerPool = new ThreadPoolExecutor(CONCURRENT_CONSUMER_NUM,
+        Integer.MAX_VALUE, 0, TimeUnit.SECONDS, new SynchronousQueue<>());
+    this.name = name;
+  }
+
+  @Override
+  public void close() {
+    consumerPool.shutdownNow();
+  }
+
+  @Override
+  // synchronized: when a log is draining consumers, avoid other threads adding more logs so that
+  // the consumers will never be drained
+  public synchronized void apply(Log log) {
+
+    PartialPath logKey;
+    try {
+      logKey = getLogKey(log);
+    } catch (StorageGroupNotSetException e) {
+      logger.debug("Exception occurred when applying {}", log, e);
+      log.setException(e);
+      log.setApplied(true);
+      return;
+    }
+
+    if (logKey != null) {
+      // this plan only affects one sg, so we can run it with other plans in parallel
+      long startTime = Statistic.RAFT_SENDER_COMMIT_TO_CONSUMER_LOGS.getOperationStartTime();
+      provideLogToConsumers(logKey, log);
+      Statistic.RAFT_SENDER_COMMIT_TO_CONSUMER_LOGS.calOperationCostTimeFromStart(startTime);
+      return;
+    }
+
+    logger.debug("{}: {} is waiting for consumers to drain", name, log);
+    long startTime = Statistic.RAFT_SENDER_COMMIT_EXCLUSIVE_LOGS.getOperationStartTime();
+    drainConsumers();
+    applyInternal(log);
+    Statistic.RAFT_SENDER_COMMIT_EXCLUSIVE_LOGS.calOperationCostTimeFromStart(startTime);
+  }
+
+  private PartialPath getLogKey(Log log) throws StorageGroupNotSetException {
+    // we can only apply some kinds of plans in parallel, for other logs, we must wait until all
+    // previous logs are applied, or the order of deletions and insertions may get wrong
+    if (log instanceof PhysicalPlanLog) {
+      PhysicalPlanLog physicalPlanLog = (PhysicalPlanLog) log;
+      PhysicalPlan plan = physicalPlanLog.getPlan();
+      // this plan only affects one sg, so we can run it with other plans in parallel
+      return getPlanKey(plan);
+    } else if (log instanceof CloseFileLog) {
+      CloseFileLog closeFileLog = (CloseFileLog) log;
+      PartialPath partialPath = null;
+      try {
+        partialPath = new PartialPath(closeFileLog.getStorageGroupName());
+      } catch (IllegalPathException e) {
+        // unreachable
+      }
+      return partialPath;
+    }
+    return null;
+  }
+
+  private PartialPath getPlanKey(PhysicalPlan plan) throws StorageGroupNotSetException {
+    return getPlanSG(plan);
+  }
+
+  private PartialPath getPlanSG(PhysicalPlan plan) throws StorageGroupNotSetException {
+    PartialPath sgPath = null;
+    if (plan instanceof InsertPlan) {
+      PartialPath deviceId = ((InsertPlan) plan).getDeviceId();
+      sgPath = IoTDB.metaManager.getStorageGroupPath(deviceId);
+    } else if (plan instanceof CreateTimeSeriesPlan) {
+      PartialPath path = ((CreateTimeSeriesPlan) plan).getPath();
+      sgPath = IoTDB.metaManager.getStorageGroupPath(path);
+    }
+    return sgPath;
+  }
+
+  private void provideLogToConsumers(PartialPath planKey, Log log) {
+    if (Timer.ENABLE_INSTRUMENTING) {
+      log.setEnqueueTime(System.nanoTime());
+    }
+    consumerMap.computeIfAbsent(planKey, d -> new DataLogConsumer(name + "-" + d)).accept(log);
+  }
+
+  private void drainConsumers() {
+    synchronized (consumerEmptyCondition) {
+      while (!allConsumersEmpty()) {
+        // wait until all consumers empty
+        try {
+          consumerEmptyCondition.wait(5);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          return;
+        }
+      }
+    }
+  }
+
+  private boolean allConsumersEmpty() {
+    for (DataLogConsumer consumer : consumerMap.values()) {
+      if (!consumer.isEmpty()) {
+        if (logger.isDebugEnabled()) {
+          logger.debug("Consumer not empty: {}", consumer);
+        }
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private void applyInternal(Log log) {
+    long startTime = Statistic.RAFT_SENDER_DATA_LOG_APPLY.getOperationStartTime();
+    embeddedApplier.apply(log);
+    if (Timer.ENABLE_INSTRUMENTING) {
+      Statistic.RAFT_SENDER_DATA_LOG_APPLY.calOperationCostTimeFromStart(startTime);
+    }
+  }
+
+  private class DataLogConsumer implements Runnable, Consumer<Log> {
+
+    private BlockingQueue<Log> logQueue = new ArrayBlockingQueue<>(4096);
+    private volatile long lastLogIndex;
+    private volatile long lastAppliedLogIndex;
+    private String name;
+    private Future<?> future;
+
+    public DataLogConsumer(String name) {
+      this.name = name;
+    }
+
+    public boolean isEmpty() {
+      return lastLogIndex == lastAppliedLogIndex;
+    }
+
+    @Override
+    public void run() {
+      // appliers have a higher priority than normal threads (like client threads and low
+      // priority background threads), to assure fast ingestion, but a lower priority than
+      // heartbeat threads
+      Thread.currentThread().setPriority(8);
+      Thread.currentThread().setName(name);
+      while (!Thread.currentThread().isInterrupted()) {
+        try {
+          Log log = logQueue.take();
+          Statistic.RAFT_SENDER_IN_APPLY_QUEUE.calOperationCostTimeFromStart(log.getEnqueueTime());
+          try {
+            applyInternal(log);
+          } finally {
+            lastAppliedLogIndex = log.getCurrLogIndex();
+            if (isEmpty()) {
+              synchronized (consumerEmptyCondition) {
+                consumerEmptyCondition.notifyAll();
+              }
+            }
+          }
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          break;
+        } catch (Exception e) {
+          logger.error("DataLogConsumer exits", e);
+          return;
+        }
+      }
+      logger.info("DataLogConsumer exits");
+    }
+
+    @Override
+    public void accept(Log log) {
+      if (future == null || future.isCancelled() || future.isDone()) {
+        if (future != null) {
+          try {
+            future.get();
+          } catch (InterruptedException e) {
+            logger.error("Last applier thread exits unexpectedly", e);
+            Thread.currentThread().interrupt();
+          } catch (ExecutionException e) {
+            logger.error("Last applier thread exits unexpectedly", e);
+          }
+        }
+        future = consumerPool.submit(this);
+      }
+
+      try {
+        lastLogIndex = log.getCurrLogIndex();
+        logQueue.put(log);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        log.setException(e);
+        log.setApplied(true);
+        lastAppliedLogIndex = log.getCurrLogIndex();
+      }
+    }
+
+    @Override
+    public String toString() {
+      return "DataLogConsumer{" +
+          "logQueue=" + logQueue.size() +
+          ", lastLogIndex=" + lastLogIndex +
+          ", lastAppliedLogIndex=" + lastAppliedLogIndex +
+          ", name='" + name + '\'' +
+          '}';
+    }
+  }
+}
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
new file mode 100644
index 0000000..75cef69
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/BaseApplier.java
@@ -0,0 +1,150 @@
+/*
+ * 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.log.applier;
+
+import java.util.Collections;
+import org.apache.iotdb.cluster.exception.CheckConsistencyException;
+import org.apache.iotdb.cluster.log.LogApplier;
+import org.apache.iotdb.cluster.metadata.CMManager;
+import org.apache.iotdb.cluster.query.ClusterPlanExecutor;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.server.member.DataGroupMember;
+import org.apache.iotdb.cluster.server.member.MetaGroupMember;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+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.service.IoTDB;
+import org.apache.iotdb.db.utils.SchemaUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * BaseApplier use PlanExecutor to execute PhysicalPlans.
+ */
+abstract class BaseApplier implements LogApplier {
+
+  private static final Logger logger = LoggerFactory.getLogger(BaseApplier.class);
+
+  MetaGroupMember metaGroupMember;
+  private PlanExecutor queryExecutor;
+
+  BaseApplier(MetaGroupMember metaGroupMember) {
+    this.metaGroupMember = metaGroupMember;
+  }
+
+  /**
+   * @param plan
+   * @param dataGroupMember the data group member that is applying the log, null if the log is
+   *                        applied by a meta group member
+   * @throws QueryProcessException
+   * @throws StorageGroupNotSetException
+   * @throws StorageEngineException
+   */
+  void applyPhysicalPlan(PhysicalPlan plan, DataGroupMember dataGroupMember)
+      throws QueryProcessException, StorageGroupNotSetException, StorageEngineException {
+    if (plan instanceof InsertPlan) {
+      processPlanWithTolerance((InsertPlan) plan, dataGroupMember);
+    } else if (plan != null && !plan.isQuery()) {
+      try {
+        getQueryExecutor().processNonQuery(plan);
+      } catch (QueryProcessException e) {
+        if (e.getCause() instanceof StorageGroupNotSetException) {
+          executeAfterSync(plan);
+        } else {
+          throw e;
+        }
+      } catch (StorageGroupNotSetException e) {
+        executeAfterSync(plan);
+      }
+    } else if (plan != null){
+      logger.error("Unsupported physical plan: {}", plan);
+    }
+  }
+
+  private void executeAfterSync(PhysicalPlan plan)
+      throws QueryProcessException, StorageGroupNotSetException, StorageEngineException {
+    try {
+      metaGroupMember.syncLeaderWithConsistencyCheck(true);
+    } catch (CheckConsistencyException ce) {
+      throw new QueryProcessException(ce.getMessage());
+    }
+    getQueryExecutor().processNonQuery(plan);
+  }
+
+  /**
+   * @param plan
+   * @param dataGroupMember the data group member that is applying the log, null if the log is
+   *                        applied by a meta group member
+   * @throws QueryProcessException
+   * @throws StorageGroupNotSetException
+   * @throws StorageEngineException
+   */
+  private void processPlanWithTolerance(InsertPlan plan, DataGroupMember dataGroupMember)
+      throws QueryProcessException, StorageGroupNotSetException, StorageEngineException {
+    try {
+      getQueryExecutor().processNonQuery(plan);
+    } catch (QueryProcessException | StorageGroupNotSetException | StorageEngineException e) {
+      // check if this is caused by metadata missing, if so, pull metadata and retry
+      Throwable metaMissingException = SchemaUtils.findMetaMissingException(e);
+      boolean causedByPathNotExist = metaMissingException instanceof PathNotExistException;
+
+      if (causedByPathNotExist) {
+        if (logger.isDebugEnabled()) {
+          logger.debug("Timeseries is not found locally[{}], try pulling it from another group: {}",
+              metaGroupMember.getName(), e.getCause().getMessage());
+        }
+        pullTimeseriesSchema(plan, dataGroupMember.getHeader());
+        plan.recoverFromFailure();
+        getQueryExecutor().processNonQuery(plan);
+      } else {
+        throw e;
+      }
+    }
+  }
+
+  /**
+   * @param plan
+   * @param ignoredGroup do not pull schema from the group to avoid backward dependency
+   * @throws QueryProcessException
+   */
+  private void pullTimeseriesSchema(InsertPlan plan, Node ignoredGroup)
+      throws QueryProcessException {
+    try {
+      PartialPath path = plan.getDeviceId();
+      ((CMManager) IoTDB.metaManager)
+          .pullTimeSeriesSchemas(Collections.singletonList(path), ignoredGroup);
+    } catch (MetadataException e1) {
+      throw new QueryProcessException(e1);
+    }
+  }
+
+  private PlanExecutor getQueryExecutor() throws QueryProcessException {
+    if (queryExecutor == null) {
+      queryExecutor = new ClusterPlanExecutor(metaGroupMember);
+    }
+    return queryExecutor;
+  }
+}
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
new file mode 100644
index 0000000..8ce84b5
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/DataLogApplier.java
@@ -0,0 +1,114 @@
+/*
+ * 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.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.CloseFileLog;
+import org.apache.iotdb.cluster.log.logtypes.PhysicalPlanLog;
+import org.apache.iotdb.cluster.partition.slot.SlotPartitionTable;
+import org.apache.iotdb.cluster.server.member.DataGroupMember;
+import org.apache.iotdb.cluster.server.member.MetaGroupMember;
+import org.apache.iotdb.cluster.utils.IOUtils;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.service.IoTDB;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DataLogApplier applies logs like data insertion/deletion/update and timeseries creation to
+ * IoTDB.
+ */
+public class DataLogApplier extends BaseApplier {
+
+  private static final Logger logger = LoggerFactory.getLogger(DataLogApplier.class);
+
+  private DataGroupMember dataGroupMember;
+
+  public DataLogApplier(MetaGroupMember metaGroupMember, DataGroupMember dataGroupMember) {
+    super(metaGroupMember);
+    this.dataGroupMember = dataGroupMember;
+  }
+
+  @Override
+  public void apply(Log log) {
+    logger.debug("DataMember [{}] start applying Log {}", dataGroupMember.getName(), log);
+
+    try {
+      if (log instanceof PhysicalPlanLog) {
+        PhysicalPlanLog physicalPlanLog = (PhysicalPlanLog) log;
+        PhysicalPlan plan = physicalPlanLog.getPlan();
+        if (plan instanceof InsertPlan) {
+          applyInsert((InsertPlan) plan);
+        } else {
+          applyPhysicalPlan(plan, dataGroupMember);
+        }
+      } else if (log instanceof CloseFileLog) {
+        CloseFileLog closeFileLog = ((CloseFileLog) log);
+        StorageEngine.getInstance()
+            .closeStorageGroupProcessor(new PartialPath(closeFileLog.getStorageGroupName()),
+                closeFileLog.getPartitionId(),
+                closeFileLog.isSeq(), false);
+      } else {
+        logger.error("Unsupported log: {}", log);
+      }
+    } catch (Exception e) {
+      Throwable rootCause = IOUtils.getRootCause(e);
+      if (!(rootCause instanceof PathNotExistException)) {
+        logger.debug("Exception occurred when applying {}", log, e);
+      }
+      log.setException(e);
+    } finally {
+      log.setApplied(true);
+    }
+  }
+
+  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.getDeviceId());
+    } catch (StorageGroupNotSetException e) {
+      // the sg may not exist because the node does not catch up with the leader, retry after
+      // synchronization
+      try {
+        metaGroupMember.syncLeaderWithConsistencyCheck(true);
+      } catch (CheckConsistencyException ce) {
+        throw new QueryProcessException(ce.getMessage());
+      }
+      sg = IoTDB.metaManager.getStorageGroupPath(plan.getDeviceId());
+    }
+    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
new file mode 100644
index 0000000..d7dd5f9
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/MetaLogApplier.java
@@ -0,0 +1,70 @@
+/*
+ * 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.log.applier;
+
+import org.apache.iotdb.cluster.log.Log;
+import org.apache.iotdb.cluster.log.logtypes.AddNodeLog;
+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.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;
+
+/**
+ * MetaLogApplier applies logs like node addition and storage group creation to IoTDB.
+ */
+public class MetaLogApplier extends BaseApplier {
+
+  private static final Logger logger = LoggerFactory.getLogger(MetaLogApplier.class);
+  private MetaGroupMember member;
+
+  public MetaLogApplier(MetaGroupMember member) {
+    super(member);
+    this.member = member;
+  }
+
+  @Override
+  public void apply(Log log) {
+    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);
+      } else if (log instanceof PhysicalPlanLog) {
+        applyPhysicalPlan(((PhysicalPlanLog) log).getPlan(), null);
+      } else if (log instanceof RemoveNodeLog) {
+        RemoveNodeLog removeNodeLog = ((RemoveNodeLog) log);
+        member.applyRemoveNode(removeNodeLog.getRemovedNode());
+      } else {
+        logger.error("Unsupported log: {} {}", log.getClass().getName(), log);
+      }
+    } catch (StorageEngineException | StorageGroupNotSetException | QueryProcessException e) {
+      logger.debug("Exception occurred when executing {}", log, e);
+      log.setException(e);
+    } finally {
+      log.setApplied(true);
+    }
+  }
+}
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
new file mode 100644
index 0000000..23687ad
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/CatchUpTask.java
@@ -0,0 +1,359 @@
+/*
+ * 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.log.catchup;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.ConcurrentModificationException;
+import java.util.List;
+import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.exception.EntryCompactedException;
+import org.apache.iotdb.cluster.exception.LeaderUnknownException;
+import org.apache.iotdb.cluster.log.Log;
+import org.apache.iotdb.cluster.log.Snapshot;
+import org.apache.iotdb.cluster.log.logtypes.EmptyContentLog;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService.Client;
+import org.apache.iotdb.cluster.server.NodeCharacter;
+import org.apache.iotdb.cluster.server.Peer;
+import org.apache.iotdb.cluster.server.member.RaftMember;
+import org.apache.iotdb.cluster.utils.ClientUtils;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CatchUpTask implements Runnable {
+
+  private static final Logger logger = LoggerFactory.getLogger(CatchUpTask.class);
+
+  private Node node;
+  private Peer peer;
+  private RaftMember raftMember;
+  private Snapshot snapshot;
+  private List<Log> logs;
+  private long lastLogIndex;
+  private boolean abort;
+  private String name;
+
+  public CatchUpTask(Node node, Peer peer, RaftMember raftMember, long lastLogIdx) {
+    this.node = node;
+    this.peer = peer;
+    this.raftMember = raftMember;
+    this.logs = Collections.emptyList();
+    this.snapshot = null;
+    this.lastLogIndex = lastLogIdx;
+    this.name = raftMember.getName() + "@" + System.currentTimeMillis();
+  }
+
+  /**
+   * @return true if a matched index is found so that we can use logs only to catch up, or false if
+   * the catch up must be done with a snapshot.
+   * @throws TException
+   * @throws InterruptedException
+   */
+  private boolean checkMatchIndex()
+      throws TException, InterruptedException, LeaderUnknownException {
+
+    logger.debug("Checking the match index of {}", node);
+    long lo = 0;
+    long hi = 0;
+    long localFirstIndex = 0;
+    try {
+      // to avoid snapshot catch up when index is volatile
+      synchronized (raftMember.getLogManager()) {
+        localFirstIndex = raftMember.getLogManager().getFirstIndex();
+        lo = Math.max(localFirstIndex, peer.getMatchIndex() + 1);
+        hi = raftMember.getLogManager().getLastLogIndex() + 1;
+        logs = raftMember.getLogManager().getEntries(lo, hi);
+      }
+      // this may result from peer's match index being changed concurrently, making the peer
+      // actually catch up now
+      if (logger.isInfoEnabled()) {
+        logger.info(
+            "{}: use {} logs of [{}, {}] to fix log inconsistency with node [{}], "
+                + "local first index: {}",
+            raftMember.getName(), logs.size(), lo, hi, node, localFirstIndex);
+      }
+    } catch (ConcurrentModificationException e) {
+      // ignore
+    } catch (Exception e) {
+      logger.error("Unexpected error in logManager's getEntries during matchIndexCheck", e);
+    }
+
+    if (logs.isEmpty()) {
+      return true;
+    }
+
+    int index = findLastMatchIndex(logs);
+    if (index == -1) {
+      logger.info("Cannot find matched of {} within [{}, {}] in memory", node, lo, hi);
+      if (judgeUseLogsInDiskToCatchUp()) {
+        long startIndex = peer.getMatchIndex() + 1;
+        long endIndex = raftMember.getLogManager().getCommitLogIndex();
+        List<Log> logsInDisk = getLogsInStableEntryManager(startIndex, endIndex);
+        if (!logsInDisk.isEmpty()) {
+          logger.info(
+              "{}, found {} logs in disk to catch up {} , startIndex={}, endIndex={}, memoryFirstIndex={}, getFirstLogIndex={}",
+              name, logsInDisk.size(), node, startIndex, endIndex, localFirstIndex,
+              logsInDisk.get(0).getCurrLogIndex());
+          logs = logsInDisk;
+          return true;
+        }
+      }
+      return false;
+    }
+    long newMatchedIndex = logs.get(index).getCurrLogIndex() - 1;
+    if (newMatchedIndex > lastLogIndex) {
+      logger.info("{}: matched index of {} has moved beyond last log index, node is "
+          + "self-catching-up, abort this catch up to avoid duplicates", name,
+          node);
+      abort = true;
+      return true;
+    }
+    logger.info("{}: {} matches at {}", name, node,
+        newMatchedIndex);
+
+    peer.setMatchIndex(newMatchedIndex);
+    // if follower return RESPONSE.AGREE with this empty log, then start sending real logs from index.
+    logs.subList(0, index).clear();
+    if (logger.isInfoEnabled()) {
+      if (logs.isEmpty()) {
+        logger.info("{}: {} has caught up by previous catch up", name, node);
+      } else {
+        logger.info("{}: makes {} catch up with {} and other {} logs", name,
+            node, logs.get(0), logs.size());
+      }
+    }
+    return true;
+  }
+
+  @SuppressWarnings("squid:S1135")
+  private boolean judgeUseLogsInDiskToCatchUp() {
+    //TODO use log in disk to snapshot first, if the log not found on disk, then use snapshot.
+    if (!ClusterDescriptor.getInstance().getConfig().isEnableRaftLogPersistence()) {
+      return false;
+    }
+    // TODO judge the cost of snapshot and logs in disk
+    return ClusterDescriptor.getInstance().getConfig().isEnableUsePersistLogOnDiskToCatchUp();
+  }
+
+  private List<Log> getLogsInStableEntryManager(long startIndex, long endIndex) {
+    List<Log> logsInDisk = raftMember.getLogManager().getStableEntryManager()
+        .getLogs(startIndex, endIndex);
+    logger.debug("{}, found {} logs in disk to catchup {}, startIndex={}, endIndex={}",
+        raftMember.getName(), logsInDisk.size(), node, startIndex, endIndex);
+    return logsInDisk;
+  }
+
+  /**
+   * return the index of log whose previous log is matched, or -1 when can not found
+   *
+   * @param logs
+   * @return
+   * @throws LeaderUnknownException
+   * @throws TException
+   * @throws InterruptedException
+   */
+  public int findLastMatchIndex(List<Log> logs)
+      throws LeaderUnknownException, TException, InterruptedException {
+    int start = 0;
+    int end = logs.size() - 1;
+    int matchedIndex = -1;
+    while (start <= end) {
+      int mid = start + (end - start) / 2;
+      if (checkMatchIndex(mid)) {
+        start = mid + 1;
+        matchedIndex = mid;
+      } else {
+        end = mid - 1;
+      }
+    }
+    return matchedIndex;
+  }
+
+  /**
+   * @param index the index of a log in logs
+   * @return true if the previous log at logs[index] matches a log in the remote node, false if the
+   * corresponding log cannot be found
+   * @throws LeaderUnknownException
+   * @throws TException
+   * @throws InterruptedException
+   */
+  private boolean checkMatchIndex(int index)
+      throws LeaderUnknownException, TException, InterruptedException {
+    Log log = logs.get(index);
+    synchronized (raftMember.getTerm()) {
+      // make sure this node is still a leader
+      if (raftMember.getCharacter() != NodeCharacter.LEADER) {
+        throw new LeaderUnknownException(raftMember.getAllNodes());
+      }
+    }
+
+    long prevLogIndex = log.getCurrLogIndex() - 1;
+    long prevLogTerm = getPrevLogTerm(index);
+
+    if (prevLogTerm == -1) {
+      // prev log cannot be found, we cannot know whether is matches if it is not the first log
+      return prevLogIndex == -1;
+    }
+
+    boolean matched = checkLogIsMatch(prevLogIndex, prevLogTerm);
+    raftMember.getLastCatchUpResponseTime().put(node, System.currentTimeMillis());
+    logger.info("{} check {}'s matchIndex {} with log [{}]", raftMember.getName(), node,
+        matched ? "succeed" : "failed", log);
+    return matched;
+  }
+
+  /**
+   * @param logIndex the log index needs to check
+   * @param logTerm  the log term need to check
+   * @return true if the log's index and term matches a log in the remote node, false if the
+   * corresponding log cannot be found
+   * @throws TException
+   * @throws InterruptedException
+   */
+  private boolean checkLogIsMatch(long logIndex, long logTerm)
+      throws TException, InterruptedException {
+    boolean matched;
+    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      RaftService.AsyncClient client = raftMember.getAsyncClient(node);
+      if (client == null) {
+        return false;
+      }
+      Node header = raftMember.getHeader();
+      matched = SyncClientAdaptor.matchTerm(client, node, logIndex, logTerm, header);
+    } else {
+      Client client = raftMember.getSyncClient(node);
+      if (client == null) {
+        return false;
+      }
+      try {
+        matched = client.matchTerm(logIndex, logTerm, raftMember.getHeader());
+      } catch (TException e) {
+        client.getInputProtocol().getTransport().close();
+        throw e;
+      } finally {
+        ClientUtils.putBackSyncClient(client);
+      }
+    }
+    return matched;
+  }
+
+  private long getPrevLogTerm(int index) {
+    long prevLogTerm = -1;
+    if (index > 0) {
+      prevLogTerm = logs.get(index - 1).getCurrLogTerm();
+    } else {
+      try {
+        prevLogTerm = raftMember.getLogManager().getTerm(logs.get(0).getCurrLogIndex() - 1);
+      } catch (EntryCompactedException e) {
+        logger.info("Log [{}] is compacted during catchup", logs.get(0).getCurrLogIndex() - 1);
+      }
+    }
+    return prevLogTerm;
+  }
+
+  private void doSnapshot() {
+    try {
+      raftMember.getLogManager().takeSnapshot();
+    } catch (IOException e) {
+      logger.error("Unexpected error when taking snapshot.", e);
+    }
+    snapshot = raftMember.getLogManager().getSnapshot(peer.getMatchIndex());
+    if (logger.isInfoEnabled()) {
+      logger
+          .info("{}: Logs in {} are too old, catch up with snapshot", raftMember.getName(), node);
+    }
+  }
+
+  /**
+   * Remove logs that are contained in the snapshot.
+   */
+  private void removeSnapshotLogs() {
+    Log logToSearch = new EmptyContentLog(snapshot.getLastLogIndex(), snapshot.getLastLogTerm());
+    int pos = Collections
+        .binarySearch(logs, logToSearch, Comparator.comparingLong(Log::getCurrLogIndex));
+    int prevSize = logs.size();
+    if (pos >= 0) {
+      logs.subList(0, pos + 1).clear();
+    } else {
+      int insertPos = -pos - 1;
+      if (insertPos > 0) {
+        logs.subList(0, insertPos).clear();
+      }
+    }
+    logger.info("Logs are reduced from {} to {}", prevSize, logs.size());
+  }
+
+  @Override
+  public void run() {
+    try {
+      boolean findMatchedIndex = checkMatchIndex();
+      if (abort) {
+        peer.resetInconsistentHeartbeatNum();
+        raftMember.getLastCatchUpResponseTime().remove(node);
+        return;
+      }
+      boolean catchUpSucceeded;
+      if (!findMatchedIndex) {
+        logger.info("{}: performing a snapshot catch-up to {}", raftMember.getName(), node);
+        doSnapshot();
+        // snapshot may overlap with logs
+        removeSnapshotLogs();
+        SnapshotCatchUpTask task = new SnapshotCatchUpTask(logs, snapshot, node, raftMember);
+        catchUpSucceeded = task.call();
+      } else {
+        logger.info("{}: performing a log catch-up to {}", raftMember.getName(), node);
+        LogCatchUpTask task = new LogCatchUpTask(logs, node, raftMember);
+        catchUpSucceeded = task.call();
+      }
+      if (catchUpSucceeded) {
+        // the catch up may be triggered by an old heartbeat, and the node may have already
+        // caught up, so logs can be empty
+        if (!logs.isEmpty() || snapshot != null) {
+          long lastIndex = !logs.isEmpty() ? logs.get(logs.size() - 1).getCurrLogIndex() :
+              snapshot.getLastLogIndex();
+          peer.setMatchIndex(lastIndex);
+        }
+        if (logger.isInfoEnabled()) {
+          logger.info("{}: Catch up {} finished, update it's matchIndex to {}",
+              raftMember.getName(), node, peer.getMatchIndex());
+        }
+        peer.resetInconsistentHeartbeatNum();
+      }
+
+    } catch (LeaderUnknownException e) {
+      logger.warn("Catch up {} failed because leadership is lost", node);
+    } catch (Exception e) {
+      logger.error("Catch up {} errored", node, e);
+    }
+    // the next catch up is enabled
+    raftMember.getLastCatchUpResponseTime().remove(node);
+  }
+
+  @TestOnly
+  public void setLogs(List<Log> logs) {
+    this.logs = logs;
+  }
+}
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
new file mode 100644
index 0000000..3520ce4
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/LogCatchUpTask.java
@@ -0,0 +1,343 @@
+/*
+ * 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.log.catchup;
+
+import org.apache.iotdb.cluster.config.ClusterConstant;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.exception.LeaderUnknownException;
+import org.apache.iotdb.cluster.log.Log;
+import org.apache.iotdb.cluster.rpc.thrift.AppendEntriesRequest;
+import org.apache.iotdb.cluster.rpc.thrift.AppendEntryRequest;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService.AsyncClient;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService.Client;
+import org.apache.iotdb.cluster.server.NodeCharacter;
+import org.apache.iotdb.cluster.server.RaftServer;
+import org.apache.iotdb.cluster.server.handlers.caller.LogCatchUpHandler;
+import org.apache.iotdb.cluster.server.handlers.caller.LogCatchUpInBatchHandler;
+import org.apache.iotdb.cluster.server.member.RaftMember;
+import org.apache.iotdb.cluster.utils.ClientUtils;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * LogCatchUpTask sends a list of logs to a node to make the node keep up with the leader.
+ */
+@SuppressWarnings("java:S2274") // enable timeout
+public class LogCatchUpTask implements Callable<Boolean> {
+
+  // sending logs may take longer than normal communications
+  private static final long SEND_LOGS_WAIT_MS = RaftServer.getWriteOperationTimeoutMS();
+  private static final Logger logger = LoggerFactory.getLogger(LogCatchUpTask.class);
+  Node node;
+  RaftMember raftMember;
+  private List<Log> logs;
+  private boolean useBatch = ClusterDescriptor.getInstance().getConfig().isUseBatchInLogCatchUp();
+  boolean abort = false;
+
+  LogCatchUpTask(List<Log> logs, Node node, RaftMember raftMember) {
+    this.logs = logs;
+    this.node = node;
+    this.raftMember = raftMember;
+  }
+
+  @TestOnly
+  LogCatchUpTask(List<Log> logs, Node node, RaftMember raftMember, boolean useBatch) {
+    this.logs = logs;
+    this.node = node;
+    this.raftMember = raftMember;
+    this.useBatch = useBatch;
+  }
+
+  @TestOnly
+  void setUseBatch(boolean useBatch) {
+    this.useBatch = useBatch;
+  }
+
+  void doLogCatchUp() throws TException, InterruptedException, LeaderUnknownException {
+    AppendEntryRequest request = new AppendEntryRequest();
+    if (raftMember.getHeader() != null) {
+      request.setHeader(raftMember.getHeader());
+    }
+    request.setLeader(raftMember.getThisNode());
+    request.setLeaderCommit(raftMember.getLogManager().getCommitLogIndex());
+
+    for (int i = 0; i < logs.size() && !abort; i++) {
+      Log log = logs.get(i);
+      synchronized (raftMember.getTerm()) {
+        // make sure this node is still a leader
+        if (raftMember.getCharacter() != NodeCharacter.LEADER) {
+          throw new LeaderUnknownException(raftMember.getAllNodes());
+        }
+        request.setTerm(raftMember.getTerm().get());
+      }
+      request.setPrevLogIndex(log.getCurrLogIndex() - 1);
+      if (i == 0) {
+        try {
+          request.setPrevLogTerm(raftMember.getLogManager().getTerm(log.getCurrLogIndex() - 1));
+        } catch (Exception e) {
+          logger.error("getTerm failed for newly append entries", e);
+        }
+      } else {
+        request.setPrevLogTerm(logs.get(i - 1).getCurrLogTerm());
+      }
+
+      if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+        abort = !appendEntryAsync(log, request);
+      } else {
+        abort = !appendEntrySync(log, request);
+      }
+    }
+  }
+
+  private boolean appendEntryAsync(Log log, AppendEntryRequest request)
+      throws TException, InterruptedException {
+    LogCatchUpHandler handler = getCatchUpHandler(log, request);
+    synchronized (handler.getAppendSucceed()) {
+      AsyncClient client = raftMember.getAsyncClient(node);
+      if (client == null) {
+        return false;
+      }
+      client.appendEntry(request, handler);
+      raftMember.getLastCatchUpResponseTime().put(node, System.currentTimeMillis());
+      handler.getAppendSucceed().wait(RaftServer.getWriteOperationTimeoutMS());
+    }
+    return handler.getAppendSucceed().get();
+  }
+
+  private LogCatchUpHandler getCatchUpHandler(Log log, AppendEntryRequest request) {
+    AtomicBoolean appendSucceed = new AtomicBoolean(false);
+    LogCatchUpHandler handler = new LogCatchUpHandler();
+    handler.setAppendSucceed(appendSucceed);
+    handler.setRaftMember(raftMember);
+    handler.setFollower(node);
+    handler.setLog(log);
+    request.setEntry(log.serialize());
+    return handler;
+  }
+
+  private boolean appendEntrySync(Log log, AppendEntryRequest request) {
+    LogCatchUpHandler handler = getCatchUpHandler(log, request);
+
+    Client client = raftMember.getSyncClient(node);
+    if (client == null) {
+      logger.error("No available client for {} when append entry", node);
+      return false;
+    }
+
+    try {
+      long result = client.appendEntry(request);
+      handler.onComplete(result);
+      return handler.getAppendSucceed().get();
+    } catch (TException e) {
+      client.getInputProtocol().getTransport().close();
+      handler.onError(e);
+      return false;
+    } finally {
+      ClientUtils.putBackSyncClient(client);
+    }
+  }
+
+  private AppendEntriesRequest prepareRequest(List<ByteBuffer> logList, int startPos) {
+    AppendEntriesRequest request = new AppendEntriesRequest();
+
+    if (raftMember.getHeader() != null) {
+      request.setHeader(raftMember.getHeader());
+    }
+    request.setLeader(raftMember.getThisNode());
+    request.setLeaderCommit(raftMember.getLogManager().getCommitLogIndex());
+
+    synchronized (raftMember.getTerm()) {
+      // make sure this node is still a leader
+      if (raftMember.getCharacter() != NodeCharacter.LEADER) {
+        logger.debug("Leadership is lost when doing a catch-up to {}, aborting", node);
+        abort = true;
+        return null;
+      }
+      request.setTerm(raftMember.getTerm().get());
+    }
+
+    request.setEntries(logList);
+    // set index for raft
+    request.setPrevLogIndex(logs.get(startPos).getCurrLogIndex() - 1);
+    if (startPos != 0) {
+      request.setPrevLogTerm(logs.get(startPos - 1).getCurrLogTerm());
+    } else {
+      try {
+        request.setPrevLogTerm(
+            raftMember.getLogManager().getTerm(logs.get(0).getCurrLogIndex() - 1));
+      } catch (Exception e) {
+        logger.error("getTerm failed for newly append entries", e);
+      }
+    }
+    logger.debug("{}, node={} catchup request={}", raftMember.getName(), node, request);
+    return request;
+  }
+
+  private void doLogCatchUpInBatch() throws TException, InterruptedException {
+    List<ByteBuffer> logList = new ArrayList<>();
+    long totalLogSize = 0;
+    int firstLogPos = 0;
+    boolean batchFull;
+
+    for (int i = 0; i < logs.size() && !abort; i++) {
+
+      ByteBuffer logData = logs.get(i).serialize();
+      int logSize = logData.array().length;
+      if (logSize > IoTDBDescriptor.getInstance().getConfig().getThriftMaxFrameSize()
+          - IoTDBConstant.LEFT_SIZE_IN_REQUEST) {
+        logger.warn("the frame size {} of thrift is too small",
+            IoTDBDescriptor.getInstance().getConfig().getThriftMaxFrameSize());
+        abort = true;
+        return;
+      }
+
+      totalLogSize += logSize;
+      // we should send logs who's size is smaller than the max frame size of thrift
+      // left 200 byte for other fields of AppendEntriesRequest
+      // send at most 100 logs a time to avoid long latency
+      if (totalLogSize >
+          IoTDBDescriptor.getInstance().getConfig().getThriftMaxFrameSize()
+              - IoTDBConstant.LEFT_SIZE_IN_REQUEST) {
+        // batch oversize, send previous batch and add the log to a new batch
+        sendBatchLogs(logList, firstLogPos);
+        logList.add(logData);
+        firstLogPos = i;
+        totalLogSize = logSize;
+      } else {
+        // just add the log the batch
+        logList.add(logData);
+      }
+
+      batchFull = logList.size() >= ClusterConstant.LOG_NUM_IN_BATCH;
+      if (batchFull) {
+        sendBatchLogs(logList, firstLogPos);
+        firstLogPos = i + 1;
+        totalLogSize = 0;
+      }
+    }
+
+    if (!logList.isEmpty()) {
+      sendBatchLogs(logList, firstLogPos);
+    }
+  }
+
+  private void sendBatchLogs(List<ByteBuffer> logList, int firstLogPos)
+      throws TException, InterruptedException {
+    if (logger.isInfoEnabled()) {
+      logger.info("{} send logs from {} num {} for {}", raftMember.getThisNode(),
+          logs.get(firstLogPos).getCurrLogIndex(), logList.size(), node);
+    }
+    AppendEntriesRequest request = prepareRequest(logList, firstLogPos);
+    if (request == null) {
+      return;
+    }
+    // do append entries
+    if (logger.isInfoEnabled()) {
+      logger.info("{}: sending {} logs to {}", raftMember.getName(), node, logList.size());
+    }
+    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      abort = !appendEntriesAsync(logList, request);
+    } else {
+      abort = !appendEntriesSync(logList, request);
+    }
+    if (!abort && logger.isInfoEnabled()) {
+      logger.info("{}: sent {} logs to {}", raftMember.getName(), node, logList.size());
+    }
+    logList.clear();
+  }
+
+  private boolean appendEntriesAsync(List<ByteBuffer> logList, AppendEntriesRequest request)
+      throws TException, InterruptedException {
+    AtomicBoolean appendSucceed = new AtomicBoolean(false);
+
+    LogCatchUpInBatchHandler handler = new LogCatchUpInBatchHandler();
+    handler.setAppendSucceed(appendSucceed);
+    handler.setRaftMember(raftMember);
+    handler.setFollower(node);
+    handler.setLogs(logList);
+    synchronized (appendSucceed) {
+      appendSucceed.set(false);
+      AsyncClient client = raftMember.getAsyncClient(node);
+      if (client == null) {
+        return false;
+      }
+      client.appendEntries(request, handler);
+      raftMember.getLastCatchUpResponseTime().put(node, System.currentTimeMillis());
+      appendSucceed.wait(SEND_LOGS_WAIT_MS);
+    }
+    return appendSucceed.get();
+  }
+
+  private boolean appendEntriesSync(List<ByteBuffer> logList, AppendEntriesRequest request) {
+    AtomicBoolean appendSucceed = new AtomicBoolean(false);
+    LogCatchUpInBatchHandler handler = new LogCatchUpInBatchHandler();
+    handler.setAppendSucceed(appendSucceed);
+    handler.setRaftMember(raftMember);
+    handler.setFollower(node);
+    handler.setLogs(logList);
+
+    Client client = raftMember.getSyncClient(node);
+    if (client == null) {
+      logger.error("No available client for {} when append entries", node);
+      return false;
+    }
+    try {
+      long result = client.appendEntries(request);
+      handler.onComplete(result);
+      return appendSucceed.get();
+    } catch (TException e) {
+      client.getInputProtocol().getTransport().close();
+      handler.onError(e);
+      logger.warn("Failed logs: {}, first index: {}", logList, request.prevLogIndex + 1);
+      return false;
+    } finally {
+      ClientUtils.putBackSyncClient(client);
+    }
+  }
+
+  @Override
+  public Boolean call() throws TException, InterruptedException, LeaderUnknownException {
+    if (logs.isEmpty()) {
+      return true;
+    }
+
+    if (useBatch) {
+      doLogCatchUpInBatch();
+    } else {
+      doLogCatchUp();
+    }
+    logger.info("{}: Catch up {} finished with result {}", raftMember.getName(), node, !abort);
+
+    // the next catch up is enabled
+    raftMember.getLastCatchUpResponseTime().remove(node);
+    return !abort;
+  }
+}
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
new file mode 100644
index 0000000..1a858b0
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/SnapshotCatchUpTask.java
@@ -0,0 +1,153 @@
+/*
+ * 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.log.catchup;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.exception.LeaderUnknownException;
+import org.apache.iotdb.cluster.log.Log;
+import org.apache.iotdb.cluster.log.Snapshot;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+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.SendSnapshotRequest;
+import org.apache.iotdb.cluster.server.NodeCharacter;
+import org.apache.iotdb.cluster.server.handlers.caller.SnapshotCatchUpHandler;
+import org.apache.iotdb.cluster.server.member.RaftMember;
+import org.apache.iotdb.cluster.utils.ClientUtils;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * SnapshotCatchUpTask first sends the snapshot to the stale node then sends the logs to the node.
+ */
+public class SnapshotCatchUpTask extends LogCatchUpTask implements Callable<Boolean> {
+
+  private static final Logger logger = LoggerFactory.getLogger(SnapshotCatchUpTask.class);
+
+  // sending a snapshot may take longer than normal communications
+  private static final long SEND_SNAPSHOT_WAIT_MS = ClusterDescriptor.getInstance().getConfig()
+      .getCatchUpTimeoutMS();
+  private Snapshot snapshot;
+
+  SnapshotCatchUpTask(List<Log> logs, Snapshot snapshot, Node node, RaftMember raftMember) {
+    super(logs, node, raftMember);
+    this.snapshot = snapshot;
+  }
+
+  private void doSnapshotCatchUp()
+      throws TException, InterruptedException, LeaderUnknownException {
+    SendSnapshotRequest request = new SendSnapshotRequest();
+    if (raftMember.getHeader() != null) {
+      request.setHeader(raftMember.getHeader());
+    }
+    logger.info("Start to send snapshot to {}", node);
+    ByteBuffer data = snapshot.serialize();
+    if (logger.isInfoEnabled()) {
+      logger.info("Do snapshot catch up with size {}", data.array().length);
+    }
+    request.setSnapshotBytes(data);
+
+    synchronized (raftMember.getTerm()) {
+      // make sure this node is still a leader
+      if (raftMember.getCharacter() != NodeCharacter.LEADER) {
+        throw new LeaderUnknownException(raftMember.getAllNodes());
+      }
+    }
+
+    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      abort = !sendSnapshotAsync(request);
+    } else {
+      abort = !sendSnapshotSync(request);
+    }
+  }
+
+  @SuppressWarnings("java:S2274") // enable timeout
+  private boolean sendSnapshotAsync(SendSnapshotRequest request)
+      throws TException, InterruptedException {
+    AtomicBoolean succeed = new AtomicBoolean(false);
+    SnapshotCatchUpHandler handler = new SnapshotCatchUpHandler(succeed, node, snapshot);
+    AsyncClient client = raftMember.getAsyncClient(node);
+    if (client == null) {
+      logger.info("{}: client null for node {}", raftMember.getThisNode(), node);
+      abort = true;
+      return false;
+    }
+
+    logger.info("{}: the snapshot request size={}", raftMember.getName(),
+        request.getSnapshotBytes().length);
+    synchronized (succeed) {
+      client.sendSnapshot(request, handler);
+      raftMember.getLastCatchUpResponseTime().put(node, System.currentTimeMillis());
+      succeed.wait(SEND_SNAPSHOT_WAIT_MS);
+    }
+    if (logger.isInfoEnabled()) {
+      logger.info("send snapshot to node {} success {}", raftMember.getThisNode(), succeed.get());
+    }
+    return succeed.get();
+  }
+
+  private boolean sendSnapshotSync(SendSnapshotRequest request) throws TException {
+    logger.info("{}: sending a snapshot request size={} to {}", raftMember.getName(),
+        request.getSnapshotBytes().length, node);
+    Client client = raftMember.getSyncClient(node);
+    if (client == null) {
+      return false;
+    }
+    try {
+      try {
+        client.sendSnapshot(request);
+        logger.info("{}: snapshot is sent to {}", raftMember.getName(), node);
+        return true;
+      } catch (TException e) {
+        client.getInputProtocol().getTransport().close();
+        throw e;
+      }
+    } finally {
+      ClientUtils.putBackSyncClient(client);
+    }
+  }
+
+  @Override
+  public Boolean call() throws InterruptedException, TException, LeaderUnknownException {
+    doSnapshotCatchUp();
+    if (abort) {
+      logger.warn("{}: Snapshot catch up {} failed", raftMember.getName(), node);
+      raftMember.getLastCatchUpResponseTime().remove(node);
+      return false;
+    }
+    logger
+        .info("{}: Snapshot catch up {} finished, begin to catch up log", raftMember.getName(),
+            node);
+    doLogCatchUp();
+    if (!abort) {
+      logger.info("{}: Catch up {} finished", raftMember.getName(), node);
+    } else {
+      logger.warn("{}: Log catch up {} failed", raftMember.getName(), node);
+    }
+    // the next catch up is enabled
+    raftMember.getLastCatchUpResponseTime().remove(node);
+    return !abort;
+  }
+}
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
new file mode 100644
index 0000000..f54725d
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/AddNodeLog.java
@@ -0,0 +1,93 @@
+/*
+ * 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.log.logtypes;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+import org.apache.iotdb.cluster.log.Log;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.db.utils.SerializeUtils;
+
+/**
+ * AddNodeLog records the operation of adding a node into this cluster.
+ */
+public class AddNodeLog extends Log {
+
+  private Node newNode;
+
+  public Node getNewNode() {
+    return newNode;
+  }
+
+  public void setNewNode(Node newNode) {
+    this.newNode = newNode;
+  }
+
+  @Override
+  public ByteBuffer serialize() {
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+    try (DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
+      dataOutputStream.writeByte(Types.ADD_NODE.ordinal());
+      dataOutputStream.writeLong(getCurrLogIndex());
+      dataOutputStream.writeLong(getCurrLogTerm());
+
+      SerializeUtils.serialize(newNode, dataOutputStream);
+    } catch (IOException e) {
+      // ignored
+    }
+    return ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
+  }
+
+  @Override
+  public void deserialize(ByteBuffer buffer) {
+
+    // marker is previously read, remaining fields:
+    // curr index(long), curr term(long)
+    // ipLength(int), inBytes(byte[]), port(int), identifier(int), dataPort(int)
+    setCurrLogIndex(buffer.getLong());
+    setCurrLogTerm(buffer.getLong());
+
+    newNode = new Node();
+    SerializeUtils.deserialize(newNode, buffer);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    if (!super.equals(o)) {
+      return false;
+    }
+    AddNodeLog that = (AddNodeLog) o;
+    return Objects.equals(newNode, that.newNode);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), newNode);
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/CloseFileLog.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/CloseFileLog.java
new file mode 100644
index 0000000..2948d52
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/CloseFileLog.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.log.logtypes;
+
+import static org.apache.iotdb.cluster.log.Log.Types.CLOSE_FILE;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+import org.apache.iotdb.cluster.log.Log;
+import org.apache.iotdb.db.utils.SerializeUtils;
+
+public class CloseFileLog extends Log {
+
+  private String storageGroupName;
+  private boolean isSeq;
+  private long partitionId;
+
+  public CloseFileLog() {
+  }
+
+  public CloseFileLog(String storageGroupName, long partitionId, boolean isSeq) {
+    this.storageGroupName = storageGroupName;
+    this.isSeq = isSeq;
+    this.partitionId = partitionId;
+  }
+
+  @Override
+  public ByteBuffer serialize() {
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(DEFAULT_BUFFER_SIZE);
+    try (DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
+      dataOutputStream.writeByte((byte) CLOSE_FILE.ordinal());
+
+      dataOutputStream.writeLong(getCurrLogIndex());
+      dataOutputStream.writeLong(getCurrLogTerm());
+
+      SerializeUtils.serialize(storageGroupName, dataOutputStream);
+      dataOutputStream.writeBoolean(isSeq);
+      dataOutputStream.writeLong(partitionId);
+
+    } catch (IOException e) {
+      // unreachable
+    }
+
+    return ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
+  }
+
+  @Override
+  public void deserialize(ByteBuffer buffer) {
+
+    setCurrLogIndex(buffer.getLong());
+    setCurrLogTerm(buffer.getLong());
+
+    storageGroupName = SerializeUtils.deserializeString(buffer);
+    isSeq = buffer.get() == 1;
+    partitionId = buffer.getLong();
+  }
+
+  public boolean isSeq() {
+    return isSeq;
+  }
+
+  public String getStorageGroupName() {
+    return storageGroupName;
+  }
+
+  public long getPartitionId() {
+    return partitionId;
+  }
+
+  @Override
+  public String toString() {
+    return "CloseFileLog{" +
+        "storageGroupName='" + storageGroupName + '\'' +
+        ", isSeq=" + isSeq +
+        ", partitionId=" + partitionId +
+        '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    if (!super.equals(o)) {
+      return false;
+    }
+    CloseFileLog that = (CloseFileLog) o;
+    return isSeq == that.isSeq &&
+        Objects.equals(storageGroupName, that.storageGroupName) && partitionId == that.partitionId;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), storageGroupName, partitionId, isSeq);
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/EmptyContentLog.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/EmptyContentLog.java
new file mode 100644
index 0000000..915ab35
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/EmptyContentLog.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.cluster.log.logtypes;
+
+import static org.apache.iotdb.cluster.log.Log.Types.EMPTY_CONTENT;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.iotdb.cluster.log.Log;
+
+public class EmptyContentLog extends Log {
+
+  public EmptyContentLog() {
+  }
+
+  public EmptyContentLog(long index, long term) {
+    this.setCurrLogIndex(index);
+    this.setCurrLogTerm(term);
+  }
+
+  @Override
+  public ByteBuffer serialize() {
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(DEFAULT_BUFFER_SIZE);
+    try (DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
+      dataOutputStream.writeByte((byte) EMPTY_CONTENT.ordinal());
+      dataOutputStream.writeLong(getCurrLogIndex());
+      dataOutputStream.writeLong(getCurrLogTerm());
+    } catch (IOException e) {
+      // unreachable
+    }
+    return ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
+  }
+
+  @Override
+  public void deserialize(ByteBuffer buffer) {
+    setCurrLogIndex(buffer.getLong());
+    setCurrLogTerm(buffer.getLong());
+  }
+
+  @Override
+  public String toString() {
+    return "term:" + getCurrLogTerm() + ",index:" + getCurrLogIndex();
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/LargeTestLog.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/LargeTestLog.java
new file mode 100644
index 0000000..74981a4
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/LargeTestLog.java
@@ -0,0 +1,78 @@
+/*
+ * 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.log.logtypes;
+
+import org.apache.iotdb.cluster.log.Log;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+import static org.apache.iotdb.cluster.log.Log.Types.TEST_LARGE_CONTENT;
+
+public class LargeTestLog extends Log {
+  private ByteBuffer data;
+  public LargeTestLog() {
+    data = ByteBuffer.wrap(new byte[8192]);
+  }
+
+  @Override
+  public ByteBuffer serialize() {
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(DEFAULT_BUFFER_SIZE);
+    try (DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
+      dataOutputStream.writeByte((byte) TEST_LARGE_CONTENT.ordinal());
+      dataOutputStream.writeLong(getCurrLogIndex());
+      dataOutputStream.writeLong(getCurrLogTerm());
+      dataOutputStream.write(data.array());
+    } catch (IOException e) {
+      // unreachable
+    }
+    return ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
+  }
+
+  @Override
+  public void deserialize(ByteBuffer buffer) {
+    setCurrLogIndex(buffer.getLong());
+    setCurrLogTerm(buffer.getLong());
+    data.put(buffer);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (!(obj instanceof LargeTestLog)) {
+      return false;
+    }
+    LargeTestLog obj1 = (LargeTestLog) obj;
+    return getCurrLogIndex() == obj1.getCurrLogIndex() &&
+      getCurrLogTerm() == obj1.getCurrLogTerm();
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(getCurrLogIndex(), getCurrLogTerm());
+  }
+
+  @Override
+  public String toString() {
+    return "LargeTestLog{" + getCurrLogIndex() + "-" + getCurrLogTerm() + "}";
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/PhysicalPlanLog.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/PhysicalPlanLog.java
new file mode 100644
index 0000000..e0927c0
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/PhysicalPlanLog.java
@@ -0,0 +1,112 @@
+/*
+ * 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.log.logtypes;
+
+import static org.apache.iotdb.cluster.log.Log.Types.PHYSICAL_PLAN;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+import org.apache.iotdb.cluster.log.Log;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * PhysicalPlanLog contains a non-partitioned physical plan like set storage group.
+ */
+public class PhysicalPlanLog extends Log {
+
+  private static final Logger logger = LoggerFactory.getLogger(PhysicalPlanLog.class);
+  private PhysicalPlan plan;
+
+  public PhysicalPlanLog() {
+  }
+
+  public PhysicalPlanLog(PhysicalPlan plan) {
+    this.plan = plan;
+  }
+
+  @Override
+  public ByteBuffer serialize() {
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(DEFAULT_BUFFER_SIZE);
+    try (DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
+      dataOutputStream.writeByte((byte) PHYSICAL_PLAN.ordinal());
+
+      dataOutputStream.writeLong(getCurrLogIndex());
+      dataOutputStream.writeLong(getCurrLogTerm());
+
+      plan.serialize(dataOutputStream);
+    } catch (IOException e) {
+      // unreachable
+    }
+
+    return ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
+  }
+
+  @Override
+  public void deserialize(ByteBuffer buffer) {
+    setCurrLogIndex(buffer.getLong());
+    setCurrLogTerm(buffer.getLong());
+
+    try {
+      plan = PhysicalPlan.Factory.create(buffer);
+    } catch (IOException | IllegalPathException e) {
+      logger.error("Cannot parse a physical {}:{} plan {}", getCurrLogIndex(), getCurrLogTerm(),
+          buffer.array().length, e);
+    }
+  }
+
+  public PhysicalPlan getPlan() {
+    return plan;
+  }
+
+  public void setPlan(PhysicalPlan plan) {
+    this.plan = plan;
+  }
+
+  @Override
+  public String toString() {
+    return plan.toString() + ",term:" + getCurrLogTerm() + ",index:" + getCurrLogIndex();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    if (!super.equals(o)) {
+      return false;
+    }
+    PhysicalPlanLog that = (PhysicalPlanLog) o;
+    return Objects.equals(plan, that.plan);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), plan);
+  }
+}
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
new file mode 100644
index 0000000..02d89d0
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/RemoveNodeLog.java
@@ -0,0 +1,87 @@
+/*
+ * 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.log.logtypes;
+
+import java.io.IOException;
+import java.util.Objects;
+import org.apache.iotdb.cluster.log.Log;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.nio.ByteBuffer;
+import org.apache.iotdb.db.utils.SerializeUtils;
+
+public class RemoveNodeLog extends Log {
+
+    private Node removedNode;
+
+    @Override
+    public ByteBuffer serialize() {
+        ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+        try (DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
+            dataOutputStream.writeByte(Types.REMOVE_NODE.ordinal());
+            dataOutputStream.writeLong(getCurrLogIndex());
+            dataOutputStream.writeLong(getCurrLogTerm());
+
+            SerializeUtils.serialize(removedNode, dataOutputStream);
+        } catch (IOException e) {
+            // ignored
+        }
+        return ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
+    }
+
+    @Override
+    public void deserialize(ByteBuffer buffer) {
+        setCurrLogIndex(buffer.getLong());
+        setCurrLogTerm(buffer.getLong());
+
+        removedNode = new Node();
+        SerializeUtils.deserialize(removedNode, buffer);
+    }
+
+    public Node getRemovedNode() {
+        return removedNode;
+    }
+
+    public void setRemovedNode(Node removedNode) {
+        this.removedNode = removedNode;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        if (!super.equals(o)) {
+            return false;
+        }
+        RemoveNodeLog that = (RemoveNodeLog) o;
+        return Objects.equals(removedNode, that.removedNode);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(super.hashCode(), removedNode);
+    }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/CommittedEntryManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/CommittedEntryManager.java
new file mode 100644
index 0000000..259b3e5
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/CommittedEntryManager.java
@@ -0,0 +1,236 @@
+/*
+ * 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.log.manage;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.iotdb.cluster.exception.EntryCompactedException;
+import org.apache.iotdb.cluster.exception.EntryUnavailableException;
+import org.apache.iotdb.cluster.exception.TruncateCommittedEntryException;
+import org.apache.iotdb.cluster.log.Log;
+import org.apache.iotdb.cluster.log.Snapshot;
+import org.apache.iotdb.cluster.log.logtypes.EmptyContentLog;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CommittedEntryManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(CommittedEntryManager.class);
+
+  // memory cache for logs which have been persisted in disk.
+  private List<Log> entries;
+
+  /**
+   * Note that it is better to use applyingSnapshot to update dummy entry immediately after this
+   * instance is created.
+   */
+  CommittedEntryManager(int maxNumOfLogInMem) {
+    entries = Collections.synchronizedList(new ArrayList<>(maxNumOfLogInMem));
+    entries.add(new EmptyContentLog(-1, -1));
+  }
+
+  /**
+   * Overwrite the contents of this object with those of the given snapshot. Note that this function
+   * is only used if you want to override all the contents, otherwise please use
+   * compactEntries(snapshot.lastIndex()).
+   *
+   * @param snapshot snapshot
+   */
+  void applyingSnapshot(Snapshot snapshot) {
+    long localIndex = getDummyIndex();
+    long snapIndex = snapshot.getLastLogIndex();
+    if (localIndex >= snapIndex) {
+      logger.info("requested snapshot is older than the existing snapshot");
+      return;
+    }
+    entries.clear();
+    entries.add(new EmptyContentLog(snapshot.getLastLogIndex(), snapshot.getLastLogTerm()));
+  }
+
+  /**
+   * Return the last entry's index which have been compacted.
+   *
+   * @return dummyIndex
+   */
+  Long getDummyIndex() {
+    return entries.get(0).getCurrLogIndex();
+  }
+
+  /**
+   * Return the first entry's index which have not been compacted.
+   *
+   * @return firstIndex
+   */
+  Long getFirstIndex() {
+    return getDummyIndex() + 1;
+  }
+
+  /**
+   * Return the last entry's index which have been committed and persisted.
+   *
+   * @return getLastIndex
+   */
+  Long getLastIndex() {
+    return getDummyIndex() + entries.size() - 1;
+  }
+
+  /**
+   * Return the entries's size
+   *
+   * @return entries's size
+   */
+  long getTotalSize() {
+    return getLastIndex() - getFirstIndex() + 1;
+  }
+
+  /**
+   * Return the entry's term for given index. Note that the called should ensure index <=
+   * entries[entries.size()-1].index.
+   *
+   * @param index request entry index
+   * @return -1 if index > entries[entries.size()-1].index, throw EntryCompactedException if index <
+   * dummyIndex, or return the entry's term for given index
+   * @throws EntryCompactedException
+   */
+  public long maybeTerm(long index) throws EntryCompactedException {
+    Log log = getEntry(index);
+    if (log == null) {
+      return -1;
+    }
+    return log.getCurrLogTerm();
+  }
+
+  /**
+   * Pack entries from low through high - 1, just like slice (entries[low:high]). dummyIndex < low
+   * <= high. Note that caller must ensure low <= high.
+   *
+   * @param low  request index low bound
+   * @param high request index upper bound
+   */
+  public List<Log> getEntries(long low, long high) {
+    if (low > high) {
+      logger.debug("invalid getEntries: parameter: {} > {}", low, high);
+      return Collections.emptyList();
+    }
+    long dummyIndex = getDummyIndex();
+    if (low <= dummyIndex) {
+      logger.debug(
+          "entries low ({}) is out of bound dummyIndex ({}), adjust parameter 'low' to {}",
+          low, dummyIndex, dummyIndex);
+      low = dummyIndex + 1;
+    }
+    long lastIndex = getLastIndex();
+    if (high > lastIndex + 1) {
+      logger.debug(
+          "entries high ({}) is out of bound lastIndex ({}), adjust parameter 'high' to {}",
+          high, lastIndex, lastIndex);
+      high = lastIndex + 1;
+    }
+    return entries.subList((int) (low - dummyIndex), (int) (high - dummyIndex));
+  }
+
+  /**
+   * Return the entry's log for given index. Note that the called should ensure index <=
+   * entries[entries.size()-1].index.
+   *
+   * @param index request entry index
+   * @return null if index > entries[entries.size()-1].index, throw EntryCompactedException if index
+   * < dummyIndex, or return the entry's log for given index
+   * @throws EntryCompactedException
+   */
+  Log getEntry(long index) throws EntryCompactedException {
+    long dummyIndex = getDummyIndex();
+    if (index < dummyIndex) {
+      logger.debug(
+          "invalid committedEntryManager getEntry: parameter: index({}) < compactIndex({})",
+          index, dummyIndex);
+      throw new EntryCompactedException(index, dummyIndex);
+    }
+    if ((int) (index - dummyIndex) >= entries.size()) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "invalid committedEntryManager getEntry : parameter: index({}) > lastIndex({})",
+            index, getLastIndex());
+      }
+      return null;
+    }
+    return entries.get((int) (index - dummyIndex));
+  }
+
+  /**
+   * Discards all log entries prior to compactIndex.
+   *
+   * @param compactIndex request compactIndex
+   * @throws EntryUnavailableException
+   */
+  void compactEntries(long compactIndex) throws EntryUnavailableException {
+    long dummyIndex = getDummyIndex();
+    if (compactIndex < dummyIndex) {
+      logger.info(
+          "entries before request index ({}) have been compacted, and the compactIndex is ({})",
+          compactIndex, dummyIndex);
+      return;
+    }
+    if (compactIndex > getLastIndex()) {
+      logger
+          .info("compact ({}) is out of bound lastIndex ({})", compactIndex, getLastIndex());
+      throw new EntryUnavailableException(compactIndex, getLastIndex());
+    }
+    int index = (int) (compactIndex - dummyIndex);
+    entries.set(0, new EmptyContentLog(entries.get(index).getCurrLogIndex(),
+        entries.get(index).getCurrLogTerm()));
+    entries.subList(1, index + 1).clear();
+  }
+
+  /**
+   * Append committed entries. This method will truncate conflict entries if it finds
+   * inconsistencies.
+   *
+   * @param appendingEntries request entries
+   * @throws TruncateCommittedEntryException
+   */
+  public void append(List<Log> appendingEntries) throws TruncateCommittedEntryException {
+    if (appendingEntries.isEmpty()) {
+      return;
+    }
+    long offset = appendingEntries.get(0).getCurrLogIndex() - getDummyIndex();
+    if (entries.size() - offset == 0) {
+      entries.addAll(appendingEntries);
+    } else if (entries.size() - offset > 0) {
+      throw new TruncateCommittedEntryException(appendingEntries.get(0).getCurrLogIndex(),
+          getLastIndex());
+    } else {
+      logger.error("missing log entry [last: {}, append at: {}]", getLastIndex(),
+          appendingEntries.get(0).getCurrLogIndex());
+    }
+  }
+
+  @TestOnly
+  CommittedEntryManager(List<Log> entries) {
+    this.entries = entries;
+  }
+
+  @TestOnly
+  List<Log> getAllEntries() {
+    return entries;
+  }
+}
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
new file mode 100644
index 0000000..a3b0153
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/FilePartitionedSnapshotLogManager.java
@@ -0,0 +1,205 @@
+/*
+ * 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.log.manage;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.iotdb.cluster.exception.EntryCompactedException;
+import org.apache.iotdb.cluster.log.LogApplier;
+import org.apache.iotdb.cluster.log.snapshot.FileSnapshot;
+import org.apache.iotdb.cluster.log.snapshot.FileSnapshot.Factory;
+import org.apache.iotdb.cluster.partition.PartitionTable;
+import org.apache.iotdb.cluster.partition.slot.SlotPartitionTable;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.server.member.DataGroupMember;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Different from PartitionedSnapshotLogManager, FilePartitionedSnapshotLogManager does not store
+ * the committed in memory after snapshots, it considers the logs are contained in the TsFiles so it
+ * will record every TsFiles in the slot instead.
+ */
+public class FilePartitionedSnapshotLogManager extends PartitionedSnapshotLogManager<FileSnapshot> {
+
+  private static final Logger logger = LoggerFactory
+      .getLogger(FilePartitionedSnapshotLogManager.class);
+
+  public FilePartitionedSnapshotLogManager(LogApplier logApplier, PartitionTable partitionTable,
+      Node header, Node thisNode, DataGroupMember dataGroupMember) {
+    super(logApplier, partitionTable, header, thisNode, Factory.INSTANCE, dataGroupMember);
+  }
+
+  /**
+   * send FlushPlan to all nodes in one dataGroup
+   */
+  private void syncFlushAllProcessor() {
+    logger.info("{}: Start flush all storage group processor in one data group", getName());
+    Map<String, List<Pair<Long, Boolean>>> storageGroupPartitions = StorageEngine.getInstance()
+        .getWorkingStorageGroupPartitions();
+    if (storageGroupPartitions.size() == 0) {
+      logger.info("{}: no need to flush processor", getName());
+      return;
+    }
+    dataGroupMember.flushFileWhenDoSnapshot(storageGroupPartitions);
+  }
+
+  @Override
+  @SuppressWarnings("java:S1135") // ignore todos
+  public void takeSnapshot() throws IOException {
+    try {
+      logger.info("{}: Taking snapshots, flushing IoTDB", getName());
+      // record current commit index and prevent further logs from being applied, so the
+      // underlying state machine will not change during the snapshotting
+      setBlockAppliedCommitIndex(getCommitLogIndex());
+      // 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();
+      logger.info("{}: Taking snapshots, IoTDB is flushed", getName());
+      // TODO-cluster https://issues.apache.org/jira/browse/IOTDB-820
+      synchronized (this) {
+        collectTimeseriesSchemas();
+        snapshotLastLogIndex = getBlockAppliedCommitIndex();
+        snapshotLastLogTerm = getTerm(snapshotLastLogIndex);
+        collectTsFilesAndFillTimeseriesSchemas();
+        logger.info("{}: Snapshot is taken", getName());
+      }
+    } catch (EntryCompactedException e) {
+      logger.error("failed to do snapshot.", e);
+    } finally {
+      // now further logs can be applied
+      super.resetBlockAppliedCommitIndex();
+    }
+  }
+
+  /**
+   * IMPORTANT, separate the collection timeseries schema from tsfile to avoid the following
+   * situations: If the tsfile is empty at this time (only the metadata is registered, but the
+   * tsfile has not been written yet), then the timeseries schema snapshot can still be generated
+   * and sent to the followers.
+   *
+   * @throws IOException
+   */
+  private void collectTsFilesAndFillTimeseriesSchemas() throws IOException {
+    // 1.collect tsfile
+    collectTsFiles();
+
+    //2.register the measurement
+    for (Map.Entry<Integer, Collection<TimeseriesSchema>> entry : slotTimeseries.entrySet()) {
+      int slotNum = entry.getKey();
+      FileSnapshot snapshot = slotSnapshots.computeIfAbsent(slotNum,
+          s -> new FileSnapshot());
+      if (snapshot.getTimeseriesSchemas().isEmpty()) {
+        snapshot.setTimeseriesSchemas(entry.getValue());
+      }
+    }
+  }
+
+  private void collectTsFiles() throws IOException {
+    slotSnapshots.clear();
+    Map<PartialPath, Map<Long, List<TsFileResource>>> allClosedStorageGroupTsFile = StorageEngine
+        .getInstance().getAllClosedStorageGroupTsFile();
+    List<TsFileResource> createdHardlinks = new ArrayList<>();
+    // group the TsFiles by their slots
+    for (Entry<PartialPath, Map<Long, List<TsFileResource>>> entry :
+        allClosedStorageGroupTsFile.entrySet()) {
+      PartialPath storageGroupName = entry.getKey();
+      Map<Long, List<TsFileResource>> storageGroupsFiles = entry.getValue();
+      for (Entry<Long, List<TsFileResource>> storageGroupFiles : storageGroupsFiles.entrySet()) {
+        Long partitionNum = storageGroupFiles.getKey();
+        List<TsFileResource> resourceList = storageGroupFiles.getValue();
+        if (!collectTsFiles(partitionNum, resourceList, storageGroupName, createdHardlinks)) {
+          // some file is deleted during the collecting, clean created hardlinks and restart
+          // from the beginning
+          for (TsFileResource createdHardlink : createdHardlinks) {
+            createdHardlink.remove();
+          }
+          collectTsFiles();
+          return;
+        }
+      }
+    }
+  }
+
+  /**
+   * Create hardlinks for files in one partition and add them into the corresponding snapshot.
+   *
+   * @param partitionNum
+   * @param resourceList
+   * @param storageGroupName
+   * @param createdHardlinks
+   * @return true if all hardlinks are created successfully or false if some of them failed to
+   * create
+   * @throws IOException
+   */
+  private boolean collectTsFiles(Long partitionNum, List<TsFileResource> resourceList,
+      PartialPath storageGroupName, List<TsFileResource> createdHardlinks) throws IOException {
+    int slotNum = SlotPartitionTable.getSlotStrategy().calculateSlotByPartitionNum(storageGroupName.getFullPath(),
+        partitionNum, ((SlotPartitionTable) partitionTable).getTotalSlotNumbers());
+    FileSnapshot snapshot = slotSnapshots.computeIfAbsent(slotNum,
+        s -> new FileSnapshot());
+    for (TsFileResource tsFileResource : resourceList) {
+      TsFileResource hardlink = tsFileResource.createHardlink();
+      if (hardlink == null) {
+        return false;
+      }
+      createdHardlinks.add(hardlink);
+      logger.debug("{}: File {} is put into snapshot #{}", getName(), tsFileResource, slotNum);
+      snapshot.addFile(hardlink, thisNode, isPlanIndexRangeUnique(tsFileResource, resourceList));
+    }
+    snapshot.getDataFiles().sort(Comparator.comparingLong(TsFileResource::getMaxPlanIndex));
+    return true;
+  }
+
+  /**
+   * Check if the plan index of 'resource' overlaps any one in 'others' from the same time
+   * partition. For example, we have plan {1,2,3,4,5,6}, plan 1 and 6 are written into an
+   * unsequnce file Unseq1, and {2,3} and {4,5} are written to sequence files Seq1 and Seq2
+   * respectively (notice the numbers are just indexes, not timestamps, so they can be written
+   * anywhere if properly constructed). So Unseq1 both overlaps Seq1 and Seq2. If Unseq1 merges
+   * with Seq1 and generated Seq1' (ranges [1, 6]), it will also overlap with Seq2. But if Seq1'
+   * further merge with Seq2, its range remains to be [1,6], and we cannot find any other files
+   * that overlap with it, so we can conclude with confidence that the file contains all plans
+   * within [1,6].
+   * @param resource
+   * @param others
+   * @return
+   */
+  private boolean isPlanIndexRangeUnique(TsFileResource resource, List<TsFileResource> others) {
+    for (TsFileResource other : others) {
+      if (other != resource && other.isPlanIndexOverlap(resource)) {
+        return false;
+      }
+    }
+    return true;
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/MetaSingleSnapshotLogManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/MetaSingleSnapshotLogManager.java
new file mode 100644
index 0000000..ff650e3
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/MetaSingleSnapshotLogManager.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.cluster.log.manage;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iotdb.cluster.log.LogApplier;
+import org.apache.iotdb.cluster.log.Snapshot;
+import org.apache.iotdb.cluster.log.manage.serializable.SyncLogDequeSerializer;
+import org.apache.iotdb.cluster.log.snapshot.MetaSimpleSnapshot;
+import org.apache.iotdb.cluster.server.member.MetaGroupMember;
+import org.apache.iotdb.db.auth.AuthException;
+import org.apache.iotdb.db.auth.authorizer.BasicAuthorizer;
+import org.apache.iotdb.db.auth.authorizer.IAuthorizer;
+import org.apache.iotdb.db.auth.entity.Role;
+import org.apache.iotdb.db.auth.entity.User;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.service.IoTDB;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * MetaSingleSnapshotLogManager provides a MetaSimpleSnapshot as snapshot.
+ */
+public class MetaSingleSnapshotLogManager extends RaftLogManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(MetaSingleSnapshotLogManager.class);
+  private Map<PartialPath, Long> storageGroupTTLMap;
+  private Map<String, User> userMap;
+  private Map<String, Role> roleMap;
+  private MetaGroupMember metaGroupMember;
+  private long commitIndex;
+  private long term;
+
+  public MetaSingleSnapshotLogManager(LogApplier logApplier, MetaGroupMember metaGroupMember) {
+    super(new SyncLogDequeSerializer(0), logApplier, metaGroupMember.getName());
+    this.metaGroupMember = metaGroupMember;
+  }
+
+  @Override
+  @SuppressWarnings("java:S1135") // ignore todos
+  public void takeSnapshot() throws IOException {
+    // TODO-cluster https://issues.apache.org/jira/browse/IOTDB-820
+    super.takeSnapshot();
+    synchronized (this) {
+      storageGroupTTLMap = IoTDB.metaManager.getStorageGroupsTTL();
+      try {
+        IAuthorizer authorizer = BasicAuthorizer.getInstance();
+        userMap = authorizer.getAllUsers();
+        roleMap = authorizer.getAllRoles();
+        commitIndex = getCommitLogIndex();
+        term = getCommitLogTerm();
+      } catch (AuthException e) {
+        logger.error("get user or role info failed", e);
+      }
+    }
+  }
+
+  @Override
+  public Snapshot getSnapshot(long minIndex) {
+    MetaSimpleSnapshot snapshot = new MetaSimpleSnapshot(storageGroupTTLMap, userMap, roleMap,
+        metaGroupMember.getPartitionTable().serialize());
+    snapshot.setLastLogIndex(commitIndex);
+    snapshot.setLastLogTerm(term);
+    return snapshot;
+  }
+}
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
new file mode 100644
index 0000000..1ca26a2
... 56181 lines suppressed ...