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

[iotdb] branch master updated: [IOTDB-1639] Refactoring the cluster class structure to make it consistent with the server module (#4079)

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

tanxinyu 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 278fcf3  [IOTDB-1639] Refactoring the cluster class structure to make it consistent with the server module (#4079)
278fcf3 is described below

commit 278fcf3f5fd4cee8dca4d24dc11c6e2fda0270a6
Author: Potato <TX...@gmail.com>
AuthorDate: Mon Nov 8 20:50:20 2021 +0800

    [IOTDB-1639] Refactoring the cluster class structure to make it consistent with the server module (#4079)
    
    1. Split Thrift RPC service and RPC implementation to make the logic here clear. For detail, refer the discuss: [Cluster-refactor] About refine classes name ยท Issue #3881
    
    2. Weaken the role of MetaGroupMember. metaGroupMember is just an engine for serving meta raft group, which should not control the whole server too deep. Many fields (like coordinator, etc.) are extracted to ClusterIoTDB (renamed from ClusterMain), and ClusterIoTDB is responsible for creating them.
    
    3. Similar with the relationship between StorageEngine and StorageProcessor in the server module, DataGroupMember can be considered as StorageProcessor, we created a DataGroupEngine to control them.
    
    4. Refactored thrift client class hierarchy to reduce the duplication and imported Apache commons-pool the help manager thrift client object.
    
    5. Write related unit tests for new adding code. For existing code, fixed all failed unit test cases
    
    6. Performance verify. The read and write performance of the two branches(cluster-/master) is basically the same.
---
 .gitignore                                         |    1 -
 cli/pom.xml                                        |    5 +
 cluster/pom.xml                                    |    4 +
 .../resources/conf/iotdb-cluster.properties        |    5 -
 cluster/src/assembly/resources/sbin/add-node.bat   |    2 +-
 cluster/src/assembly/resources/sbin/add-node.sh    |    2 +-
 .../src/assembly/resources/sbin/remove-node.bat    |    4 +-
 cluster/src/assembly/resources/sbin/remove-node.sh |    2 +-
 cluster/src/assembly/resources/sbin/start-node.bat |    4 +-
 cluster/src/assembly/resources/sbin/start-node.sh  |    2 +-
 cluster/src/assembly/resources/sbin/stop-node.bat  |    2 +-
 cluster/src/assembly/resources/sbin/stop-node.sh   |    2 +-
 .../java/org/apache/iotdb/cluster/ClientMain.java  |    3 +-
 .../org/apache/iotdb/cluster/ClusterIoTDB.java     |  689 ++++++++++++
 ...ncClientFactory.java => ClusterIoTDBMBean.java} |   28 +-
 .../java/org/apache/iotdb/cluster/ClusterMain.java |  331 ------
 .../apache/iotdb/cluster/client/BaseFactory.java   |   54 +
 .../SyncClientFactory.java => ClientCategory.java} |   30 +-
 .../apache/iotdb/cluster/client/ClientManager.java |  223 ++++
 .../iotdb/cluster/client/ClientPoolFactory.java    |   98 ++
 .../iotdb/cluster/client/DataClientProvider.java   |   95 --
 .../SyncClientFactory.java => IClientManager.java} |   21 +-
 .../cluster/client/async/AsyncBaseFactory.java     |   68 ++
 .../cluster/client/async/AsyncClientFactory.java   |   65 --
 .../cluster/client/async/AsyncClientPool.java      |  216 ----
 .../cluster/client/async/AsyncDataClient.java      |  199 ++--
 .../client/async/AsyncDataHeartbeatClient.java     |   81 --
 .../cluster/client/async/AsyncMetaClient.java      |  147 ++-
 .../client/async/AsyncMetaHeartbeatClient.java     |   81 --
 .../cluster/client/sync/SyncClientAdaptor.java     |   58 +-
 .../iotdb/cluster/client/sync/SyncClientPool.java  |  169 ---
 .../iotdb/cluster/client/sync/SyncDataClient.java  |  114 +-
 .../client/sync/SyncDataHeartbeatClient.java       |   79 --
 .../iotdb/cluster/client/sync/SyncMetaClient.java  |  121 ++-
 .../client/sync/SyncMetaHeartbeatClient.java       |   78 --
 .../apache/iotdb/cluster/config/ClusterConfig.java |    9 -
 .../iotdb/cluster/config/ClusterConstant.java      |   86 ++
 .../iotdb/cluster/config/ClusterDescriptor.java    |   12 +-
 .../iotdb/cluster/coordinator/Coordinator.java     |   55 +-
 .../apache/iotdb/cluster/log/LogDispatcher.java    |   18 +-
 .../iotdb/cluster/log/applier/DataLogApplier.java  |    9 +-
 .../iotdb/cluster/log/catchup/LogCatchUpTask.java  |    5 +-
 .../manage/FilePartitionedSnapshotLogManager.java  |    2 +
 .../log/manage/PartitionedSnapshotLogManager.java  |    5 +-
 .../iotdb/cluster/log/manage/RaftLogManager.java   |   23 +-
 .../iotdb/cluster/log/snapshot/FileSnapshot.java   |   12 +-
 .../cluster/log/snapshot/MetaSimpleSnapshot.java   |    2 +-
 .../cluster/log/snapshot/PullSnapshotTask.java     |    5 +-
 .../apache/iotdb/cluster/metadata/CMManager.java   |  199 ++--
 .../apache/iotdb/cluster/metadata/MetaPuller.java  |   77 +-
 .../iotdb/cluster/partition/PartitionGroup.java    |    5 +
 .../iotdb/cluster/query/ClusterPlanExecutor.java   |  297 +++---
 .../cluster/query/aggregate/ClusterAggregator.java |   33 +-
 .../cluster/query/fill/ClusterPreviousFill.java    |   63 +-
 .../query/groupby/RemoteGroupByExecutor.java       |   81 +-
 .../query/last/ClusterLastQueryExecutor.java       |   67 +-
 .../cluster/query/reader/ClusterReaderFactory.java |   59 +-
 .../iotdb/cluster/query/reader/DataSourceInfo.java |   78 +-
 .../reader/RemoteSeriesReaderByTimestamp.java      |   15 +-
 .../query/reader/RemoteSimpleSeriesReader.java     |   15 +-
 .../query/reader/mult/MultDataSourceInfo.java      |   56 +-
 .../query/reader/mult/RemoteMultSeriesReader.java  |   31 +-
 .../apache/iotdb/cluster/server/ClientServer.java  |  311 ------
 .../iotdb/cluster/server/ClusterRPCService.java    |   62 +-
 .../ClusterRPCServiceMBean.java}                   |   25 +-
 .../iotdb/cluster/server/ClusterTSServiceImpl.java |  172 +++
 .../iotdb/cluster/server/DataClusterServer.java    | 1109 --------------------
 .../iotdb/cluster/server/MetaClusterServer.java    |  379 -------
 .../cluster/server/PullSnapshotHintService.java    |   18 +-
 .../apache/iotdb/cluster/server/RaftServer.java    |  285 -----
 .../org/apache/iotdb/cluster/server/Response.java  |    3 +
 .../iotdb/cluster/server/StoppedMemberManager.java |    9 +-
 .../server/clusterinfo/ClusterInfoServer.java      |    1 +
 .../handlers/caller/AppendNodeEntryHandler.java    |    8 +-
 .../server/handlers/caller/HeartbeatHandler.java   |    6 +-
 .../server/heartbeat/DataHeartbeatServer.java      |   83 --
 .../cluster/server/heartbeat/HeartbeatServer.java  |  212 ----
 .../cluster/server/heartbeat/HeartbeatThread.java  |   29 +-
 .../server/heartbeat/MetaHeartbeatServer.java      |   91 --
 .../server/heartbeat/MetaHeartbeatThread.java      |    6 +
 .../cluster/server/member/DataGroupMember.java     |   90 +-
 .../member/DataGroupMemberMBean.java}              |   21 +-
 .../cluster/server/member/MetaGroupMember.java     |  493 ++++-----
 .../member/MetaGroupMemberMBean.java}              |   25 +-
 .../iotdb/cluster/server/member/RaftMember.java    |  268 ++---
 .../member/RaftMemberMBean.java}                   |   41 +-
 .../server/raft/AbstractDataRaftService.java       |   54 +
 .../server/raft/AbstractMetaRaftService.java       |   52 +
 .../cluster/server/raft/AbstractRaftService.java   |   76 ++
 .../server/raft/DataRaftHeartBeatService.java      |   68 ++
 .../raft/DataRaftHeartBeatServiceMBean.java}       |   22 +-
 .../iotdb/cluster/server/raft/DataRaftService.java |   65 ++
 .../raft/DataRaftServiceMBean.java}                |   22 +-
 .../server/raft/MetaRaftHeartBeatService.java      |   68 ++
 .../raft/MetaRaftHeartBeatServiceMBean.java}       |   22 +-
 .../iotdb/cluster/server/raft/MetaRaftService.java |   65 ++
 .../raft/MetaRaftServiceMBean.java}                |   22 +-
 .../raft/RaftServiceHandler.java}                  |   33 +-
 .../cluster/server/service/DataGroupEngine.java    |  510 +++++++++
 .../service/DataGroupEngineMBean.java}             |   21 +-
 .../server/service/DataGroupServiceImpls.java      |  743 +++++++++++++
 .../cluster/server/service/MetaAsyncService.java   |   20 +-
 .../cluster/server/service/MetaSyncService.java    |   33 +-
 .../apache/iotdb/cluster/utils/ClientUtils.java    |   64 +-
 .../apache/iotdb/cluster/utils/ClusterUtils.java   |   49 -
 .../cluster/utils/nodetool/ClusterMonitor.java     |   28 +-
 .../iotdb/cluster/client/BaseClientTest.java       |  156 +++
 .../iotdb/cluster/client/ClientManagerTest.java    |  212 ++++
 .../cluster/client/ClientPoolFactoryTest.java      |  262 +++++
 .../cluster/client/DataClientProviderTest.java     |  242 -----
 .../iotdb/cluster/client/MockClientManager.java}   |   33 +-
 .../cluster/client/async/AsyncClientPoolTest.java  |  209 ----
 .../cluster/client/async/AsyncDataClientTest.java  |  109 +-
 .../client/async/AsyncDataHeartbeatClientTest.java |   60 --
 .../cluster/client/async/AsyncMetaClientTest.java  |  108 +-
 .../client/async/AsyncMetaHeartbeatClientTest.java |   61 --
 .../cluster/client/sync/SyncClientPoolTest.java    |  167 ---
 .../cluster/client/sync/SyncDataClientTest.java    |  172 ++-
 .../client/sync/SyncDataHeartbeatClientTest.java   |   66 --
 .../cluster/client/sync/SyncMetaClientTest.java    |  170 ++-
 .../client/sync/SyncMetaHeartbeatClientTest.java   |   66 --
 .../cluster/common/TestAsyncClientFactory.java     |   55 -
 .../iotdb/cluster/common/TestAsyncDataClient.java  |    3 +-
 .../iotdb/cluster/common/TestAsyncMetaClient.java  |    9 +-
 .../apache/iotdb/cluster/common/TestSnapshot.java  |    6 +-
 .../cluster/common/TestSyncClientFactory.java      |   88 --
 .../org/apache/iotdb/cluster/common/TestUtils.java |   11 +-
 .../cluster/integration/BaseSingleNodeTest.java    |   16 +-
 .../iotdb/cluster/integration/SingleNodeTest.java  |    7 +-
 .../apache/iotdb/cluster/log/LogParserTest.java    |    6 +-
 .../cluster/log/applier/DataLogApplierTest.java    |  170 +--
 .../cluster/log/applier/MetaLogApplierTest.java    |    4 +-
 .../iotdb/cluster/log/catchup/CatchUpTaskTest.java |    5 -
 .../cluster/log/catchup/LogCatchUpTaskTest.java    |    7 +-
 .../log/catchup/SnapshotCatchUpTaskTest.java       |    9 +-
 .../cluster/log/logtypes/SerializeLogTest.java     |    4 +-
 .../cluster/log/snapshot/DataSnapshotTest.java     |   10 +-
 .../log/snapshot/MetaSimpleSnapshotTest.java       |    2 +-
 .../cluster/log/snapshot/PullSnapshotTaskTest.java |    5 -
 .../iotdb/cluster/partition/MManagerWhiteBox.java  |    2 +-
 .../cluster/partition/SlotPartitionTableTest.java  |    2 +-
 .../cluster/query/manage/QueryCoordinatorTest.java |    8 +-
 .../cluster/query/reader/DatasourceInfoTest.java   |   55 +-
 .../reader/RemoteSeriesReaderByTimestampTest.java  |  132 ++-
 .../query/reader/RemoteSimpleSeriesReaderTest.java |  104 +-
 .../mult/AssignPathManagedMergeReaderTest.java     |  115 +-
 .../reader/mult/RemoteMultSeriesReaderTest.java    |  220 ++--
 .../server/clusterinfo/ClusterInfoServerTest.java  |    5 +-
 .../clusterinfo/ClusterInfoServiceImplTest.java    |   26 +-
 .../caller/AppendGroupEntryHandlerTest.java        |    3 +-
 .../caller/AppendNodeEntryHandlerTest.java         |    3 +-
 .../handlers/caller/ElectionHandlerTest.java       |    3 +-
 .../handlers/caller/HeartbeatHandlerTest.java      |    3 +-
 .../handlers/caller/LogCatchUpHandlerTest.java     |    3 +-
 .../server/heartbeat/DataHeartbeatThreadTest.java  |    5 -
 .../server/heartbeat/HeartbeatThreadTest.java      |   23 +-
 .../server/heartbeat/MetaHeartbeatThreadTest.java  |    5 -
 .../iotdb/cluster/server/member/BaseMember.java    |   60 +-
 .../cluster/server/member/DataGroupMemberTest.java |   33 +-
 .../cluster/server/member/MetaGroupMemberTest.java |  200 ++--
 .../resources/node1conf/iotdb-engine.properties    |   12 +-
 .../resources/node2conf/iotdb-engine.properties    |   12 +-
 .../resources/node3conf/iotdb-engine.properties    |   12 +-
 docs/UserGuide/Appendix/Status-Codes.md            |    1 +
 docs/zh/UserGuide/Appendix/Status-Codes.md         |    2 +
 flink-iotdb-connector/pom.xml                      |    5 +
 jdbc/pom.xml                                       |    5 +
 .../org/apache/iotdb/jdbc/IoTDBConnection.java     |   10 +-
 pom.xml                                            |   16 +-
 .../resources/conf/iotdb-engine.properties         |    1 +
 .../db/concurrent/IoTDBDaemonThreadFactory.java    |   31 +-
 .../db/concurrent/IoTDBThreadPoolFactory.java      |  186 +++-
 .../org/apache/iotdb/db/concurrent/ThreadName.java |   13 +-
 .../db/concurrent/threadpool/IThreadPoolMBean.java |   35 +-
 .../WrappedScheduledExecutorService.java           |  193 ++++
 .../WrappedScheduledExecutorServiceMBean.java      |   22 +-
 .../WrappedSingleThreadExecutorService.java        |  119 +++
 .../WrappedSingleThreadExecutorServiceMBean.java   |   22 +-
 .../WrappedSingleThreadScheduledExecutor.java      |  124 +++
 .../WrappedSingleThreadScheduledExecutorMBean.java |   22 +-
 .../threadpool/WrappedThreadPoolExecutor.java      |   82 ++
 .../threadpool/WrappedThreadPoolExecutorMBean.java |   22 +-
 .../org/apache/iotdb/db/conf/IoTDBConfigCheck.java |   29 +-
 .../org/apache/iotdb/db/conf/IoTDBConstant.java    |    1 +
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |    4 +
 .../apache/iotdb/db/cq/ContinuousQueryService.java |    4 +-
 .../engine/compaction/CompactionTaskManager.java   |    8 +-
 .../cross/inplace/manage/MergeManager.java         |    4 +-
 .../apache/iotdb/db/engine/flush/FlushManager.java |    2 +-
 .../iotdb/db/exception/ConfigurationException.java |   34 +-
 .../org/apache/iotdb/db/metadata/MManager.java     |    5 +-
 .../iotdb/db/rescon/AbstractPoolManager.java       |   11 +
 .../java/org/apache/iotdb/db/service/IoTDB.java    |    5 +-
 .../apache/iotdb/db/service/MetricsService.java    |    4 +-
 .../org/apache/iotdb/db/service/RPCService.java    |   14 +-
 .../iotdb/db/service/RPCServiceThriftHandler.java  |    2 +-
 .../org/apache/iotdb/db/service/ServiceType.java   |   10 +
 .../org/apache/iotdb/db/service/TSServiceImpl.java |    4 +-
 .../org/apache/iotdb/db/service/UpgradeSevice.java |    9 +-
 .../iotdb/db/service/thrift/ThriftService.java     |   22 +-
 .../db/service/thrift/ThriftServiceThread.java     |  213 +++-
 .../iotdb/db/sync/receiver/SyncServerManager.java  |    6 +
 .../iotdb/db/sync/sender/transfer/SyncClient.java  |   11 +-
 .../writelog/manager/MultiFileLogNodeManager.java  |    5 +-
 .../db/writelog/node/ExclusiveWriteLogNode.java    |    1 +
 .../iotdb/db/integration/IoTDBCheckConfigIT.java   |   41 +-
 .../org/apache/iotdb/rpc/RpcTransportFactory.java  |   36 +-
 .../java/org/apache/iotdb/rpc/TSStatusCode.java    |    5 +-
 .../apache/iotdb/session/SessionConnection.java    |    8 +-
 209 files changed, 7623 insertions(+), 7528 deletions(-)

diff --git a/.gitignore b/.gitignore
index 31c481f..cb0dc1a 100644
--- a/.gitignore
+++ b/.gitignore
@@ -3,7 +3,6 @@
 **/logs/*
 **/lib/**
 **/data/**
-**/raft/**
 
 # Python runtime file
 **/__pycache__/**
diff --git a/cli/pom.xml b/cli/pom.xml
index b0c4b24..9bcb861 100644
--- a/cli/pom.xml
+++ b/cli/pom.xml
@@ -80,6 +80,11 @@
             <artifactId>commons-csv</artifactId>
             <version>1.9.0</version>
         </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/cluster/pom.xml b/cluster/pom.xml
index ff950d0..0e9a74e 100644
--- a/cluster/pom.xml
+++ b/cluster/pom.xml
@@ -125,6 +125,10 @@
             <artifactId>powermock-api-mockito2</artifactId>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-pool2</artifactId>
+        </dependency>
     </dependencies>
     <profiles>
         <profile>
diff --git a/cluster/src/assembly/resources/conf/iotdb-cluster.properties b/cluster/src/assembly/resources/conf/iotdb-cluster.properties
index 2b01d5b..9bcf19a 100644
--- a/cluster/src/assembly/resources/conf/iotdb-cluster.properties
+++ b/cluster/src/assembly/resources/conf/iotdb-cluster.properties
@@ -64,11 +64,6 @@ seed_nodes=127.0.0.1:9003
 # 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=1
 
diff --git a/cluster/src/assembly/resources/sbin/add-node.bat b/cluster/src/assembly/resources/sbin/add-node.bat
index 34f90f1..15a708a 100755
--- a/cluster/src/assembly/resources/sbin/add-node.bat
+++ b/cluster/src/assembly/resources/sbin/add-node.bat
@@ -87,7 +87,7 @@ IF EXIST "%IOTDB_CONF%\iotdb-env.bat" (
 
 @setlocal ENABLEDELAYEDEXPANSION ENABLEEXTENSIONS
 set CONF_PARAMS=-a
-if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.cluster.ClusterMain
+if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.cluster.ClusterIoTDB
 if NOT DEFINED JAVA_HOME goto :err
 
 @REM -----------------------------------------------------------------------------
diff --git a/cluster/src/assembly/resources/sbin/add-node.sh b/cluster/src/assembly/resources/sbin/add-node.sh
index 4deaee2..fdd3f65 100755
--- a/cluster/src/assembly/resources/sbin/add-node.sh
+++ b/cluster/src/assembly/resources/sbin/add-node.sh
@@ -75,7 +75,7 @@ CLASSPATH=""
 for f in ${IOTDB_HOME}/lib/*.jar; do
   CLASSPATH=${CLASSPATH}":"$f
 done
-classname=org.apache.iotdb.cluster.ClusterMain
+classname=org.apache.iotdb.cluster.ClusterIoTDB
 
 launch_service()
 {
diff --git a/cluster/src/assembly/resources/sbin/remove-node.bat b/cluster/src/assembly/resources/sbin/remove-node.bat
index 0346771..fc3202b 100755
--- a/cluster/src/assembly/resources/sbin/remove-node.bat
+++ b/cluster/src/assembly/resources/sbin/remove-node.bat
@@ -64,7 +64,7 @@ for %%i in (%*) do (
 	set CONF_PARAMS=!CONF_PARAMS! %%i
 )
 
-if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.cluster.ClusterMain
+if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.cluster.ClusterIoTDB
 if NOT DEFINED JAVA_HOME goto :err
 
 @REM -----------------------------------------------------------------------------
@@ -82,7 +82,7 @@ set CLASSPATH="%IOTDB_HOME%\lib"
 
 @REM For each jar in the IOTDB_HOME lib directory call append to build the CLASSPATH variable.
 set CLASSPATH=%CLASSPATH%;"%IOTDB_HOME%\lib\*"
-set CLASSPATH=%CLASSPATH%;iotdb.ClusterMain
+set CLASSPATH=%CLASSPATH%;iotdb.ClusterIoTDB
 goto okClasspath
 
 :append
diff --git a/cluster/src/assembly/resources/sbin/remove-node.sh b/cluster/src/assembly/resources/sbin/remove-node.sh
index c6f89bb..0cafad1 100755
--- a/cluster/src/assembly/resources/sbin/remove-node.sh
+++ b/cluster/src/assembly/resources/sbin/remove-node.sh
@@ -51,7 +51,7 @@ CLASSPATH=""
 for f in ${IOTDB_HOME}/lib/*.jar; do
   CLASSPATH=${CLASSPATH}":"$f
 done
-classname=org.apache.iotdb.cluster.ClusterMain
+classname=org.apache.iotdb.cluster.ClusterIoTDB
 
 launch_service()
 {
diff --git a/cluster/src/assembly/resources/sbin/start-node.bat b/cluster/src/assembly/resources/sbin/start-node.bat
index 2874aae..a23eaf8 100755
--- a/cluster/src/assembly/resources/sbin/start-node.bat
+++ b/cluster/src/assembly/resources/sbin/start-node.bat
@@ -87,7 +87,7 @@ IF EXIST "%IOTDB_CONF%\iotdb-env.bat" (
 
 @setlocal ENABLEDELAYEDEXPANSION ENABLEEXTENSIONS
 set CONF_PARAMS=-s
-if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.cluster.ClusterMain
+if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.cluster.ClusterIoTDB
 if NOT DEFINED JAVA_HOME goto :err
 
 @REM -----------------------------------------------------------------------------
@@ -105,7 +105,7 @@ set CLASSPATH="%IOTDB_HOME%\lib"
 
 @REM For each jar in the IOTDB_HOME lib directory call append to build the CLASSPATH variable.
 set CLASSPATH=%CLASSPATH%;"%IOTDB_HOME%\lib\*"
-set CLASSPATH=%CLASSPATH%;iotdb.ClusterMain
+set CLASSPATH=%CLASSPATH%;iotdb.ClusterIoTDB
 goto okClasspath
 
 :append
diff --git a/cluster/src/assembly/resources/sbin/start-node.sh b/cluster/src/assembly/resources/sbin/start-node.sh
index 7fcd79f..001407f 100755
--- a/cluster/src/assembly/resources/sbin/start-node.sh
+++ b/cluster/src/assembly/resources/sbin/start-node.sh
@@ -76,7 +76,7 @@ CLASSPATH=""
 for f in ${IOTDB_HOME}/lib/*.jar; do
   CLASSPATH=${CLASSPATH}":"$f
 done
-classname=org.apache.iotdb.cluster.ClusterMain
+classname=org.apache.iotdb.cluster.ClusterIoTDB
 
 launch_service()
 {
diff --git a/cluster/src/assembly/resources/sbin/stop-node.bat b/cluster/src/assembly/resources/sbin/stop-node.bat
index f6338b6..29c54c9 100644
--- a/cluster/src/assembly/resources/sbin/stop-node.bat
+++ b/cluster/src/assembly/resources/sbin/stop-node.bat
@@ -23,5 +23,5 @@ 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
+wmic process where (commandline like "%%iotdb.ClusterIoTDB%%" 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
index c5dba79..4d670c4 100644
--- a/cluster/src/assembly/resources/sbin/stop-node.sh
+++ b/cluster/src/assembly/resources/sbin/stop-node.sh
@@ -19,7 +19,7 @@
 #
 
 
-PIDS=$(ps ax | grep -i 'ClusterMain' | grep java | grep -v grep | awk '{print $1}')
+PIDS=$(ps ax | grep -i 'ClusterIoTDB' | grep java | grep -v grep | awk '{print $1}')
 sig=0
 for every_pid in ${PIDS}
 do
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/ClientMain.java b/cluster/src/main/java/org/apache/iotdb/cluster/ClientMain.java
index a19245b..28b0a3f 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/ClientMain.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/ClientMain.java
@@ -54,7 +54,6 @@ 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;
@@ -277,7 +276,7 @@ public class ClientMain {
   @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));
+    TTransport transport = RpcTransportFactory.INSTANCE.getTransportWithNoTimeout(ip, port);
     transport.open();
     TProtocol protocol =
         IoTDBDescriptor.getInstance().getConfig().isRpcThriftCompressionEnable()
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/ClusterIoTDB.java b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterIoTDB.java
new file mode 100644
index 0000000..200ef2f
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterIoTDB.java
@@ -0,0 +1,689 @@
+/*
+ * 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 org.apache.iotdb.cluster.client.ClientCategory;
+import org.apache.iotdb.cluster.client.ClientManager;
+import org.apache.iotdb.cluster.client.IClientManager;
+import org.apache.iotdb.cluster.client.async.AsyncDataClient;
+import org.apache.iotdb.cluster.client.async.AsyncMetaClient;
+import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
+import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterConstant;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.coordinator.Coordinator;
+import org.apache.iotdb.cluster.exception.ConfigInconsistentException;
+import org.apache.iotdb.cluster.exception.StartUpCheckFailureException;
+import org.apache.iotdb.cluster.metadata.CMManager;
+import org.apache.iotdb.cluster.metadata.MetaPuller;
+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.ClusterRPCService;
+import org.apache.iotdb.cluster.server.ClusterTSServiceImpl;
+import org.apache.iotdb.cluster.server.HardLinkCleaner;
+import org.apache.iotdb.cluster.server.Response;
+import org.apache.iotdb.cluster.server.clusterinfo.ClusterInfoServer;
+import org.apache.iotdb.cluster.server.member.MetaGroupMember;
+import org.apache.iotdb.cluster.server.monitor.NodeReport;
+import org.apache.iotdb.cluster.server.raft.DataRaftHeartBeatService;
+import org.apache.iotdb.cluster.server.raft.DataRaftService;
+import org.apache.iotdb.cluster.server.raft.MetaRaftHeartBeatService;
+import org.apache.iotdb.cluster.server.raft.MetaRaftService;
+import org.apache.iotdb.cluster.server.service.DataGroupEngine;
+import org.apache.iotdb.cluster.server.service.DataGroupServiceImpls;
+import org.apache.iotdb.cluster.server.service.MetaAsyncService;
+import org.apache.iotdb.cluster.server.service.MetaSyncService;
+import org.apache.iotdb.cluster.utils.ClusterUtils;
+import org.apache.iotdb.cluster.utils.nodetool.ClusterMonitor;
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfigCheck;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.ConfigurationException;
+import org.apache.iotdb.db.exception.StartupException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.service.JMXService;
+import org.apache.iotdb.db.service.RegisterManager;
+import org.apache.iotdb.db.service.thrift.ThriftServiceThread;
+import org.apache.iotdb.db.utils.TestOnly;
+
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.iotdb.cluster.config.ClusterConstant.THREAD_POLL_WAIT_TERMINATION_TIME_S;
+import static org.apache.iotdb.cluster.utils.ClusterUtils.UNKNOWN_CLIENT_IP;
+
+/** we do not inherent IoTDB instance, as it may break the singleton mode of IoTDB. */
+public class ClusterIoTDB implements ClusterIoTDBMBean {
+
+  private static final Logger logger = LoggerFactory.getLogger(ClusterIoTDB.class);
+  private final String mbeanName =
+      String.format(
+          "%s:%s=%s", "org.apache.iotdb.cluster.service", IoTDBConstant.JMX_TYPE, "ClusterIoTDB");
+
+  // TODO: better to throw exception if the client can not be get. Then we can remove this field.
+  private boolean printClientConnectionErrorStack = false;
+
+  // 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";
+
+  private MetaGroupMember metaGroupMember;
+
+  private DataGroupEngine dataGroupEngine;
+
+  private Node thisNode;
+  private Coordinator coordinator;
+
+  private final IoTDB iotdb = IoTDB.getInstance();
+
+  // Cluster IoTDB uses a individual registerManager with its parent.
+  private final RegisterManager registerManager = new RegisterManager();
+
+  /**
+   * a single thread pool, every "REPORT_INTERVAL_SEC" seconds, "reportThread" will print the status
+   * of all raft members in this node.
+   */
+  private ScheduledExecutorService reportThread;
+
+  private boolean allowReport = true;
+
+  /** hardLinkCleaner will periodically clean expired hardlinks created during snapshots. */
+  private ScheduledExecutorService hardLinkCleanerThread;
+
+  /**
+   * The clientManager is only used by those instances who do not belong to any DataGroup or
+   * MetaGroup.
+   */
+  private IClientManager clientManager;
+
+  private ClusterIoTDB() {
+    // we do not init anything here, so that we can re-initialize the instance in IT.
+  }
+
+  /** initialize the current node and its services */
+  public boolean initLocalEngines() {
+    ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
+    thisNode = new Node();
+    // set internal rpc ip and ports
+    thisNode.setInternalIp(config.getInternalIp());
+    thisNode.setMetaPort(config.getInternalMetaPort());
+    thisNode.setDataPort(config.getInternalDataPort());
+    // set client rpc ip and ports
+    thisNode.setClientPort(config.getClusterRpcPort());
+    thisNode.setClientIp(IoTDBDescriptor.getInstance().getConfig().getRpcAddress());
+    coordinator = new Coordinator();
+    // local engine
+    TProtocolFactory protocolFactory =
+        ThriftServiceThread.getProtocolFactory(
+            IoTDBDescriptor.getInstance().getConfig().isRpcThriftCompressionEnable());
+    metaGroupMember = new MetaGroupMember(protocolFactory, thisNode, coordinator);
+    IoTDB.setClusterMode();
+    IoTDB.setMetaManager(CMManager.getInstance());
+    ((CMManager) IoTDB.metaManager).setMetaGroupMember(metaGroupMember);
+    ((CMManager) IoTDB.metaManager).setCoordinator(coordinator);
+    MetaPuller.getInstance().init(metaGroupMember);
+
+    // from the scope of the DataGroupEngine,it should be singleton pattern
+    // the way of setting MetaGroupMember in DataGroupEngine may need a better modification in
+    // future commit.
+    DataGroupEngine.setProtocolFactory(protocolFactory);
+    DataGroupEngine.setMetaGroupMember(metaGroupMember);
+    dataGroupEngine = DataGroupEngine.getInstance();
+    clientManager =
+        new ClientManager(
+            ClusterDescriptor.getInstance().getConfig().isUseAsyncServer(),
+            ClientManager.Type.RequestForwardClient);
+    initTasks();
+    try {
+      // we need to check config after initLocalEngines.
+      startServerCheck();
+      JMXService.registerMBean(metaGroupMember, metaGroupMember.getMBeanName());
+    } catch (StartupException e) {
+      logger.error("Failed to check cluster config.", e);
+      stop();
+      return false;
+    }
+    return true;
+  }
+
+  private void initTasks() {
+    reportThread = IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("NodeReportThread");
+    reportThread.scheduleAtFixedRate(
+        this::generateNodeReport,
+        ClusterConstant.REPORT_INTERVAL_SEC,
+        ClusterConstant.REPORT_INTERVAL_SEC,
+        TimeUnit.SECONDS);
+    hardLinkCleanerThread =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("HardLinkCleaner");
+    hardLinkCleanerThread.scheduleAtFixedRate(
+        new HardLinkCleaner(),
+        ClusterConstant.CLEAN_HARDLINK_INTERVAL_SEC,
+        ClusterConstant.CLEAN_HARDLINK_INTERVAL_SEC,
+        TimeUnit.SECONDS);
+  }
+
+  /**
+   * Generate a report containing the status of both MetaGroupMember and DataGroupMembers of this
+   * node. This will help to see if the node is in a consistent and right state during debugging.
+   */
+  private void generateNodeReport() {
+    if (logger.isDebugEnabled() && allowReport) {
+      try {
+        NodeReport report = new NodeReport(thisNode);
+        report.setMetaMemberReport(metaGroupMember.genMemberReport());
+        report.setDataMemberReportList(dataGroupEngine.genMemberReports());
+        logger.debug(report.toString());
+      } catch (Exception e) {
+        logger.error("exception occurred when generating node report", e);
+      }
+    }
+  }
+
+  public static void main(String[] args) {
+    if (args.length < 1) {
+      logger.error(
+          "Usage: <-s|-a|-r> "
+              + "[-D{} <configure folder>] \n"
+              + "-s: start the node as a seed\n"
+              + "-a: start the node as a new node\n"
+              + "-r: remove the node out of the cluster\n",
+          IoTDBConstant.IOTDB_CONF);
+      return;
+    }
+
+    ClusterIoTDB cluster = ClusterIoTDBHolder.INSTANCE;
+    // check config of iotdb,and set some configs in cluster mode
+    try {
+      if (!cluster.serverCheckAndInit()) {
+        return;
+      }
+    } catch (ConfigurationException | IOException e) {
+      logger.error("meet error when doing start checking", e);
+      return;
+    }
+    String mode = args[0];
+    logger.info("Running mode {}", mode);
+
+    // initialize the current node and its services
+    if (!cluster.initLocalEngines()) {
+      logger.error("initLocalEngines error, stop process!");
+      return;
+    }
+
+    // we start IoTDB kernel first. then we start the cluster module.
+    if (MODE_START.equals(mode)) {
+      cluster.activeStartNodeMode();
+    } else if (MODE_ADD.equals(mode)) {
+      cluster.activeAddNodeMode();
+    } else if (MODE_REMOVE.equals(mode)) {
+      try {
+        cluster.doRemoveNode(args);
+      } catch (IOException e) {
+        logger.error("Fail to remove node in cluster", e);
+      }
+    } else {
+      logger.error("Unrecognized mode {}", mode);
+    }
+  }
+
+  private boolean serverCheckAndInit() throws ConfigurationException, IOException {
+    IoTDBConfigCheck.getInstance().checkConfig();
+    // init server's configuration first, because the cluster configuration may read settings from
+    // the server's configuration.
+    IoTDBDescriptor.getInstance().getConfig().setSyncEnable(false);
+    // auto create schema is took over by cluster module, so we disable it in the server module.
+    IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(false);
+    // check cluster config
+    String checkResult = clusterConfigCheck();
+    if (checkResult != null) {
+      logger.error(checkResult);
+      return false;
+    }
+    return true;
+  }
+
+  private String clusterConfigCheck() {
+    try {
+      ClusterDescriptor.getInstance().replaceHostnameWithIp();
+    } catch (Exception e) {
+      return String.format("replace hostname with ip failed, %s", e.getMessage());
+    }
+    ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
+    // check the initial replicateNum and refuse to start when the replicateNum <= 0
+    if (config.getReplicationNum() <= 0) {
+      return String.format(
+          "ReplicateNum should be greater than 0 instead of %d.", config.getReplicationNum());
+    }
+    // check the initial cluster size and refuse to start when the size < quorum
+    int quorum = config.getReplicationNum() / 2 + 1;
+    if (config.getSeedNodeUrls().size() < quorum) {
+      return String.format(
+          "Seed number less than quorum, seed number: %s, quorum: " + "%s.",
+          config.getSeedNodeUrls().size(), quorum);
+    }
+    // TODO: duplicate code
+    Set<Node> seedNodes = new HashSet<>();
+    for (String url : config.getSeedNodeUrls()) {
+      Node node = ClusterUtils.parseNode(url);
+      if (seedNodes.contains(node)) {
+        return String.format(
+            "SeedNodes must not repeat each other. SeedNodes: %s", config.getSeedNodeUrls());
+      }
+      seedNodes.add(node);
+    }
+    return null;
+  }
+
+  /** Start as a seed node */
+  public void activeStartNodeMode() {
+    try {
+      // start iotdb server first
+      IoTDB.getInstance().active();
+      // some work about cluster
+      preInitCluster();
+      // try to build cluster
+      metaGroupMember.buildCluster();
+      // register service after cluster build
+      postInitCluster();
+      // init ServiceImpl to handle request of client
+      startClientRPC();
+    } catch (StartupException
+        | StartUpCheckFailureException
+        | ConfigInconsistentException
+        | QueryProcessException e) {
+      logger.error("Fail to start  server", e);
+      stop();
+    }
+  }
+
+  private void preInitCluster() throws StartupException {
+    stopRaftInfoReport();
+    JMXService.registerMBean(this, mbeanName);
+    // register MetaGroupMember. MetaGroupMember has the same position with "StorageEngine" in the
+    // cluster module.
+    // TODO: it is better to remove coordinator out of metaGroupEngine
+
+    registerManager.register(metaGroupMember);
+    registerManager.register(dataGroupEngine);
+
+    // rpc service initialize
+    DataGroupServiceImpls dataGroupServiceImpls = new DataGroupServiceImpls();
+    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      MetaAsyncService metaAsyncService = new MetaAsyncService(metaGroupMember);
+      MetaRaftHeartBeatService.getInstance().initAsyncedServiceImpl(metaAsyncService);
+      MetaRaftService.getInstance().initAsyncedServiceImpl(metaAsyncService);
+      DataRaftService.getInstance().initAsyncedServiceImpl(dataGroupServiceImpls);
+      DataRaftHeartBeatService.getInstance().initAsyncedServiceImpl(dataGroupServiceImpls);
+    } else {
+      MetaSyncService syncService = new MetaSyncService(metaGroupMember);
+      MetaRaftHeartBeatService.getInstance().initSyncedServiceImpl(syncService);
+      MetaRaftService.getInstance().initSyncedServiceImpl(syncService);
+      DataRaftService.getInstance().initSyncedServiceImpl(dataGroupServiceImpls);
+      DataRaftHeartBeatService.getInstance().initSyncedServiceImpl(dataGroupServiceImpls);
+    }
+    // start RPC service
+    logger.info("start Meta Heartbeat RPC service... ");
+    registerManager.register(MetaRaftHeartBeatService.getInstance());
+    /* TODO: better to start the Meta RPC service until the heartbeatService has elected the leader
+    and quorum of followers have caught up. */
+    logger.info("start Meta RPC service... ");
+    registerManager.register(MetaRaftService.getInstance());
+  }
+
+  private void postInitCluster() throws StartupException {
+    logger.info("start Data Heartbeat RPC service... ");
+    registerManager.register(DataRaftHeartBeatService.getInstance());
+    logger.info("start Data RPC service... ");
+    registerManager.register(DataRaftService.getInstance());
+    // RPC based DBA API
+    registerManager.register(ClusterInfoServer.getInstance());
+    // JMX based DBA API
+    registerManager.register(ClusterMonitor.INSTANCE);
+  }
+
+  private void startClientRPC() throws QueryProcessException, StartupException {
+    // we must wait until the metaGroup established.
+    // So that the ClusterRPCService can work.
+    ClusterTSServiceImpl clusterServiceImpl = new ClusterTSServiceImpl();
+    clusterServiceImpl.setCoordinator(coordinator);
+    clusterServiceImpl.setExecutor(metaGroupMember);
+    ClusterRPCService.getInstance().initSyncedServiceImpl(clusterServiceImpl);
+    registerManager.register(ClusterRPCService.getInstance());
+  }
+
+  /** Be added to the cluster by seed nodes */
+  public void activeAddNodeMode() {
+    try {
+      final long startTime = System.currentTimeMillis();
+      preInitCluster();
+      metaGroupMember.joinCluster();
+      postInitCluster();
+      dataGroupEngine.pullSnapshots();
+      startClientRPC();
+      logger.info(
+          "Adding this node {} to cluster costs {} ms",
+          thisNode,
+          (System.currentTimeMillis() - startTime));
+    } catch (StartupException
+        | QueryProcessException
+        | StartUpCheckFailureException
+        | ConfigInconsistentException e) {
+      stop();
+      logger.error("Fail to join cluster", e);
+    }
+  }
+
+  private void startServerCheck() throws StartupException {
+    ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
+    // 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(metaGroupMember.getName(), message);
+      }
+      seedNodes.add(node);
+    }
+
+    // assert this node is in all nodes when restart
+    if (!metaGroupMember.getAllNodes().isEmpty()) {
+      if (!metaGroupMember.getAllNodes().contains(metaGroupMember.getThisNode())) {
+        String message =
+            String.format(
+                "All nodes in partitionTables must contains local node in start-server mode. "
+                    + "LocalNode: %s, AllNodes: %s",
+                metaGroupMember.getThisNode(), metaGroupMember.getAllNodes());
+        throw new StartupException(metaGroupMember.getName(), message);
+      } else {
+        return;
+      }
+    }
+
+    // assert this node is in seed nodes list
+    if (!seedNodes.contains(thisNode)) {
+      String message =
+          String.format(
+              "SeedNodes must contains local node in start-server mode. LocalNode: %s ,SeedNodes: "
+                  + "%s",
+              thisNode.toString(), config.getSeedNodeUrls());
+      throw new StartupException(metaGroupMember.getName(), message);
+    }
+  }
+
+  private void doRemoveNode(String[] args) throws IOException {
+    if (args.length != 3) {
+      logger.error("Usage: <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.setInternalIp(ip).setMetaPort(metaPort).setClientIp(UNKNOWN_CLIENT_IP);
+    // try sending the request to each seed node
+    for (String url : config.getSeedNodeUrls()) {
+      Node node = ClusterUtils.parseNode(url);
+      if (node == null) {
+        continue;
+      }
+      AsyncMetaClient client =
+          new AsyncMetaClient(factory, new TAsyncClientManager(), node, ClientCategory.META);
+      Long response = null;
+      long startTime = System.currentTimeMillis();
+      try {
+        logger.info("Start removing node {} with the help of node {}", nodeToRemove, node);
+        response = SyncClientAdaptor.removeNode(client, nodeToRemove);
+      } 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, startTime);
+        return;
+      }
+    }
+  }
+
+  private void handleNodeRemovalResp(Long response, Node nodeToRemove, long startTime) {
+    if (response == Response.RESPONSE_AGREE) {
+      logger.info(
+          "Node {} is successfully removed, cost {}ms",
+          nodeToRemove,
+          (System.currentTimeMillis() - startTime));
+    } else if (response == Response.RESPONSE_CLUSTER_TOO_SMALL) {
+      logger.error("Cluster size is too small, cannot remove any node");
+    } else if (response == Response.RESPONSE_REJECT) {
+      logger.error("Node {} is not found in the cluster, please check", nodeToRemove);
+    } else if (response == Response.RESPONSE_DATA_MIGRATION_NOT_FINISH) {
+      logger.warn(
+          "The data migration of the previous membership change operation is not finished. Please "
+              + "try again later");
+    } else {
+      logger.error("Unexpected response {}", response);
+    }
+  }
+
+  /** Developers may perform pre-start customizations here for debugging or experiments. */
+  @SuppressWarnings("java:S125") // leaving examples
+  private void preStartCustomize() {
+    // customize data distribution
+    // The given example tries to divide storage groups like "root.sg_1", "root.sg_2"... into k
+    // nodes evenly, and use default strategy for other groups
+    SlotPartitionTable.setSlotStrategy(
+        new SlotStrategy() {
+          final SlotStrategy defaultStrategy = new SlotStrategy.DefaultStrategy();
+          final int clusterSize =
+              ClusterDescriptor.getInstance().getConfig().getSeedNodeUrls().size();
+
+          @Override
+          public int calculateSlotByTime(String storageGroupName, long timestamp, int maxSlotNum) {
+            int sgSerialNum = extractSerialNumInSGName(storageGroupName) % clusterSize;
+            if (sgSerialNum >= 0) {
+              return maxSlotNum / clusterSize * sgSerialNum;
+            } else {
+              return defaultStrategy.calculateSlotByTime(storageGroupName, timestamp, maxSlotNum);
+            }
+          }
+
+          @Override
+          public int calculateSlotByPartitionNum(
+              String storageGroupName, long partitionId, int maxSlotNum) {
+            int sgSerialNum = extractSerialNumInSGName(storageGroupName) % clusterSize;
+            if (sgSerialNum >= 0) {
+              return maxSlotNum / clusterSize * 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;
+            }
+          }
+        });
+  }
+
+  public void stop() {
+    deactivate();
+  }
+
+  private void deactivate() {
+    logger.info("Deactivating Cluster IoTDB...");
+    stopThreadPools();
+    registerManager.deregisterAll();
+    JMXService.deregisterMBean(mbeanName);
+    logger.info("ClusterIoTDB is deactivated.");
+    // stop the iotdb kernel
+    iotdb.stop();
+  }
+
+  private void stopThreadPools() {
+    stopThreadPool(reportThread, "reportThread");
+    stopThreadPool(hardLinkCleanerThread, "hardLinkCleanerThread");
+  }
+
+  private void stopThreadPool(ExecutorService pool, String name) {
+    if (pool != null) {
+      pool.shutdownNow();
+      try {
+        pool.awaitTermination(THREAD_POLL_WAIT_TERMINATION_TIME_S, TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        logger.error("Unexpected interruption when waiting for {} to end", name, e);
+      }
+    }
+  }
+
+  @TestOnly
+  public void setClientManager(IClientManager clientManager) {
+    this.clientManager = clientManager;
+  }
+
+  @TestOnly
+  public IClientManager getClientManager() {
+    return this.clientManager;
+  }
+
+  @TestOnly
+  public void setDataGroupEngine(DataGroupEngine dataGroupEngine) {
+    this.dataGroupEngine = dataGroupEngine;
+  }
+
+  public MetaGroupMember getMetaGroupMember() {
+    return metaGroupMember;
+  }
+
+  public Node getThisNode() {
+    return thisNode;
+  }
+
+  public Coordinator getCoordinator() {
+    return coordinator;
+  }
+
+  public IoTDB getIotdb() {
+    return iotdb;
+  }
+
+  public RegisterManager getRegisterManager() {
+    return registerManager;
+  }
+
+  public DataGroupEngine getDataGroupEngine() {
+    return dataGroupEngine;
+  }
+
+  public void setMetaGroupMember(MetaGroupMember metaGroupMember) {
+    this.metaGroupMember = metaGroupMember;
+  }
+
+  public static ClusterIoTDB getInstance() {
+    return ClusterIoTDBHolder.INSTANCE;
+  }
+
+  @Override
+  public boolean startRaftInfoReport() {
+    logger.info("Raft status report is enabled.");
+    allowReport = true;
+    return logger.isDebugEnabled();
+  }
+
+  @Override
+  public void stopRaftInfoReport() {
+    logger.info("Raft status report is disabled.");
+    allowReport = false;
+  }
+
+  @Override
+  public void enablePrintClientConnectionErrorStack() {
+    printClientConnectionErrorStack = true;
+  }
+
+  @Override
+  public void disablePrintClientConnectionErrorStack() {
+    printClientConnectionErrorStack = false;
+  }
+
+  public boolean shouldPrintClientConnectionErrorStack() {
+    return printClientConnectionErrorStack;
+  }
+
+  public SyncDataClient getSyncDataClient(Node node, int readOperationTimeoutMS)
+      throws IOException {
+    SyncDataClient dataClient =
+        (SyncDataClient) clientManager.borrowSyncClient(node, ClientCategory.DATA);
+    if (dataClient != null) {
+      dataClient.setTimeout(readOperationTimeoutMS);
+    }
+    return dataClient;
+  }
+
+  public AsyncDataClient getAsyncDataClient(Node node, int readOperationTimeoutMS)
+      throws IOException {
+    AsyncDataClient dataClient =
+        (AsyncDataClient) clientManager.borrowAsyncClient(node, ClientCategory.DATA);
+    if (dataClient != null) {
+      dataClient.setTimeout(readOperationTimeoutMS);
+    }
+    return dataClient;
+  }
+
+  private static class ClusterIoTDBHolder {
+
+    private static final ClusterIoTDB INSTANCE = new ClusterIoTDB();
+
+    private ClusterIoTDBHolder() {}
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterIoTDBMBean.java
similarity index 57%
copy from cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/ClusterIoTDBMBean.java
index 6bfe9003..7fedf0f 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterIoTDBMBean.java
@@ -17,24 +17,20 @@
  * 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.RaftService;
-
-import org.apache.thrift.transport.TTransportException;
-
-import java.io.IOException;
-
-public interface SyncClientFactory {
+package org.apache.iotdb.cluster;
 
+// we do not inherent IoTDB instance, as it may break the singleton mode of IoTDB.
+public interface ClusterIoTDBMBean {
   /**
-   * Get a client which will connect the given node and be cached in the given pool.
+   * try to enable the raft info report.
    *
-   * @param node the cluster node the client will connect.
-   * @param pool the pool that will cache the client for reusing.
-   * @return
-   * @throws IOException
+   * @return true only if the log level is lower than debug and the report is enabled.
    */
-  RaftService.Client getSyncClient(Node node, SyncClientPool pool) throws TTransportException;
+  boolean startRaftInfoReport();
+
+  void stopRaftInfoReport();
+
+  void enablePrintClientConnectionErrorStack();
+
+  void disablePrintClientConnectionErrorStack();
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java
deleted file mode 100644
index 79c86f4..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java
+++ /dev/null
@@ -1,331 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.cluster;
-
-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.server.clusterinfo.ClusterInfoServer;
-import org.apache.iotdb.cluster.utils.ClusterUtils;
-import org.apache.iotdb.db.conf.IoTDBConfigCheck;
-import org.apache.iotdb.db.conf.IoTDBConstant;
-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.iotdb.db.utils.TestOnly;
-
-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;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-
-import static org.apache.iotdb.cluster.utils.ClusterUtils.UNKNOWN_CLIENT_IP;
-
-public class ClusterMain {
-
-  private static final Logger logger = LoggerFactory.getLogger(ClusterMain.class);
-
-  // 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";
-
-  private static MetaClusterServer metaServer;
-
-  public static void main(String[] args) {
-    if (args.length < 1) {
-      logger.error(
-          "Usage: <-s|-a|-r> "
-              + "[-D{} <configure folder>] \n"
-              + "-s: start the node as a seed\n"
-              + "-a: start the node as a new node\n"
-              + "-r: remove the node out of the cluster\n",
-          IoTDBConstant.IOTDB_CONF);
-
-      return;
-    }
-
-    try {
-      IoTDBConfigCheck.getInstance().checkConfig();
-    } catch (IOException e) {
-      logger.error("meet error when doing start checking", e);
-    }
-
-    // init server's configuration first, because the cluster configuration may read settings from
-    // the server's configuration.
-    IoTDBDescriptor.getInstance().getConfig().setSyncEnable(false);
-    // auto create schema is took over by cluster module, so we disable it in the server module.
-    IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(false);
-
-    // params check
-    try {
-      ClusterDescriptor.getInstance().replaceHostnameWithIp();
-    } catch (Exception e) {
-      logger.error("replace hostname with ip failed, {}", e.getMessage());
-      return;
-    }
-
-    String mode = args[0];
-    logger.info("Running mode {}", mode);
-
-    if (MODE_START.equals(mode)) {
-      try {
-        metaServer = new MetaClusterServer();
-        startServerCheck();
-        preStartCustomize();
-        metaServer.start();
-        metaServer.buildCluster();
-        // Currently, we do not register ClusterInfoService as a JMX Bean,
-        // so we use startService() rather than start()
-        ClusterInfoServer.getInstance().startService();
-      } catch (TTransportException
-          | StartupException
-          | QueryProcessException
-          | StartUpCheckFailureException
-          | ConfigInconsistentException e) {
-        metaServer.stop();
-        logger.error("Fail to start meta server", e);
-      }
-    } else if (MODE_ADD.equals(mode)) {
-      try {
-        long startTime = System.currentTimeMillis();
-        metaServer = new MetaClusterServer();
-        preStartCustomize();
-        metaServer.start();
-        metaServer.joinCluster();
-        // Currently, we do not register ClusterInfoService as a JMX Bean,
-        // so we use startService() rather than start()
-        ClusterInfoServer.getInstance().startService();
-
-        logger.info(
-            "Adding this node {} to cluster costs {} ms",
-            metaServer.getMember().getThisNode(),
-            (System.currentTimeMillis() - startTime));
-      } catch (TTransportException
-          | StartupException
-          | QueryProcessException
-          | 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 all nodes when restart
-    if (!metaServer.getMember().getAllNodes().isEmpty()) {
-      if (!metaServer.getMember().getAllNodes().contains(metaServer.getMember().getThisNode())) {
-        String message =
-            String.format(
-                "All nodes in partitionTables must contains local node in start-server mode. "
-                    + "LocalNode: %s, AllNodes: %s",
-                metaServer.getMember().getThisNode(), metaServer.getMember().getAllNodes());
-        throw new StartupException(metaServer.getMember().getName(), message);
-      } else {
-        return;
-      }
-    }
-
-    // assert this node is in seed nodes list
-    Node localNode = new Node();
-    localNode
-        .setInternalIp(config.getInternalIp())
-        .setMetaPort(config.getInternalMetaPort())
-        .setDataPort(config.getInternalDataPort())
-        .setClientPort(config.getClusterRpcPort())
-        .setClientIp(IoTDBDescriptor.getInstance().getConfig().getRpcAddress());
-    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 doRemoveNode(String[] args) throws IOException {
-    if (args.length != 3) {
-      logger.error("Usage: <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.setInternalIp(ip).setMetaPort(metaPort).setClientIp(UNKNOWN_CLIENT_IP);
-    // try sending the request to each seed node
-    for (String url : config.getSeedNodeUrls()) {
-      Node node = ClusterUtils.parseNode(url);
-      if (node == null) {
-        continue;
-      }
-      AsyncMetaClient client = new AsyncMetaClient(factory, new TAsyncClientManager(), node, null);
-      Long response = null;
-      long startTime = System.currentTimeMillis();
-      try {
-        logger.info("Start removing node {} with the help of node {}", nodeToRemove, node);
-        response = SyncClientAdaptor.removeNode(client, nodeToRemove);
-      } 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, startTime);
-        return;
-      }
-    }
-  }
-
-  private static void handleNodeRemovalResp(Long response, Node nodeToRemove, long startTime) {
-    if (response == Response.RESPONSE_AGREE) {
-      logger.info(
-          "Node {} is successfully removed, cost {}ms",
-          nodeToRemove,
-          (System.currentTimeMillis() - startTime));
-    } else if (response == Response.RESPONSE_CLUSTER_TOO_SMALL) {
-      logger.error("Cluster size is too small, cannot remove any node");
-    } else if (response == Response.RESPONSE_REJECT) {
-      logger.error("Node {} is not found in the cluster, please check", nodeToRemove);
-    } else if (response == Response.RESPONSE_DATA_MIGRATION_NOT_FINISH) {
-      logger.warn(
-          "The data migration of the previous membership change operation is not finished. Please try again later");
-    } else {
-      logger.error("Unexpected response {}", response);
-    }
-  }
-
-  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_1", "root.sg_2"... 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;
-            }
-          }
-        });
-  }
-
-  @TestOnly
-  public static void setMetaClusterServer(MetaClusterServer metaClusterServer) {
-    metaServer = metaClusterServer;
-  }
-}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/BaseFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/BaseFactory.java
new file mode 100644
index 0000000..1c60df6
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/BaseFactory.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.cluster.client;
+
+import org.apache.commons.pool2.KeyedPooledObjectFactory;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.thrift.async.TAsyncClientManager;
+import org.apache.thrift.protocol.TProtocolFactory;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public abstract class BaseFactory<K, T> implements KeyedPooledObjectFactory<K, T> {
+
+  protected TAsyncClientManager[] managers;
+  protected TProtocolFactory protocolFactory;
+  protected AtomicInteger clientCnt = new AtomicInteger();
+  protected ClientCategory category;
+  protected IClientManager clientPoolManager;
+
+  protected BaseFactory(TProtocolFactory protocolFactory, ClientCategory category) {
+    this.protocolFactory = protocolFactory;
+    this.category = category;
+  }
+
+  protected BaseFactory(
+      TProtocolFactory protocolFactory, ClientCategory category, IClientManager clientManager) {
+    this.protocolFactory = protocolFactory;
+    this.category = category;
+    this.clientPoolManager = clientManager;
+  }
+
+  @Override
+  public void activateObject(K node, PooledObject<T> pooledObject) throws Exception {}
+
+  @Override
+  public void passivateObject(K node, PooledObject<T> pooledObject) throws Exception {}
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/ClientCategory.java
similarity index 56%
copy from cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/client/ClientCategory.java
index 6bfe9003..fdd6669 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/ClientCategory.java
@@ -17,24 +17,22 @@
  * under the License.
  */
 
-package org.apache.iotdb.cluster.client.sync;
+package org.apache.iotdb.cluster.client;
 
-import org.apache.iotdb.cluster.rpc.thrift.Node;
-import org.apache.iotdb.cluster.rpc.thrift.RaftService;
+public enum ClientCategory {
+  META("MetaClient"),
+  META_HEARTBEAT("MetaHeartbeatClient"),
+  DATA("DataClient"),
+  DATA_HEARTBEAT("DataHeartbeatClient"),
+  DATA_ASYNC_APPEND_CLIENT("DataAsyncAppendClient");
 
-import org.apache.thrift.transport.TTransportException;
+  private final String name;
 
-import java.io.IOException;
+  ClientCategory(String name) {
+    this.name = name;
+  }
 
-public interface SyncClientFactory {
-
-  /**
-   * 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;
+  public String getName() {
+    return name;
+  }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/ClientManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/ClientManager.java
new file mode 100644
index 0000000..ae24a6d
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/ClientManager.java
@@ -0,0 +1,223 @@
+/*
+ * 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 org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService;
+
+import com.google.common.collect.Maps;
+import org.apache.commons.pool2.KeyedObjectPool;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * One should borrow the reusable client from this manager and return the client after use. The
+ * underlying client pool is powered by Apache Commons Pool. The class provided 3 default pool group
+ * according to current usage: RequestForwardClient, DataGroupClients, MetaGroupClients.
+ *
+ * <p>TODO: We can refine the client structure by reorg the interfaces defined in cluster-thrift.
+ */
+public class ClientManager implements IClientManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(ClientManager.class);
+
+  private Map<ClientCategory, KeyedObjectPool<Node, RaftService.AsyncClient>> asyncClientPoolMap;
+  private Map<ClientCategory, KeyedObjectPool<Node, RaftService.Client>> syncClientPoolMap;
+  private final ClientPoolFactory clientPoolFactory;
+
+  /**
+   * {@link ClientManager.Type#RequestForwardClient} represents the clients used to forward external
+   * client requests to proper node to handle such as query, insert request.
+   *
+   * <p>{@link ClientManager.Type#DataGroupClient} represents the clients used to appendEntry,
+   * appendEntries, sendHeartbeat, etc for data raft group.
+   *
+   * <p>{@link ClientManager.Type#MetaGroupClient} represents the clients used to appendEntry,
+   * appendEntries, sendHeartbeat, etc for meta raft group. *
+   */
+  public enum Type {
+    RequestForwardClient,
+    DataGroupClient,
+    MetaGroupClient
+  }
+
+  public ClientManager(boolean isAsyncMode, Type type) {
+    clientPoolFactory = new ClientPoolFactory();
+    clientPoolFactory.setClientManager(this);
+    if (isAsyncMode) {
+      asyncClientPoolMap = Maps.newHashMap();
+      constructAsyncClientMap(type);
+    } else {
+      syncClientPoolMap = Maps.newHashMap();
+      constructSyncClientMap(type);
+    }
+  }
+
+  private void constructAsyncClientMap(Type type) {
+    switch (type) {
+      case RequestForwardClient:
+        asyncClientPoolMap.put(
+            ClientCategory.DATA, clientPoolFactory.createAsyncDataPool(ClientCategory.DATA));
+        break;
+      case MetaGroupClient:
+        asyncClientPoolMap.put(
+            ClientCategory.META, clientPoolFactory.createAsyncMetaPool(ClientCategory.META));
+        asyncClientPoolMap.put(
+            ClientCategory.META_HEARTBEAT,
+            clientPoolFactory.createAsyncMetaPool(ClientCategory.META_HEARTBEAT));
+        break;
+      case DataGroupClient:
+        asyncClientPoolMap.put(
+            ClientCategory.DATA, clientPoolFactory.createAsyncDataPool(ClientCategory.DATA));
+        asyncClientPoolMap.put(
+            ClientCategory.DATA_HEARTBEAT,
+            clientPoolFactory.createAsyncDataPool(ClientCategory.DATA_HEARTBEAT));
+        asyncClientPoolMap.put(
+            ClientCategory.DATA_ASYNC_APPEND_CLIENT,
+            clientPoolFactory.createSingleManagerAsyncDataPool());
+        break;
+      default:
+        logger.warn("unsupported ClientManager type: {}", type);
+        break;
+    }
+  }
+
+  private void constructSyncClientMap(Type type) {
+    switch (type) {
+      case RequestForwardClient:
+        syncClientPoolMap.put(
+            ClientCategory.DATA, clientPoolFactory.createSyncDataPool(ClientCategory.DATA));
+        break;
+      case MetaGroupClient:
+        syncClientPoolMap.put(
+            ClientCategory.META, clientPoolFactory.createSyncMetaPool(ClientCategory.META));
+        syncClientPoolMap.put(
+            ClientCategory.META_HEARTBEAT,
+            clientPoolFactory.createSyncMetaPool(ClientCategory.META_HEARTBEAT));
+        break;
+      case DataGroupClient:
+        syncClientPoolMap.put(
+            ClientCategory.DATA, clientPoolFactory.createSyncDataPool(ClientCategory.DATA));
+        syncClientPoolMap.put(
+            ClientCategory.DATA_HEARTBEAT,
+            clientPoolFactory.createSyncDataPool(ClientCategory.DATA_HEARTBEAT));
+        break;
+      default:
+        logger.warn("unsupported ClientManager type: {}", type);
+        break;
+    }
+  }
+
+  /**
+   * It's safe to convert: 1. RaftService.AsyncClient to TSDataService.AsyncClient when category is
+   * DATA or DATA_HEARTBEAT; 2. RaftService.AsyncClient to TSMetaService.AsyncClient when category
+   * is META or META_HEARTBEAT.
+   *
+   * @return RaftService.AsyncClient
+   */
+  @Override
+  public RaftService.AsyncClient borrowAsyncClient(Node node, ClientCategory category)
+      throws IOException {
+    KeyedObjectPool<Node, RaftService.AsyncClient> pool;
+    RaftService.AsyncClient client = null;
+    if (asyncClientPoolMap != null && (pool = asyncClientPoolMap.get(category)) != null) {
+      try {
+        client = pool.borrowObject(node);
+      } catch (IOException e) {
+        // external needs the IOException to check connection
+        throw e;
+      } catch (Exception e) {
+        // external doesn't care of other exceptions
+        logger.error("BorrowAsyncClient fail.", e);
+      }
+    } else {
+      logger.warn(
+          "BorrowSyncClient invoke on unsupported mode or category: Node:{}, ClientCategory:{}, "
+              + "isSyncMode:{}",
+          node,
+          clientPoolFactory,
+          syncClientPoolMap != null);
+    }
+    return client;
+  }
+
+  /**
+   * It's safe to convert: 1. RaftService.Client to TSDataService.Client when category is DATA or
+   * DATA_HEARTBEAT; 2. RaftService.Client to TSMetaService.Client when category is META or
+   * META_HEARTBEAT.
+   *
+   * @return RaftService.Client
+   */
+  @Override
+  public RaftService.Client borrowSyncClient(Node node, ClientCategory category)
+      throws IOException {
+    KeyedObjectPool<Node, RaftService.Client> pool;
+    RaftService.Client client = null;
+    if (syncClientPoolMap != null && (pool = syncClientPoolMap.get(category)) != null) {
+      try {
+        client = pool.borrowObject(node);
+      } catch (TTransportException e) {
+        // external needs to check transport related exception
+        throw new IOException(e);
+      } catch (IOException e) {
+        // external needs the IOException to check connection
+        throw e;
+      } catch (Exception e) {
+        // external doesn't care of other exceptions
+        logger.error("BorrowSyncClient fail.", e);
+      }
+    } else {
+      logger.warn(
+          "BorrowSyncClient invoke on unsupported mode or category: Node:{}, ClientCategory:{}, "
+              + "isSyncMode:{}",
+          node,
+          clientPoolFactory,
+          syncClientPoolMap != null);
+    }
+    return client;
+  }
+
+  @Override
+  public void returnAsyncClient(
+      RaftService.AsyncClient client, Node node, ClientCategory category) {
+    if (client != null && node != null) {
+      try {
+        asyncClientPoolMap.get(category).returnObject(node, client);
+      } catch (Exception e) {
+        logger.error("AsyncClient return error: {}", client, e);
+      }
+    }
+  }
+
+  @Override
+  public void returnSyncClient(RaftService.Client client, Node node, ClientCategory category) {
+    if (client != null && node != null) {
+      try {
+        syncClientPoolMap.get(category).returnObject(node, client);
+      } catch (Exception e) {
+        logger.error("SyncClient return error: {}", client, e);
+      }
+    }
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/ClientPoolFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/ClientPoolFactory.java
new file mode 100644
index 0000000..0887992
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/ClientPoolFactory.java
@@ -0,0 +1,98 @@
+/*
+ * 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 org.apache.iotdb.cluster.client.async.AsyncDataClient;
+import org.apache.iotdb.cluster.client.async.AsyncMetaClient;
+import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.client.sync.SyncMetaClient;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+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.commons.pool2.impl.GenericKeyedObjectPool;
+import org.apache.commons.pool2.impl.GenericKeyedObjectPoolConfig;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocolFactory;
+
+import java.time.Duration;
+
+public class ClientPoolFactory {
+
+  protected long waitClientTimeoutMS;
+  protected int maxConnectionForEachNode;
+  private final TProtocolFactory protocolFactory;
+  private GenericKeyedObjectPoolConfig poolConfig;
+  private IClientManager clientManager;
+
+  public ClientPoolFactory() {
+    ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
+    this.waitClientTimeoutMS = config.getWaitClientTimeoutMS();
+    this.maxConnectionForEachNode = config.getMaxClientPerNodePerMember();
+    protocolFactory =
+        config.isRpcThriftCompressionEnabled()
+            ? new TCompactProtocol.Factory()
+            : new TBinaryProtocol.Factory();
+    poolConfig = new GenericKeyedObjectPoolConfig();
+    poolConfig.setMaxTotalPerKey(maxConnectionForEachNode);
+    poolConfig.setMaxWait(Duration.ofMillis(waitClientTimeoutMS));
+    poolConfig.setTestOnReturn(true);
+    poolConfig.setTestOnBorrow(true);
+  }
+
+  public void setClientManager(IClientManager clientManager) {
+    this.clientManager = clientManager;
+  }
+
+  public GenericKeyedObjectPool<Node, RaftService.Client> createSyncDataPool(
+      ClientCategory category) {
+    return new GenericKeyedObjectPool<>(
+        new SyncDataClient.SyncDataClientFactory(protocolFactory, category, clientManager),
+        poolConfig);
+  }
+
+  public GenericKeyedObjectPool<Node, RaftService.Client> createSyncMetaPool(
+      ClientCategory category) {
+    return new GenericKeyedObjectPool<>(
+        new SyncMetaClient.SyncMetaClientFactory(protocolFactory, category, clientManager),
+        poolConfig);
+  }
+
+  public GenericKeyedObjectPool<Node, RaftService.AsyncClient> createAsyncDataPool(
+      ClientCategory category) {
+    return new GenericKeyedObjectPool<>(
+        new AsyncDataClient.AsyncDataClientFactory(protocolFactory, category, clientManager),
+        poolConfig);
+  }
+
+  public GenericKeyedObjectPool<Node, RaftService.AsyncClient> createAsyncMetaPool(
+      ClientCategory category) {
+    return new GenericKeyedObjectPool<>(
+        new AsyncMetaClient.AsyncMetaClientFactory(protocolFactory, category, clientManager),
+        poolConfig);
+  }
+
+  public GenericKeyedObjectPool<Node, RaftService.AsyncClient> createSingleManagerAsyncDataPool() {
+    return new GenericKeyedObjectPool<>(
+        new AsyncDataClient.SingleManagerFactory(protocolFactory, clientManager), poolConfig);
+  }
+}
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
deleted file mode 100644
index 8b954ec..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/DataClientProvider.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.cluster.client;
-
-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.iotdb.cluster.rpc.thrift.RaftService.Client;
-
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TProtocolFactory;
-
-import java.io.IOException;
-
-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));
-    }
-  }
-
-  AsyncClientPool getDataAsyncClientPool() {
-    return dataAsyncClientPool;
-  }
-
-  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);
-    if (client == null) {
-      throw new IOException("can not get client for node=" + node);
-    }
-    client.setTimeout(timeout);
-    return client;
-  }
-
-  /**
-   * IMPORTANT!!! After calling this function, the caller should make sure to call {@link
-   * org.apache.iotdb.cluster.utils.ClientUtils#putBackSyncClient(Client)} to put the client back
-   * into the client pool, otherwise there is a risk of client leakage.
-   *
-   * <p>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) throws TException {
-    SyncDataClient client = (SyncDataClient) getDataSyncClientPool().getClient(node);
-    if (client == null) {
-      throw new TException("can not get client for node=" + node);
-    }
-    client.setTimeout(timeout);
-    return client;
-  }
-}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/IClientManager.java
similarity index 65%
copy from cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/client/IClientManager.java
index 6bfe9003..6652e0c 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/IClientManager.java
@@ -17,24 +17,19 @@
  * under the License.
  */
 
-package org.apache.iotdb.cluster.client.sync;
+package org.apache.iotdb.cluster.client;
 
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.RaftService;
 
-import org.apache.thrift.transport.TTransportException;
-
 import java.io.IOException;
 
-public interface SyncClientFactory {
+public interface IClientManager {
+  RaftService.AsyncClient borrowAsyncClient(Node node, ClientCategory category) throws IOException;
+
+  RaftService.Client borrowSyncClient(Node node, ClientCategory category) throws IOException;
+
+  void returnAsyncClient(RaftService.AsyncClient client, Node node, ClientCategory category);
 
-  /**
-   * 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;
+  void returnSyncClient(RaftService.Client client, Node node, ClientCategory category);
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncBaseFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncBaseFactory.java
new file mode 100644
index 0000000..b2b8415
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncBaseFactory.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.client.async;
+
+import org.apache.iotdb.cluster.client.BaseFactory;
+import org.apache.iotdb.cluster.client.ClientCategory;
+import org.apache.iotdb.cluster.client.IClientManager;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.rpc.thrift.RaftService;
+
+import org.apache.thrift.async.TAsyncClientManager;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public abstract class AsyncBaseFactory<K, T extends RaftService.AsyncClient>
+    extends BaseFactory<K, T> {
+
+  private static final Logger logger = LoggerFactory.getLogger(AsyncBaseFactory.class);
+
+  protected AsyncBaseFactory(TProtocolFactory protocolFactory, ClientCategory category) {
+    super(protocolFactory, category);
+    managers =
+        new TAsyncClientManager
+            [ClusterDescriptor.getInstance().getConfig().getSelectorNumOfClientPool()];
+    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);
+      }
+    }
+  }
+
+  protected AsyncBaseFactory(
+      TProtocolFactory protocolFactory, ClientCategory category, IClientManager clientManager) {
+    super(protocolFactory, category, clientManager);
+    managers =
+        new TAsyncClientManager
+            [ClusterDescriptor.getInstance().getConfig().getSelectorNumOfClientPool()];
+    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);
+      }
+    }
+  }
+}
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
deleted file mode 100644
index a96364d..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncClientFactory.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.cluster.client.async;
-
-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;
-
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicInteger;
-
-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 {
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-      managers =
-          new TAsyncClientManager
-              [ClusterDescriptor.getInstance().getConfig().getSelectorNumOfClientPool()];
-      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
deleted file mode 100644
index bf0370f..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncClientPool.java
+++ /dev/null
@@ -1,216 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.cluster.client.async;
-
-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.server.monitor.NodeStatusManager;
-import org.apache.iotdb.cluster.utils.ClusterNode;
-import org.apache.iotdb.db.utils.TestOnly;
-
-import org.apache.thrift.async.TAsyncMethodCall;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Deque;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-public class AsyncClientPool {
-
-  private static final Logger logger = LoggerFactory.getLogger(AsyncClientPool.class);
-  private long waitClientTimeutMS;
-  private int maxConnectionForEachNode;
-  private Map<ClusterNode, Deque<AsyncClient>> clientCaches = new ConcurrentHashMap<>();
-  private Map<ClusterNode, Integer> nodeClientNumMap = new ConcurrentHashMap<>();
-  private AsyncClientFactory asyncClientFactory;
-
-  public AsyncClientPool(AsyncClientFactory asyncClientFactory) {
-    this.asyncClientFactory = asyncClientFactory;
-    this.waitClientTimeutMS = ClusterDescriptor.getInstance().getConfig().getWaitClientTimeoutMS();
-    this.maxConnectionForEachNode =
-        ClusterDescriptor.getInstance().getConfig().getMaxClientPerNodePerMember();
-  }
-
-  /**
-   * See getClient(Node node, boolean activatedOnly)
-   *
-   * @param node
-   * @return
-   * @throws IOException
-   */
-  public AsyncClient getClient(Node node) throws IOException {
-    return getClient(node, true);
-  }
-
-  /**
-   * Get a client of the given node from the cache if one is available, or create a new one.
-   *
-   * <p>IMPORTANT!!! The caller should check whether the return value is null or not!
-   *
-   * @param node the node want to connect
-   * @param activatedOnly if true, only return a client if the node's NodeStatus.isActivated ==
-   *     true, which avoid unnecessary wait for already down nodes, but heartbeat attempts should
-   *     always try to connect so the node can be reactivated ASAP
-   * @return if the node can connect, return the client, otherwise null
-   * @throws IOException if the node can not be connected
-   */
-  public AsyncClient getClient(Node node, boolean activatedOnly) throws IOException {
-    ClusterNode clusterNode = new ClusterNode(node);
-    if (activatedOnly && !NodeStatusManager.getINSTANCE().isActivated(node)) {
-      return null;
-    }
-
-    AsyncClient client;
-    // As clientCaches is ConcurrentHashMap, computeIfAbsent is thread safety.
-    Deque<AsyncClient> clientStack =
-        clientCaches.computeIfAbsent(clusterNode, n -> new ArrayDeque<>());
-    synchronized (clientStack) {
-      if (clientStack.isEmpty()) {
-        int nodeClientNum = nodeClientNumMap.getOrDefault(clusterNode, 0);
-        if (nodeClientNum >= maxConnectionForEachNode) {
-          client = waitForClient(clientStack, clusterNode);
-        } else {
-          client = asyncClientFactory.getAsyncClient(clusterNode, this);
-          nodeClientNumMap.compute(
-              clusterNode,
-              (n, oldValue) -> {
-                if (oldValue == null) return 1;
-                return oldValue + 1;
-              });
-        }
-      } 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 clusterNode
-   * @return
-   * @throws IOException
-   */
-  @SuppressWarnings({"squid:S2273"}) // synchronized outside
-  private AsyncClient waitForClient(Deque<AsyncClient> clientStack, ClusterNode clusterNode)
-      throws IOException {
-    // wait for an available client
-    long waitStart = System.currentTimeMillis();
-    while (clientStack.isEmpty()) {
-      try {
-        clientStack.wait(waitClientTimeutMS);
-        if (clientStack.isEmpty() && System.currentTimeMillis() - waitStart >= waitClientTimeutMS) {
-          logger.warn(
-              "{} Cannot get an available client after {}ms, create a new one.",
-              asyncClientFactory,
-              waitClientTimeutMS);
-          AsyncClient asyncClient = asyncClientFactory.getAsyncClient(clusterNode, this);
-          nodeClientNumMap.computeIfPresent(clusterNode, (n, oldValue) -> oldValue + 1);
-          return asyncClient;
-        }
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        logger.warn("Interrupted when waiting for an available client of {}", clusterNode);
-        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);
-    }
-    // As clientCaches is ConcurrentHashMap, computeIfAbsent is thread safety.
-    Deque<AsyncClient> clientStack =
-        clientCaches.computeIfAbsent(clusterNode, n -> new ArrayDeque<>());
-    synchronized (clientStack) {
-      clientStack.push(client);
-      clientStack.notifyAll();
-    }
-  }
-
-  void onError(Node node) {
-    ClusterNode clusterNode = new ClusterNode(node);
-    // clean all cached clients when network fails
-    Deque<AsyncClient> clientStack =
-        clientCaches.computeIfAbsent(clusterNode, n -> new ArrayDeque<>());
-    synchronized (clientStack) {
-      while (!clientStack.isEmpty()) {
-        AsyncClient client = clientStack.pop();
-        if (client instanceof AsyncDataClient) {
-          ((AsyncDataClient) client).close();
-        } else if (client instanceof AsyncMetaClient) {
-          ((AsyncMetaClient) client).close();
-        }
-      }
-      nodeClientNumMap.put(clusterNode, 0);
-      clientStack.notifyAll();
-      NodeStatusManager.getINSTANCE().deactivate(node);
-    }
-  }
-
-  @SuppressWarnings("squid:S1135")
-  void onComplete(Node node) {
-    NodeStatusManager.getINSTANCE().activate(node);
-  }
-
-  void recreateClient(Node node) {
-    ClusterNode clusterNode = new ClusterNode(node);
-    Deque<AsyncClient> clientStack =
-        clientCaches.computeIfAbsent(clusterNode, n -> new ArrayDeque<>());
-    synchronized (clientStack) {
-      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);
-      }
-      clientStack.notifyAll();
-    }
-  }
-
-  @TestOnly
-  public Map<ClusterNode, Integer> getNodeClientNumMap() {
-    return nodeClientNumMap;
-  }
-}
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
index cf0cd1c..cb9c798 100644
--- 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
@@ -19,13 +19,18 @@
 
 package org.apache.iotdb.cluster.client.async;
 
-import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.client.BaseFactory;
+import org.apache.iotdb.cluster.client.ClientCategory;
+import org.apache.iotdb.cluster.client.IClientManager;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 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.iotdb.cluster.rpc.thrift.TSDataService;
+import org.apache.iotdb.cluster.utils.ClientUtils;
+import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.rpc.TNonblockingSocketWrapper;
 
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
 import org.apache.thrift.async.TAsyncClientManager;
 import org.apache.thrift.async.TAsyncMethodCall;
 import org.apache.thrift.protocol.TProtocolFactory;
@@ -34,21 +39,20 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.util.Date;
 
 /**
  * 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.
+ * should not cache it anywhere else.
  */
-// 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 {
+public class AsyncDataClient extends TSDataService.AsyncClient {
 
   private static final Logger logger = LoggerFactory.getLogger(AsyncDataClient.class);
 
-  Node node;
-  AsyncClientPool pool;
+  private Node node;
+  private ClientCategory category;
+  private IClientManager clientManager;
 
+  @TestOnly
   public AsyncDataClient(
       TProtocolFactory protocolFactory,
       TAsyncClientManager clientManager,
@@ -58,104 +62,157 @@ public class AsyncDataClient extends AsyncClient {
 
   public AsyncDataClient(
       TProtocolFactory protocolFactory,
-      TAsyncClientManager clientManager,
+      TAsyncClientManager tClientManager,
       Node node,
-      AsyncClientPool pool)
+      ClientCategory category)
       throws IOException {
     // the difference of the two clients lies in the port
     super(
         protocolFactory,
-        clientManager,
+        tClientManager,
         TNonblockingSocketWrapper.wrap(
-            node.getInternalIp(), node.getDataPort(), RaftServer.getConnectionTimeoutInMS()));
+            node.getInternalIp(),
+            ClientUtils.getPort(node, category),
+            ClusterConstant.getConnectionTimeoutInMS()));
     this.node = node;
-    this.pool = pool;
+    this.category = category;
+  }
+
+  public AsyncDataClient(
+      TProtocolFactory protocolFactory,
+      TAsyncClientManager tClientManager,
+      Node node,
+      ClientCategory category,
+      IClientManager manager)
+      throws IOException {
+    this(protocolFactory, tClientManager, node, category);
+    this.clientManager = manager;
+  }
+
+  public void close() {
+    ___transport.close();
+    ___currentMethod = null;
+  }
+
+  public boolean isValid() {
+    return ___transport != null;
+  }
+
+  /**
+   * return self if clientPool is not null, the method doesn't need to call by user, it will trigger
+   * once client transport complete.
+   */
+  private void returnSelf() {
+    if (clientManager != null) {
+      clientManager.returnAsyncClient(this, node, category);
+    }
   }
 
   @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);
-    }
+    returnSelf();
   }
 
-  @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 String toString() {
+    return "Async"
+        + category.getName()
+        + "{"
+        + "node="
+        + node
+        + ","
+        + "port="
+        + ClientUtils.getPort(node, category)
+        + '}';
+  }
+
+  public Node getNode() {
+    return node;
+  }
+
+  public boolean isReady() {
+    try {
+      checkReady();
+      return true;
+    } catch (Exception e) {
+      return false;
     }
   }
 
-  public void close() {
-    ___transport.close();
-    ___currentMethod = null;
+  @TestOnly
+  TAsyncMethodCall<Object> getCurrMethod() {
+    return ___currentMethod;
   }
 
-  public static class FactoryAsync extends AsyncClientFactory {
+  public static class AsyncDataClientFactory extends AsyncBaseFactory<Node, AsyncDataClient> {
+
+    public AsyncDataClientFactory(TProtocolFactory protocolFactory, ClientCategory category) {
+      super(protocolFactory, category);
+    }
+
+    public AsyncDataClientFactory(
+        TProtocolFactory protocolFactory, ClientCategory category, IClientManager clientManager) {
+      super(protocolFactory, category, clientManager);
+    }
 
-    public FactoryAsync(org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
-      this.protocolFactory = protocolFactory;
+    @Override
+    public void destroyObject(Node node, PooledObject<AsyncDataClient> pooledObject) {
+      pooledObject.getObject().close();
     }
 
     @Override
-    public RaftService.AsyncClient getAsyncClient(Node node, AsyncClientPool pool)
-        throws IOException {
+    public PooledObject<AsyncDataClient> makeObject(Node node) throws Exception {
       TAsyncClientManager manager = managers[clientCnt.incrementAndGet() % managers.length];
       manager = manager == null ? new TAsyncClientManager() : manager;
-      return new AsyncDataClient(protocolFactory, manager, node, pool);
+      return new DefaultPooledObject<>(
+          new AsyncDataClient(protocolFactory, manager, node, category, clientPoolManager));
     }
-  }
 
-  public static class SingleManagerFactory extends AsyncClientFactory {
+    @Override
+    public boolean validateObject(Node node, PooledObject<AsyncDataClient> pooledObject) {
+      return pooledObject.getObject() != null && pooledObject.getObject().isValid();
+    }
+  }
 
-    private TAsyncClientManager manager;
+  public static class SingleManagerFactory extends BaseFactory<Node, AsyncDataClient> {
 
-    public SingleManagerFactory(org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
-      this.protocolFactory = protocolFactory;
-      if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-        try {
-          manager = new TAsyncClientManager();
-        } catch (IOException e) {
-          logger.error("Cannot init manager of SingleThreadFactoryAsync", e);
-        }
+    public SingleManagerFactory(TProtocolFactory protocolFactory) {
+      super(protocolFactory, ClientCategory.DATA);
+      managers = new TAsyncClientManager[1];
+      try {
+        managers[0] = new TAsyncClientManager();
+      } catch (IOException e) {
+        logger.error("Cannot create data heartbeat client manager for factory", e);
       }
     }
 
-    @Override
-    public RaftService.AsyncClient getAsyncClient(Node node, AsyncClientPool pool)
-        throws IOException {
-      return new AsyncDataClient(protocolFactory, manager, node, pool);
+    public SingleManagerFactory(TProtocolFactory protocolFactory, IClientManager clientManager) {
+      this(protocolFactory);
+      this.clientPoolManager = clientManager;
     }
-  }
 
-  @Override
-  public String toString() {
-    return "DataClient{" + "node=" + node + '}';
-  }
+    @Override
+    public void activateObject(Node node, PooledObject<AsyncDataClient> pooledObject) {}
 
-  public Node getNode() {
-    return node;
-  }
+    @Override
+    public void destroyObject(Node node, PooledObject<AsyncDataClient> pooledObject) {
+      pooledObject.getObject().close();
+    }
 
-  public boolean isReady() {
-    if (___currentMethod != null) {
-      logger.warn(
-          "Client {} is running {} and will timeout at {}",
-          hashCode(),
-          ___currentMethod,
-          new Date(___currentMethod.getTimeoutTimestamp()));
+    @Override
+    public PooledObject<AsyncDataClient> makeObject(Node node) throws Exception {
+      return new DefaultPooledObject<>(
+          new AsyncDataClient(
+              protocolFactory, managers[0], node, ClientCategory.DATA, clientPoolManager));
     }
-    return ___currentMethod == null && !hasError();
-  }
 
-  TAsyncMethodCall<Object> getCurrMethod() {
-    return ___currentMethod;
+    @Override
+    public void passivateObject(Node node, PooledObject<AsyncDataClient> pooledObject) {}
+
+    @Override
+    public boolean validateObject(Node node, PooledObject<AsyncDataClient> pooledObject) {
+      return pooledObject.getObject() != null && pooledObject.getObject().isValid();
+    }
   }
 }
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
deleted file mode 100644
index 146c8b7..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncDataHeartbeatClient.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.cluster.client.async;
-
-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.iotdb.rpc.TNonblockingSocketWrapper;
-
-import org.apache.thrift.async.TAsyncClientManager;
-import org.apache.thrift.protocol.TProtocolFactory;
-
-import java.io.IOException;
-
-/**
- * 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,
-        TNonblockingSocketWrapper.wrap(
-            node.getInternalIp(),
-            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
index c615df0..c2ad2c4 100644
--- 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
@@ -19,33 +19,33 @@
 
 package org.apache.iotdb.cluster.client.async;
 
+import org.apache.iotdb.cluster.client.ClientCategory;
+import org.apache.iotdb.cluster.client.IClientManager;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 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.iotdb.cluster.rpc.thrift.TSMetaService;
+import org.apache.iotdb.cluster.utils.ClientUtils;
+import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.rpc.TNonblockingSocketWrapper;
 
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
 import org.apache.thrift.async.TAsyncClientManager;
 import org.apache.thrift.async.TAsyncMethodCall;
 import org.apache.thrift.protocol.TProtocolFactory;
 import org.apache.thrift.transport.TNonblockingTransport;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.util.Date;
 
 /**
  * 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.
+ * should not cache it anywhere else.
  */
-// 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 {
+public class AsyncMetaClient extends TSMetaService.AsyncClient {
 
-  private static final Logger logger = LoggerFactory.getLogger(AsyncMetaClient.class);
-  Node node;
-  AsyncClientPool pool;
+  private Node node;
+  private ClientCategory category;
+  private IClientManager clientManager;
 
   public AsyncMetaClient(
       TProtocolFactory protocolFactory,
@@ -58,55 +58,58 @@ public class AsyncMetaClient extends AsyncClient {
       TProtocolFactory protocolFactory,
       TAsyncClientManager clientManager,
       Node node,
-      AsyncClientPool pool)
+      ClientCategory category)
       throws IOException {
     // the difference of the two clients lies in the port
     super(
         protocolFactory,
         clientManager,
         TNonblockingSocketWrapper.wrap(
-            node.getInternalIp(), node.getMetaPort(), RaftServer.getConnectionTimeoutInMS()));
+            node.getInternalIp(),
+            ClientUtils.getPort(node, category),
+            ClusterConstant.getConnectionTimeoutInMS()));
     this.node = node;
-    this.pool = pool;
+    this.category = category;
   }
 
-  @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 AsyncMetaClient(
+      TProtocolFactory protocolFactory,
+      TAsyncClientManager clientManager,
+      Node node,
+      ClientCategory category,
+      IClientManager manager)
+      throws IOException {
+    this(protocolFactory, clientManager, node, category);
+    this.clientManager = manager;
   }
 
-  public static class FactoryAsync extends AsyncClientFactory {
-
-    public FactoryAsync(org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
-      this.protocolFactory = protocolFactory;
+  /**
+   * return self if clientManager is not null, the method doesn't need to call by user, it will
+   * trigger once client transport complete.
+   */
+  public void returnSelf() {
+    if (clientManager != null) {
+      clientManager.returnAsyncClient(this, node, category);
     }
+  }
 
-    @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 void onComplete() {
+    super.onComplete();
+    returnSelf();
   }
 
   @Override
   public String toString() {
-    return "MetaClient{" + "node=" + node + '}';
+    return "Async"
+        + category.getName()
+        + "{"
+        + "node="
+        + node
+        + ","
+        + "port="
+        + ClientUtils.getPort(node, category)
+        + '}';
   }
 
   public void close() {
@@ -118,18 +121,58 @@ public class AsyncMetaClient extends AsyncClient {
     return node;
   }
 
+  @TestOnly
   public boolean isReady() {
-    if (___currentMethod != null) {
-      logger.warn(
-          "Client {} is running {} and will timeout at {}",
-          hashCode(),
-          ___currentMethod,
-          new Date(___currentMethod.getTimeoutTimestamp()));
+    try {
+      checkReady();
+      return true;
+    } catch (Exception e) {
+      return false;
     }
-    return ___currentMethod == null;
   }
 
+  public boolean isValid() {
+    return ___transport != null;
+  }
+
+  @TestOnly
   TAsyncMethodCall<Object> getCurrMethod() {
     return ___currentMethod;
   }
+
+  public static class AsyncMetaClientFactory extends AsyncBaseFactory<Node, AsyncMetaClient> {
+
+    public AsyncMetaClientFactory(TProtocolFactory protocolFactory, ClientCategory category) {
+      super(protocolFactory, category);
+    }
+
+    public AsyncMetaClientFactory(
+        TProtocolFactory protocolFactory, ClientCategory category, IClientManager clientManager) {
+      super(protocolFactory, category, clientManager);
+    }
+
+    @Override
+    public void activateObject(Node node, PooledObject<AsyncMetaClient> pooledObject) {}
+
+    @Override
+    public void destroyObject(Node node, PooledObject<AsyncMetaClient> pooledObject) {
+      pooledObject.getObject().close();
+    }
+
+    @Override
+    public PooledObject<AsyncMetaClient> makeObject(Node node) throws Exception {
+      TAsyncClientManager manager = managers[clientCnt.incrementAndGet() % managers.length];
+      manager = manager == null ? new TAsyncClientManager() : manager;
+      return new DefaultPooledObject<>(
+          new AsyncMetaClient(protocolFactory, manager, node, category, clientPoolManager));
+    }
+
+    @Override
+    public void passivateObject(Node node, PooledObject<AsyncMetaClient> pooledObject) {}
+
+    @Override
+    public boolean validateObject(Node node, PooledObject<AsyncMetaClient> pooledObject) {
+      return pooledObject != null && pooledObject.getObject().isValid();
+    }
+  }
 }
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
deleted file mode 100644
index babeae4..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/async/AsyncMetaHeartbeatClient.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.cluster.client.async;
-
-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.iotdb.rpc.TNonblockingSocketWrapper;
-
-import org.apache.thrift.async.TAsyncClientManager;
-import org.apache.thrift.protocol.TProtocolFactory;
-
-import java.io.IOException;
-
-/**
- * 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,
-        TNonblockingSocketWrapper.wrap(
-            node.getInternalIp(),
-            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
index c55ff46..7f4f3aa 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.cluster.client.sync;
 
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.async.AsyncMetaClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.log.Snapshot;
 import org.apache.iotdb.cluster.log.snapshot.SnapshotFactory;
 import org.apache.iotdb.cluster.rpc.thrift.AddNodeResponse;
@@ -39,7 +40,6 @@ 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.GetChildNodeNextLevelHandler;
 import org.apache.iotdb.cluster.server.handlers.caller.GetChildNodeNextLevelPathHandler;
@@ -100,7 +100,7 @@ public class SyncClientAdaptor {
     asyncMetaClient.removeNode(nodeToRemove, handler);
     synchronized (responseRef) {
       if (responseRef.get() == null) {
-        responseRef.wait(RaftServer.getConnectionTimeoutInMS());
+        responseRef.wait(ClusterConstant.getConnectionTimeoutInMS());
       }
     }
     return responseRef.get();
@@ -116,7 +116,7 @@ public class SyncClientAdaptor {
       client.matchTerm(prevLogIndex, prevLogTerm, header, matchTermHandler);
       synchronized (resultRef) {
         if (resultRef.get() == null) {
-          resultRef.wait(RaftServer.getConnectionTimeoutInMS());
+          resultRef.wait(ClusterConstant.getConnectionTimeoutInMS());
         }
       }
       return resultRef.get();
@@ -135,7 +135,7 @@ public class SyncClientAdaptor {
     client.querySingleSeriesByTimestamp(request, handler);
     synchronized (result) {
       if (result.get() == null && handler.getException() == null) {
-        result.wait(RaftServer.getReadOperationTimeoutMS());
+        result.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     return result.get();
@@ -159,7 +159,7 @@ public class SyncClientAdaptor {
     client.querySingleSeries(request, handler);
     synchronized (result) {
       if (result.get() == null && handler.getException() == null) {
-        result.wait(RaftServer.getReadOperationTimeoutMS());
+        result.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     return result.get();
@@ -176,7 +176,7 @@ public class SyncClientAdaptor {
     client.getNodeList(header, schemaPattern, level, handler);
     synchronized (response) {
       if (response.get() == null) {
-        response.wait(RaftServer.getReadOperationTimeoutMS());
+        response.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     return response.get();
@@ -193,7 +193,7 @@ public class SyncClientAdaptor {
     client.getChildNodeInNextLevel(header, path, handler);
     synchronized (response) {
       if (response.get() == null) {
-        response.wait(RaftServer.getReadOperationTimeoutMS());
+        response.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     return response.get();
@@ -209,7 +209,7 @@ public class SyncClientAdaptor {
     client.getChildNodePathInNextLevel(header, path, handler);
     synchronized (response) {
       if (response.get() == null) {
-        response.wait(RaftServer.getReadOperationTimeoutMS());
+        response.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     return response.get();
@@ -230,7 +230,7 @@ public class SyncClientAdaptor {
         header, ByteBuffer.wrap(byteArrayOutputStream.toByteArray()), handler);
     synchronized (response) {
       if (response.get() == null) {
-        response.wait(RaftServer.getReadOperationTimeoutMS());
+        response.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     return response.get();
@@ -244,7 +244,7 @@ public class SyncClientAdaptor {
     client.queryNodeStatus(handler);
     synchronized (resultRef) {
       if (resultRef.get() == null) {
-        resultRef.wait(RaftServer.getReadOperationTimeoutMS());
+        resultRef.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     if (handler.getException() != null) {
@@ -261,7 +261,7 @@ public class SyncClientAdaptor {
     client.checkStatus(startUpStatus, handler);
     synchronized (resultRef) {
       if (resultRef.get() == null) {
-        resultRef.wait(RaftServer.getReadOperationTimeoutMS());
+        resultRef.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     if (handler.getException() != null) {
@@ -298,7 +298,7 @@ public class SyncClientAdaptor {
             client.getNode(), pullSchemaRequest.getPrefixPaths(), measurementSchemas));
     synchronized (measurementSchemas) {
       if (measurementSchemas.get() == null) {
-        measurementSchemas.wait(RaftServer.getReadOperationTimeoutMS());
+        measurementSchemas.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     return measurementSchemas.get();
@@ -315,7 +315,7 @@ public class SyncClientAdaptor {
 
     synchronized (timeseriesSchemas) {
       if (timeseriesSchemas.get() == null) {
-        timeseriesSchemas.wait(RaftServer.getReadOperationTimeoutMS());
+        timeseriesSchemas.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     return timeseriesSchemas.get();
@@ -330,7 +330,7 @@ public class SyncClientAdaptor {
     client.getAggrResult(request, handler);
     synchronized (resultReference) {
       if (resultReference.get() == null) {
-        resultReference.wait(RaftServer.getReadOperationTimeoutMS());
+        resultReference.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     if (handler.getException() != null) {
@@ -346,7 +346,7 @@ public class SyncClientAdaptor {
     GenericHandler<List<String>> handler = new GenericHandler<>(client.getNode(), remoteResult);
 
     client.getUnregisteredTimeseries(header, seriesPaths, handler);
-    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+    return handler.getResult(ClusterConstant.getReadOperationTimeoutMS());
   }
 
   public static GetAllPathsResult getAllPaths(
@@ -357,7 +357,7 @@ public class SyncClientAdaptor {
         new GenericHandler<>(client.getNode(), remoteResult);
 
     client.getAllPaths(header, pathsToQuery, withAlias, handler);
-    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+    return handler.getResult(ClusterConstant.getReadOperationTimeoutMS());
   }
 
   public static Integer getPathCount(
@@ -367,7 +367,7 @@ public class SyncClientAdaptor {
     GenericHandler<Integer> handler = new GenericHandler<>(client.getNode(), remoteResult);
 
     client.getPathCount(header, pathsToQuery, level, handler);
-    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+    return handler.getResult(ClusterConstant.getReadOperationTimeoutMS());
   }
 
   public static Integer getDeviceCount(
@@ -377,7 +377,7 @@ public class SyncClientAdaptor {
     GenericHandler<Integer> handler = new GenericHandler<>(client.getNode(), remoteResult);
 
     client.getDeviceCount(header, pathsToQuery, handler);
-    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+    return handler.getResult(ClusterConstant.getReadOperationTimeoutMS());
   }
 
   public static Set<String> getAllDevices(
@@ -387,7 +387,7 @@ public class SyncClientAdaptor {
     GenericHandler<Set<String>> handler = new GenericHandler<>(client.getNode(), remoteResult);
 
     client.getAllDevices(header, pathsToQuery, handler);
-    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+    return handler.getResult(ClusterConstant.getReadOperationTimeoutMS());
   }
 
   public static ByteBuffer getDevices(AsyncDataClient client, RaftNode header, ShowDevicesPlan plan)
@@ -403,7 +403,7 @@ public class SyncClientAdaptor {
     client.getDevices(header, ByteBuffer.wrap(byteArrayOutputStream.toByteArray()), handler);
     synchronized (handler) {
       if (response.get() == null) {
-        response.wait(RaftServer.getReadOperationTimeoutMS());
+        response.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     return response.get();
@@ -415,7 +415,7 @@ public class SyncClientAdaptor {
     GenericHandler<Long> handler = new GenericHandler<>(client.getNode(), result);
 
     client.getGroupByExecutor(request, handler);
-    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+    return handler.getResult(ClusterConstant.getReadOperationTimeoutMS());
   }
 
   public static ByteBuffer previousFill(AsyncDataClient client, PreviousFillRequest request)
@@ -424,7 +424,7 @@ public class SyncClientAdaptor {
     GenericHandler<ByteBuffer> nodeHandler = new GenericHandler<>(client.getNode(), resultRef);
 
     client.previousFill(request, nodeHandler);
-    return nodeHandler.getResult(RaftServer.getReadOperationTimeoutMS());
+    return nodeHandler.getResult(ClusterConstant.getReadOperationTimeoutMS());
   }
 
   public static TSStatus executeNonQuery(
@@ -440,7 +440,7 @@ public class SyncClientAdaptor {
     client.executeNonQueryPlan(req, new ForwardPlanHandler(status, plan, receiver));
     synchronized (status) {
       if (status.get() == null) {
-        status.wait(RaftServer.getWriteOperationTimeoutMS());
+        status.wait(ClusterConstant.getWriteOperationTimeoutMS());
       }
     }
     return status.get();
@@ -453,7 +453,7 @@ public class SyncClientAdaptor {
     GenericHandler<ByteBuffer> handler = new GenericHandler<>(client.getNode(), result);
 
     client.readFile(remotePath, offset, fetchSize, handler);
-    return handler.getResult(RaftServer.getWriteOperationTimeoutMS());
+    return handler.getResult(ClusterConstant.getWriteOperationTimeoutMS());
   }
 
   public static List<ByteBuffer> getGroupByResult(
@@ -463,7 +463,7 @@ public class SyncClientAdaptor {
     GenericHandler<List<ByteBuffer>> handler = new GenericHandler<>(client.getNode(), fetchResult);
 
     client.getGroupByResult(header, executorId, curStartTime, curEndTime, handler);
-    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+    return handler.getResult(ClusterConstant.getReadOperationTimeoutMS());
   }
 
   public static ByteBuffer peekNextNotNullValue(
@@ -473,7 +473,7 @@ public class SyncClientAdaptor {
     GenericHandler<ByteBuffer> handler = new GenericHandler<>(client.getNode(), fetchResult);
 
     client.peekNextNotNullValue(header, executorId, curStartTime, curEndTime, handler);
-    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+    return handler.getResult(ClusterConstant.getReadOperationTimeoutMS());
   }
 
   public static <T extends Snapshot> Map<Integer, T> pullSnapshot(
@@ -488,7 +488,7 @@ public class SyncClientAdaptor {
         request, new PullSnapshotHandler<>(snapshotRef, client.getNode(), slots, factory));
     synchronized (snapshotRef) {
       if (snapshotRef.get() == null) {
-        snapshotRef.wait(RaftServer.getReadOperationTimeoutMS());
+        snapshotRef.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     return snapshotRef.get();
@@ -514,7 +514,7 @@ public class SyncClientAdaptor {
             client.getNode());
 
     client.last(request, handler);
-    return handler.getResult(RaftServer.getReadOperationTimeoutMS());
+    return handler.getResult(ClusterConstant.getReadOperationTimeoutMS());
   }
 
   public static boolean onSnapshotApplied(
@@ -524,6 +524,6 @@ public class SyncClientAdaptor {
     GenericHandler<Boolean> handler = new GenericHandler<>(client.getNode(), result);
 
     client.onSnapshotApplied(header, slots, handler);
-    return handler.getResult(RaftServer.getWriteOperationTimeoutMS());
+    return handler.getResult(ClusterConstant.getWriteOperationTimeoutMS());
   }
 }
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
deleted file mode 100644
index 2c279c0..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientPool.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.cluster.client.sync;
-
-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.server.monitor.NodeStatusManager;
-import org.apache.iotdb.cluster.utils.ClusterNode;
-import org.apache.iotdb.db.utils.TestOnly;
-
-import org.apache.thrift.transport.TTransportException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayDeque;
-import java.util.Deque;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-public class SyncClientPool {
-
-  private static final Logger logger = LoggerFactory.getLogger(SyncClientPool.class);
-  private long waitClientTimeoutMS;
-  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.waitClientTimeoutMS = ClusterDescriptor.getInstance().getConfig().getWaitClientTimeoutMS();
-    this.maxConnectionForEachNode =
-        ClusterDescriptor.getInstance().getConfig().getMaxClientPerNodePerMember();
-  }
-
-  /**
-   * See getClient(Node node, boolean activatedOnly)
-   *
-   * @param node the node want to connect
-   * @return if the node can connect, return the client, otherwise null
-   */
-  public Client getClient(Node node) {
-    return getClient(node, true);
-  }
-
-  /**
-   * Get a client of the given node from the cache if one is available, or create a new one.
-   *
-   * <p>IMPORTANT!!! The caller should check whether the return value is null or not!
-   *
-   * @param node the node want to connect
-   * @param activatedOnly if true, only return a client if the node's NodeStatus.isActivated ==
-   *     true, which avoid unnecessary wait for already down nodes, but heartbeat attempts should
-   *     always try to connect so the node can be reactivated ASAP
-   * @return if the node can connect, return the client, otherwise null
-   */
-  public Client getClient(Node node, boolean activatedOnly) {
-    ClusterNode clusterNode = new ClusterNode(node);
-    if (activatedOnly && !NodeStatusManager.getINSTANCE().isActivated(node)) {
-      return null;
-    }
-
-    // As clientCaches is ConcurrentHashMap, computeIfAbsent is thread safety.
-    Deque<Client> clientStack = clientCaches.computeIfAbsent(clusterNode, n -> new ArrayDeque<>());
-    synchronized (clientStack) {
-      if (clientStack.isEmpty()) {
-        int nodeClientNum = nodeClientNumMap.getOrDefault(clusterNode, 0);
-        if (nodeClientNum >= maxConnectionForEachNode) {
-          return waitForClient(clientStack, clusterNode);
-        } else {
-          Client client = null;
-          try {
-            client = syncClientFactory.getSyncClient(clusterNode, this);
-          } catch (TTransportException e) {
-            logger.error("Cannot open transport for client {}", node, e);
-            return null;
-          }
-          nodeClientNumMap.compute(
-              clusterNode,
-              (n, oldValue) -> {
-                if (oldValue == null) return 1;
-                return oldValue + 1;
-              });
-          return client;
-        }
-      } else {
-        return clientStack.pop();
-      }
-    }
-  }
-
-  @SuppressWarnings("squid:S2273") // synchronized outside
-  private Client waitForClient(Deque<Client> clientStack, ClusterNode clusterNode) {
-    // wait for an available client
-    long waitStart = System.currentTimeMillis();
-    while (clientStack.isEmpty()) {
-      try {
-        clientStack.wait(waitClientTimeoutMS);
-        if (clientStack.isEmpty()
-            && System.currentTimeMillis() - waitStart >= waitClientTimeoutMS) {
-          logger.warn(
-              "Cannot get an available client after {}ms, create a new one", waitClientTimeoutMS);
-          Client client = syncClientFactory.getSyncClient(clusterNode, this);
-          nodeClientNumMap.computeIfPresent(clusterNode, (n, oldValue) -> oldValue + 1);
-          return client;
-        }
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        logger.warn("Interrupted when waiting for an available client of {}", clusterNode);
-        return null;
-      } catch (TTransportException e) {
-        logger.error("Cannot open transport for client {}", clusterNode, e);
-        return null;
-      }
-    }
-    return clientStack.pop();
-  }
-
-  /**
-   * Return a client of a node to the pool. Closed client should not be returned.
-   *
-   * @param node connection node
-   * @param client push client to pool
-   */
-  public 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 (clientStack) {
-      if (client.getInputProtocol() != null && client.getInputProtocol().getTransport().isOpen()) {
-        clientStack.push(client);
-        NodeStatusManager.getINSTANCE().activate(node);
-      } else {
-        try {
-          clientStack.push(syncClientFactory.getSyncClient(node, this));
-          NodeStatusManager.getINSTANCE().activate(node);
-        } catch (TTransportException e) {
-          logger.error("Cannot open transport for client {}", node, e);
-          nodeClientNumMap.computeIfPresent(clusterNode, (n, oldValue) -> oldValue - 1);
-          NodeStatusManager.getINSTANCE().deactivate(node);
-        }
-      }
-      clientStack.notifyAll();
-    }
-  }
-
-  @TestOnly
-  public Map<ClusterNode, Integer> getNodeClientNumMap() {
-    return nodeClientNumMap;
-  }
-}
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
index cf92d14..2f52d4f 100644
--- 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
@@ -19,39 +19,47 @@
 
 package org.apache.iotdb.cluster.client.sync;
 
+import org.apache.iotdb.cluster.client.BaseFactory;
+import org.apache.iotdb.cluster.client.ClientCategory;
+import org.apache.iotdb.cluster.client.IClientManager;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 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.cluster.rpc.thrift.TSDataService;
+import org.apache.iotdb.cluster.utils.ClientUtils;
 import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.rpc.RpcTransportFactory;
 import org.apache.iotdb.rpc.TConfigurationConst;
 import org.apache.iotdb.rpc.TimeoutChangeableTransport;
 
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
 import org.apache.thrift.protocol.TProtocol;
 import org.apache.thrift.protocol.TProtocolFactory;
 import org.apache.thrift.transport.TSocket;
 import org.apache.thrift.transport.TTransportException;
 
-import java.io.Closeable;
 import java.net.SocketException;
 
 /**
  * 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.
+ * should not cache it anywhere else.
  */
-// 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 implements Closeable {
+// TODO: Refine the interfaces of TSDataService. TSDataService interfaces doesn't need extends
+// RaftService interfaces.
+public class SyncDataClient extends TSDataService.Client {
 
-  Node node;
-  SyncClientPool pool;
+  private Node node;
+  private ClientCategory category;
+  private IClientManager clientManager;
 
+  @TestOnly
   public SyncDataClient(TProtocol prot) {
     super(prot);
   }
 
-  public SyncDataClient(TProtocolFactory protocolFactory, Node node, SyncClientPool pool)
+  public SyncDataClient(TProtocolFactory protocolFactory, Node node, ClientCategory category)
       throws TTransportException {
+
     // the difference of the two clients lies in the port
     super(
         protocolFactory.getProtocol(
@@ -59,60 +67,88 @@ public class SyncDataClient extends Client implements Closeable {
                 new TSocket(
                     TConfigurationConst.defaultTConfiguration,
                     node.getInternalIp(),
-                    node.getDataPort(),
-                    RaftServer.getConnectionTimeoutInMS()))));
+                    ClientUtils.getPort(node, category),
+                    ClusterConstant.getConnectionTimeoutInMS()))));
     this.node = node;
-    this.pool = pool;
+    this.category = category;
     getInputProtocol().getTransport().open();
   }
 
+  public SyncDataClient(
+      TProtocolFactory protocolFactory, Node node, ClientCategory category, IClientManager manager)
+      throws TTransportException {
+    this(protocolFactory, node, category);
+    this.clientManager = manager;
+  }
+
+  public void returnSelf() {
+    if (clientManager != null) {
+      clientManager.returnSyncClient(this, node, category);
+    }
+  }
+
   public void setTimeout(int timeout) {
     // the same transport is used in both input and output
     ((TimeoutChangeableTransport) (getInputProtocol().getTransport())).setTimeout(timeout);
   }
 
+  public void close() {
+    getInputProtocol().getTransport().close();
+  }
+
   @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();
-      }
-    }
+  @Override
+  public String toString() {
+    return "Sync"
+        + category.getName()
+        + "{"
+        + "node="
+        + node
+        + ","
+        + "port="
+        + ClientUtils.getPort(node, category)
+        + '}';
   }
 
-  /** put the client to pool, instead of close client. */
-  @Override
-  public void close() {
-    putBack();
+  public Node getNode() {
+    return node;
   }
 
-  public static class FactorySync implements SyncClientFactory {
+  public static class SyncDataClientFactory extends BaseFactory<Node, SyncDataClient> {
 
-    private TProtocolFactory protocolFactory;
+    public SyncDataClientFactory(TProtocolFactory protocolFactory, ClientCategory category) {
+      super(protocolFactory, category);
+    }
 
-    public FactorySync(TProtocolFactory protocolFactory) {
-      this.protocolFactory = protocolFactory;
+    public SyncDataClientFactory(
+        TProtocolFactory protocolFactory, ClientCategory category, IClientManager clientManager) {
+      super(protocolFactory, category, clientManager);
     }
 
     @Override
-    public SyncDataClient getSyncClient(Node node, SyncClientPool pool) throws TTransportException {
-      return new SyncDataClient(protocolFactory, node, pool);
+    public void activateObject(Node node, PooledObject<SyncDataClient> pooledObject) {
+      pooledObject.getObject().setTimeout(ClusterConstant.getConnectionTimeoutInMS());
     }
-  }
 
-  @Override
-  public String toString() {
-    return "DataClient{" + "node=" + node + '}';
-  }
+    @Override
+    public void destroyObject(Node node, PooledObject<SyncDataClient> pooledObject) {
+      pooledObject.getObject().close();
+    }
 
-  public Node getNode() {
-    return node;
+    @Override
+    public PooledObject<SyncDataClient> makeObject(Node node) throws Exception {
+      return new DefaultPooledObject<>(
+          new SyncDataClient(protocolFactory, node, category, clientPoolManager));
+    }
+
+    @Override
+    public boolean validateObject(Node node, PooledObject<SyncDataClient> pooledObject) {
+      return pooledObject.getObject() != null
+          && pooledObject.getObject().getInputProtocol().getTransport().isOpen();
+    }
   }
 }
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
deleted file mode 100644
index 38820fc..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncDataHeartbeatClient.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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.iotdb.rpc.TConfigurationConst;
-
-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(
-                    TConfigurationConst.defaultTConfiguration,
-                    node.getInternalIp(),
-                    node.getDataPort() + ClusterUtils.DATA_HEARTBEAT_PORT_OFFSET,
-                    RaftServer.getHeartbeatClientConnTimeoutMs()))));
-    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
index d29e438..1e93a17 100644
--- 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
@@ -19,35 +19,37 @@
 
 package org.apache.iotdb.cluster.client.sync;
 
+import org.apache.iotdb.cluster.client.BaseFactory;
+import org.apache.iotdb.cluster.client.ClientCategory;
+import org.apache.iotdb.cluster.client.IClientManager;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 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.cluster.rpc.thrift.TSMetaService;
+import org.apache.iotdb.cluster.utils.ClientUtils;
+import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.rpc.RpcTransportFactory;
 import org.apache.iotdb.rpc.TConfigurationConst;
+import org.apache.iotdb.rpc.TimeoutChangeableTransport;
 
-import org.apache.thrift.protocol.TProtocol;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
 import org.apache.thrift.protocol.TProtocolFactory;
 import org.apache.thrift.transport.TSocket;
 import org.apache.thrift.transport.TTransportException;
 
-import java.io.Closeable;
+import java.net.SocketException;
 
 /**
  * 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.
+ * should not cache it anywhere else.
  */
-// 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 implements Closeable {
+public class SyncMetaClient extends TSMetaService.Client {
 
-  Node node;
-  SyncClientPool pool;
+  private Node node;
+  private ClientCategory category;
+  private IClientManager clientManager;
 
-  SyncMetaClient(TProtocol prot) {
-    super(prot);
-  }
-
-  public SyncMetaClient(TProtocolFactory protocolFactory, Node node, SyncClientPool pool)
+  public SyncMetaClient(TProtocolFactory protocolFactory, Node node, ClientCategory category)
       throws TTransportException {
     super(
         protocolFactory.getProtocol(
@@ -55,39 +57,38 @@ public class SyncMetaClient extends Client implements Closeable {
                 new TSocket(
                     TConfigurationConst.defaultTConfiguration,
                     node.getInternalIp(),
-                    node.getMetaPort(),
-                    RaftServer.getConnectionTimeoutInMS()))));
+                    ClientUtils.getPort(node, category),
+                    ClusterConstant.getConnectionTimeoutInMS()))));
     this.node = node;
-    this.pool = pool;
+    this.category = category;
     getInputProtocol().getTransport().open();
   }
 
-  public void putBack() {
-    if (pool != null) {
-      pool.putClient(node, this);
-    } else {
-      getInputProtocol().getTransport().close();
-    }
+  public SyncMetaClient(
+      TProtocolFactory protocolFactory, Node node, ClientCategory category, IClientManager manager)
+      throws TTransportException {
+    this(protocolFactory, node, category);
+    this.clientManager = manager;
   }
 
-  /** put the client to pool, instead of close client. */
-  @Override
-  public void close() {
-    putBack();
+  public void returnSelf() {
+    if (clientManager != null) {
+      clientManager.returnSyncClient(this, node, category);
+    }
   }
 
-  public static class FactorySync implements SyncClientFactory {
-
-    private TProtocolFactory protocolFactory;
+  public void setTimeout(int timeout) {
+    // the same transport is used in both input and output
+    ((TimeoutChangeableTransport) (getInputProtocol().getTransport())).setTimeout(timeout);
+  }
 
-    public FactorySync(TProtocolFactory protocolFactory) {
-      this.protocolFactory = protocolFactory;
-    }
+  @TestOnly
+  public int getTimeout() throws SocketException {
+    return ((TimeoutChangeableTransport) getInputProtocol().getTransport()).getTimeOut();
+  }
 
-    @Override
-    public SyncMetaClient getSyncClient(Node node, SyncClientPool pool) throws TTransportException {
-      return new SyncMetaClient(protocolFactory, node, pool);
-    }
+  public void close() {
+    getInputProtocol().getTransport().close();
   }
 
   public Node getNode() {
@@ -96,6 +97,48 @@ public class SyncMetaClient extends Client implements Closeable {
 
   @Override
   public String toString() {
-    return "SyncMetaClient{" + " node=" + node + ", pool=" + pool + "}";
+    return "Sync"
+        + category.getName()
+        + "{"
+        + "node="
+        + node
+        + ","
+        + "port="
+        + ClientUtils.getPort(node, category)
+        + '}';
+  }
+
+  public static class SyncMetaClientFactory extends BaseFactory<Node, SyncMetaClient> {
+
+    public SyncMetaClientFactory(TProtocolFactory protocolFactory, ClientCategory category) {
+      super(protocolFactory, category);
+    }
+
+    public SyncMetaClientFactory(
+        TProtocolFactory protocolFactory, ClientCategory category, IClientManager clientManager) {
+      super(protocolFactory, category, clientManager);
+    }
+
+    @Override
+    public void activateObject(Node node, PooledObject<SyncMetaClient> pooledObject) {
+      pooledObject.getObject().setTimeout(ClusterConstant.getConnectionTimeoutInMS());
+    }
+
+    @Override
+    public void destroyObject(Node node, PooledObject<SyncMetaClient> pooledObject) {
+      pooledObject.getObject().close();
+    }
+
+    @Override
+    public PooledObject<SyncMetaClient> makeObject(Node node) throws Exception {
+      return new DefaultPooledObject<>(
+          new SyncMetaClient(protocolFactory, node, category, clientPoolManager));
+    }
+
+    @Override
+    public boolean validateObject(Node node, PooledObject<SyncMetaClient> pooledObject) {
+      return pooledObject.getObject() != null
+          && pooledObject.getObject().getInputProtocol().getTransport().isOpen();
+    }
   }
 }
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
deleted file mode 100644
index 5972259..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncMetaHeartbeatClient.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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.iotdb.rpc.TConfigurationConst;
-
-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 {
-    super(
-        protocolFactory.getProtocol(
-            RpcTransportFactory.INSTANCE.getTransport(
-                new TSocket(
-                    TConfigurationConst.defaultTConfiguration,
-                    node.getInternalIp(),
-                    node.getMetaPort() + ClusterUtils.META_HEARTBEAT_PORT_OFFSET,
-                    RaftServer.getHeartbeatClientConnTimeoutMs()))));
-    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
index afa202d..84a97a3 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
@@ -44,7 +44,6 @@ public class ClusterConfig {
   private List<String> seedNodeUrls;
 
   @ClusterConsistent private boolean isRpcThriftCompressionEnabled = false;
-  private int maxConcurrentClientNum = 10000;
 
   @ClusterConsistent private int replicationNum = 1;
 
@@ -232,14 +231,6 @@ public class ClusterConfig {
     isRpcThriftCompressionEnabled = rpcThriftCompressionEnabled;
   }
 
-  public int getMaxConcurrentClientNum() {
-    return maxConcurrentClientNum;
-  }
-
-  void setMaxConcurrentClientNum(int maxConcurrentClientNum) {
-    this.maxConcurrentClientNum = maxConcurrentClientNum;
-  }
-
   public List<String> getSeedNodeUrls() {
     return seedNodeUrls;
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java
index 8f049c5..cb892ae 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java
@@ -31,6 +31,14 @@ public class ClusterConstant {
   private static long electionMaxWaitMs =
       Math.max(ClusterDescriptor.getInstance().getConfig().getHeartbeatIntervalMs(), 50L);
 
+  // Heartbeat client connection timeout should not be larger than heartbeat interval, otherwise
+  // the thread pool of sending heartbeats or requesting votes may be used up by waiting for
+  // establishing connection with some slow or dead nodes.
+  private static final int heartbeatClientConnTimeoutMs =
+      Math.min(
+          (int) ClusterConstant.getHeartbeatIntervalMs(),
+          ClusterConstant.getConnectionTimeoutInMS());
+
   public static final int SLOT_NUM = 10000;
   public static final int HASH_SALT = 2333;
   public static final int CHECK_ALIVE_TIME_OUT_MS = 1000;
@@ -41,6 +49,18 @@ public class ClusterConstant {
 
   public static final int THREAD_POLL_WAIT_TERMINATION_TIME_S = 10;
 
+  /**
+   * every "REPORT_INTERVAL_SEC" seconds, a reporter thread will print the status of all raft
+   * members in this node.
+   */
+  public static final int REPORT_INTERVAL_SEC = 10;
+
+  /**
+   * during snapshot, hardlinks of data files are created to for downloading. hardlinks will be
+   * checked every hour by default to see if they have expired, and will be cleaned if so.
+   */
+  public static final long CLEAN_HARDLINK_INTERVAL_SEC = 3600L;
+
   public static final Node EMPTY_NODE = new Node();
 
   private ClusterConstant() {
@@ -55,4 +75,70 @@ public class ClusterConstant {
   public static void setElectionMaxWaitMs(long electionMaxWaitMs) {
     ClusterConstant.electionMaxWaitMs = electionMaxWaitMs;
   }
+
+  private static int connectionTimeoutInMS =
+      ClusterDescriptor.getInstance().getConfig().getConnectionTimeoutInMS();
+  private static int readOperationTimeoutMS =
+      ClusterDescriptor.getInstance().getConfig().getReadOperationTimeoutMS();
+  private static int writeOperationTimeoutMS =
+      ClusterDescriptor.getInstance().getConfig().getWriteOperationTimeoutMS();
+  private static int syncLeaderMaxWaitMs = 20 * 1000;
+  private static long heartbeatIntervalMs =
+      ClusterDescriptor.getInstance().getConfig().getHeartbeatIntervalMs();
+  private static long electionTimeoutMs =
+      ClusterDescriptor.getInstance().getConfig().getElectionTimeoutMs();
+
+  public static int getConnectionTimeoutInMS() {
+    return connectionTimeoutInMS;
+  }
+
+  public static void setConnectionTimeoutInMS(int connectionTimeoutInMS) {
+    ClusterConstant.connectionTimeoutInMS = connectionTimeoutInMS;
+  }
+
+  public static int getReadOperationTimeoutMS() {
+    return readOperationTimeoutMS;
+  }
+
+  public static int getWriteOperationTimeoutMS() {
+    return writeOperationTimeoutMS;
+  }
+
+  public static int getSyncLeaderMaxWaitMs() {
+    return syncLeaderMaxWaitMs;
+  }
+
+  public static void setSyncLeaderMaxWaitMs(int syncLeaderMaxWaitMs) {
+    ClusterConstant.syncLeaderMaxWaitMs = syncLeaderMaxWaitMs;
+  }
+
+  public static long getHeartbeatIntervalMs() {
+    return heartbeatIntervalMs;
+  }
+
+  public static void setHeartbeatIntervalMs(long heartBeatIntervalMs) {
+    ClusterConstant.heartbeatIntervalMs = heartBeatIntervalMs;
+  }
+
+  public static long getElectionTimeoutMs() {
+    return electionTimeoutMs;
+  }
+
+  public static void setElectionTimeoutMs(long electionTimeoutMs) {
+    ClusterConstant.electionTimeoutMs = electionTimeoutMs;
+  }
+
+  public static int getHeartbeatClientConnTimeoutMs() {
+    return heartbeatClientConnTimeoutMs;
+  }
+
+  @TestOnly
+  public static void setReadOperationTimeoutMS(int readOperationTimeoutMS) {
+    ClusterConstant.readOperationTimeoutMS = readOperationTimeoutMS;
+  }
+
+  @TestOnly
+  public static void setWriteOperationTimeoutMS(int writeOperationTimeoutMS) {
+    ClusterConstant.writeOperationTimeoutMS = writeOperationTimeoutMS;
+  }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java
index f70f08f..0ebac8c 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java
@@ -151,11 +151,6 @@ public class ClusterDescriptor {
             properties.getProperty(
                 "cluster_info_public_port", Integer.toString(config.getClusterInfoRpcPort()))));
 
-    config.setMaxConcurrentClientNum(
-        Integer.parseInt(
-            properties.getProperty(
-                "max_concurrent_client_num", String.valueOf(config.getMaxConcurrentClientNum()))));
-
     config.setMultiRaftFactor(
         Integer.parseInt(
             properties.getProperty(
@@ -383,18 +378,13 @@ public class ClusterDescriptor {
 
   /**
    * 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
+   * 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(
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java b/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
index 84e8caa..9110ce5 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.cluster.coordinator;
 
+import org.apache.iotdb.cluster.ClusterIoTDB;
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
 import org.apache.iotdb.cluster.config.ClusterConstant;
@@ -33,8 +34,6 @@ import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.query.ClusterPlanRouter;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
-import org.apache.iotdb.cluster.rpc.thrift.RaftService;
-import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.cluster.server.monitor.Timer;
 import org.apache.iotdb.cluster.utils.PartitionUtils;
@@ -58,12 +57,12 @@ import org.apache.iotdb.db.qp.physical.sys.CreateMultiTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
 import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.service.rpc.thrift.EndPoint;
 import org.apache.iotdb.service.rpc.thrift.TSStatus;
 
-import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -92,16 +91,19 @@ public class Coordinator {
       "The following errors occurred when executing "
           + "the query, please retry or contact the DBA: ";
 
+  @TestOnly
   public Coordinator(MetaGroupMember metaGroupMember) {
-    this.metaGroupMember = metaGroupMember;
-    this.name = metaGroupMember.getName();
-    this.thisNode = metaGroupMember.getThisNode();
+    linkMetaGroupMember(metaGroupMember);
   }
 
   public Coordinator() {}
 
-  public void setMetaGroupMember(MetaGroupMember metaGroupMember) {
+  public void linkMetaGroupMember(MetaGroupMember metaGroupMember) {
     this.metaGroupMember = metaGroupMember;
+    if (metaGroupMember.getCoordinator() != null && metaGroupMember.getCoordinator() != this) {
+      logger.warn("MetadataGroupMember linked inconsistent Coordinator, will correct it.");
+      metaGroupMember.setCoordinator(this);
+    }
     this.name = metaGroupMember.getName();
     this.thisNode = metaGroupMember.getThisNode();
   }
@@ -785,48 +787,21 @@ public class Coordinator {
    */
   private TSStatus forwardDataPlanAsync(PhysicalPlan plan, Node receiver, RaftNode header)
       throws IOException {
-    RaftService.AsyncClient client =
-        metaGroupMember
-            .getClientProvider()
-            .getAsyncDataClient(receiver, RaftServer.getWriteOperationTimeoutMS());
+    AsyncDataClient client =
+        ClusterIoTDB.getInstance()
+            .getAsyncDataClient(receiver, ClusterConstant.getWriteOperationTimeoutMS());
     return this.metaGroupMember.forwardPlanAsync(plan, receiver, header, client);
   }
 
   private TSStatus forwardDataPlanSync(PhysicalPlan plan, Node receiver, RaftNode header)
       throws IOException {
-    RaftService.Client client;
-    try {
-      client =
-          metaGroupMember
-              .getClientProvider()
-              .getSyncDataClient(receiver, RaftServer.getWriteOperationTimeoutMS());
-    } catch (TException e) {
-      throw new IOException(e);
-    }
+    SyncDataClient client =
+        ClusterIoTDB.getInstance()
+            .getSyncDataClient(receiver, ClusterConstant.getWriteOperationTimeoutMS());
     return this.metaGroupMember.forwardPlanSync(plan, receiver, header, 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 AsyncDataClient getAsyncDataClient(Node node, int timeout) throws IOException {
-    return metaGroupMember.getClientProvider().getAsyncDataClient(node, timeout);
-  }
-
   public Node getThisNode() {
     return thisNode;
   }
-
-  /**
-   * 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) throws TException {
-    return metaGroupMember.getClientProvider().getSyncDataClient(node, timeout);
-  }
 }
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
index a600c4a..226593a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/LogDispatcher.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/LogDispatcher.java
@@ -30,11 +30,11 @@ import org.apache.iotdb.cluster.server.member.RaftMember;
 import org.apache.iotdb.cluster.server.monitor.Peer;
 import org.apache.iotdb.cluster.server.monitor.Timer;
 import org.apache.iotdb.cluster.utils.ClientUtils;
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.utils.TestOnly;
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.thrift.TException;
 import org.apache.thrift.async.AsyncMethodCallback;
 import org.slf4j.Logger;
@@ -46,7 +46,6 @@ 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.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -66,16 +65,19 @@ public class LogDispatcher {
   private RaftMember member;
   private boolean useBatchInLogCatchUp =
       ClusterDescriptor.getInstance().getConfig().isUseBatchInLogCatchUp();
+  // each follower has a queue and a dispatch thread is attached in executorService.
   private List<BlockingQueue<SendLogRequest>> nodeLogQueues = new ArrayList<>();
   private ExecutorService executorService;
-  private static ExecutorService serializationService =
-      Executors.newFixedThreadPool(
-          Runtime.getRuntime().availableProcessors(),
-          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("DispatcherEncoder-%d").build());
+
+  // TODO we have no way to close this pool.
+  private static final ExecutorService serializationService =
+      IoTDBThreadPoolFactory.newFixedThreadPoolWithDaemonThread(
+          Runtime.getRuntime().availableProcessors(), "DispatcherEncoder");
 
   public LogDispatcher(RaftMember member) {
     this.member = member;
-    executorService = Executors.newCachedThreadPool();
+    executorService =
+        IoTDBThreadPoolFactory.newCachedThreadPool("LogDispatcher-" + member.getName());
     for (Node node : member.getAllNodes()) {
       if (!node.equals(member.getThisNode())) {
         nodeLogQueues.add(createQueueAndBindingThread(node));
@@ -90,6 +92,8 @@ public class LogDispatcher {
   }
 
   public void offer(SendLogRequest log) {
+    // if nodeLogQueues.isEmpty(), then nothing to do.
+
     // do serialization here to avoid taking LogManager for too long
     if (!nodeLogQueues.isEmpty()) {
       log.serializedLogFuture =
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/DataLogApplier.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/DataLogApplier.java
index 14abaa6..9a8b81a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/DataLogApplier.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/DataLogApplier.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.cluster.log.applier;
 
+import org.apache.iotdb.cluster.ClusterIoTDB;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.log.Log;
 import org.apache.iotdb.cluster.log.logtypes.AddNodeLog;
@@ -67,14 +68,14 @@ public class DataLogApplier extends BaseApplier {
 
     try {
       if (log instanceof AddNodeLog) {
-        metaGroupMember
-            .getDataClusterServer()
+        ClusterIoTDB.getInstance()
+            .getDataGroupEngine()
             .preAddNodeForDataGroup((AddNodeLog) log, dataGroupMember);
         dataGroupMember.setAndSaveLastAppliedPartitionTableVersion(
             ((AddNodeLog) log).getMetaLogIndex());
       } else if (log instanceof RemoveNodeLog) {
-        metaGroupMember
-            .getDataClusterServer()
+        ClusterIoTDB.getInstance()
+            .getDataGroupEngine()
             .preRemoveNodeForDataGroup((RemoveNodeLog) log, dataGroupMember);
         dataGroupMember.setAndSaveLastAppliedPartitionTableVersion(
             ((RemoveNodeLog) log).getMetaLogIndex());
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/LogCatchUpTask.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/LogCatchUpTask.java
index 99b3e4d..544cc71 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/LogCatchUpTask.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/catchup/LogCatchUpTask.java
@@ -29,7 +29,6 @@ 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;
@@ -53,7 +52,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 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 long SEND_LOGS_WAIT_MS = ClusterConstant.getWriteOperationTimeoutMS();
   private static final Logger logger = LoggerFactory.getLogger(LogCatchUpTask.class);
   Node node;
   RaftMember raftMember;
@@ -129,7 +128,7 @@ public class LogCatchUpTask implements Callable<Boolean> {
       }
       client.appendEntry(request, handler);
       raftMember.getLastCatchUpResponseTime().put(node, System.currentTimeMillis());
-      handler.getAppendSucceed().wait(RaftServer.getWriteOperationTimeoutMS());
+      handler.getAppendSucceed().wait(ClusterConstant.getWriteOperationTimeoutMS());
     }
     return handler.getAppendSucceed().get();
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/FilePartitionedSnapshotLogManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/FilePartitionedSnapshotLogManager.java
index 29c1258..5e61874 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/FilePartitionedSnapshotLogManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/FilePartitionedSnapshotLogManager.java
@@ -53,6 +53,8 @@ import java.util.Map.Entry;
  * 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.
+ *
+ * <p>FilePartitionedSnapshotLogManager is used for dataGroup
  */
 public class FilePartitionedSnapshotLogManager extends PartitionedSnapshotLogManager<FileSnapshot> {
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/PartitionedSnapshotLogManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/PartitionedSnapshotLogManager.java
index 20a6944..71ba5c93 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/PartitionedSnapshotLogManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/PartitionedSnapshotLogManager.java
@@ -69,7 +69,10 @@ public abstract class PartitionedSnapshotLogManager<T extends Snapshot> extends
       Node thisNode,
       SnapshotFactory<T> factory,
       DataGroupMember dataGroupMember) {
-    super(new SyncLogDequeSerializer(header.nodeIdentifier), logApplier, header.toString());
+    super(
+        new SyncLogDequeSerializer(header.nodeIdentifier),
+        logApplier,
+        Integer.toString(header.getNodeIdentifier()));
     this.partitionTable = partitionTable;
     this.factory = factory;
     this.thisNode = thisNode;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
index e5bfbf6..653680d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
@@ -32,10 +32,10 @@ import org.apache.iotdb.cluster.log.Snapshot;
 import org.apache.iotdb.cluster.log.StableEntryManager;
 import org.apache.iotdb.cluster.log.manage.serializable.LogManagerMeta;
 import org.apache.iotdb.cluster.server.monitor.Timer.Statistic;
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.tsfile.utils.RamUsageEstimator;
 
-import org.apache.commons.lang3.concurrent.BasicThreadFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,11 +45,9 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
 public abstract class RaftLogManager {
@@ -144,19 +142,10 @@ public abstract class RaftLogManager {
     this.blockedUnappliedLogList = new CopyOnWriteArrayList<>();
 
     this.deleteLogExecutorService =
-        new ScheduledThreadPoolExecutor(
-            1,
-            new BasicThreadFactory.Builder()
-                .namingPattern("raft-log-delete-" + name)
-                .daemon(true)
-                .build());
+        IoTDBThreadPoolFactory.newScheduledThreadPoolWithDaemon(1, "raft-log-delete-" + name);
 
     this.checkLogApplierExecutorService =
-        Executors.newSingleThreadExecutor(
-            new BasicThreadFactory.Builder()
-                .namingPattern("check-log-applier-" + name)
-                .daemon(true)
-                .build());
+        IoTDBThreadPoolFactory.newSingleThreadExecutorWithDaemon("check-log-applier-" + name);
 
     /** deletion check period of the submitted log */
     int logDeleteCheckIntervalSecond =
@@ -782,11 +771,7 @@ public abstract class RaftLogManager {
     this.blockAppliedCommitIndex = -1;
     this.blockedUnappliedLogList = new CopyOnWriteArrayList<>();
     this.checkLogApplierExecutorService =
-        Executors.newSingleThreadExecutor(
-            new BasicThreadFactory.Builder()
-                .namingPattern("check-log-applier-" + name)
-                .daemon(true)
-                .build());
+        IoTDBThreadPoolFactory.newSingleThreadExecutorWithDaemon("check-log-applier-" + name);
     this.checkLogApplierFuture = checkLogApplierExecutorService.submit(this::checkAppliedLogIndex);
     for (int i = 0; i < logUpdateConditions.length; i++) {
       logUpdateConditions[i] = new Object();
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
index 326699c..6822219 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
@@ -33,7 +33,6 @@ import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.server.handlers.caller.GenericHandler;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
 import org.apache.iotdb.cluster.server.member.RaftMember;
-import org.apache.iotdb.cluster.utils.ClientUtils;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
@@ -351,13 +350,13 @@ public class FileSnapshot extends Snapshot implements TimeseriesSchemaSnapshot {
         try {
           client.removeHardLink(resource.getTsFile().getAbsolutePath());
         } catch (TException te) {
-          client.getInputProtocol().getTransport().close();
+          client.close();
           logger.error(
               "Cannot remove hardlink {} from {}",
               resource.getTsFile().getAbsolutePath(),
               sourceNode);
         } finally {
-          ClientUtils.putBackSyncClient(client);
+          client.returnSelf();
         }
       }
     }
@@ -533,7 +532,8 @@ public class FileSnapshot extends Snapshot implements TimeseriesSchemaSnapshot {
         if (client == null) {
           throw new IOException("No available client for " + node.toString());
         }
-        ByteBuffer buffer = SyncClientAdaptor.readFile(client, remotePath, offset, fetchSize);
+        ByteBuffer buffer;
+        buffer = SyncClientAdaptor.readFile(client, remotePath, offset, fetchSize);
         int len = writeBuffer(buffer, dest);
         if (len == 0) {
           break;
@@ -579,9 +579,9 @@ public class FileSnapshot extends Snapshot implements TimeseriesSchemaSnapshot {
           offset += len;
         }
       } catch (TException e) {
-        client.getInputProtocol().getTransport().close();
+        client.close();
       } finally {
-        ClientUtils.putBackSyncClient(client);
+        client.returnSelf();
       }
       dest.flush();
     }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/MetaSimpleSnapshot.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/MetaSimpleSnapshot.java
index 8915a8b..23c5d7c 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/MetaSimpleSnapshot.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/MetaSimpleSnapshot.java
@@ -260,7 +260,7 @@ public class MetaSimpleSnapshot extends Snapshot {
         TemplateManager.getInstance().setTemplateMap(snapshot.templateMap);
 
         // 5. accept partition table
-        metaGroupMember.acceptPartitionTable(snapshot.getPartitionTableBuffer(), true);
+        metaGroupMember.acceptVerifiedPartitionTable(snapshot.getPartitionTableBuffer(), true);
 
         synchronized (metaGroupMember.getLogManager()) {
           metaGroupMember.getLogManager().applySnapshot(snapshot);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTask.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTask.java
index daa813d..50a5d99 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTask.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTask.java
@@ -30,7 +30,6 @@ import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotResp;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
-import org.apache.iotdb.cluster.utils.ClientUtils;
 
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
@@ -163,10 +162,10 @@ public class PullSnapshotTask<T extends Snapshot> implements Callable<Void> {
       try {
         pullSnapshotResp = client.pullSnapshot(request);
       } catch (TException e) {
-        client.getInputProtocol().getTransport().close();
+        client.close();
         throw e;
       } finally {
-        ClientUtils.putBackSyncClient(client);
+        client.returnSelf();
       }
       result = new HashMap<>();
       for (Entry<Integer, ByteBuffer> integerByteBufferEntry :
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
index ecd9736..644a869 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
@@ -19,9 +19,11 @@
 
 package org.apache.iotdb.cluster.metadata;
 
+import org.apache.iotdb.cluster.ClusterIoTDB;
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.coordinator.Coordinator;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
@@ -31,7 +33,6 @@ import org.apache.iotdb.cluster.query.manage.QueryCoordinator;
 import org.apache.iotdb.cluster.rpc.thrift.GetAllPathsResult;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
-import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.db.conf.IoTDBConstant;
@@ -891,8 +892,8 @@ public class CMManager extends MManager {
   private List<String> getUnregisteredSeriesListLocally(
       List<String> seriesList, PartitionGroup partitionGroup) throws CheckConsistencyException {
     DataGroupMember dataMember =
-        metaGroupMember
-            .getDataClusterServer()
+        ClusterIoTDB.getInstance()
+            .getDataGroupEngine()
             .getDataMember(partitionGroup.getHeader(), null, null);
     return dataMember.getLocalQueryExecutor().getUnregisteredTimeseries(seriesList);
   }
@@ -900,56 +901,65 @@ public class CMManager extends MManager {
   private List<String> getUnregisteredSeriesListRemotely(
       List<String> seriesList, PartitionGroup partitionGroup) {
     for (Node node : partitionGroup) {
+      List<String> result = null;
       try {
-        List<String> result;
-        if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-          AsyncDataClient client =
-              metaGroupMember
-                  .getClientProvider()
-                  .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
-          result =
-              SyncClientAdaptor.getUnregisteredMeasurements(
-                  client, partitionGroup.getHeader(), seriesList);
-        } else {
-          try (SyncDataClient syncDataClient =
-              metaGroupMember
-                  .getClientProvider()
-                  .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-            try {
-              result =
-                  syncDataClient.getUnregisteredTimeseries(partitionGroup.getHeader(), seriesList);
-            } catch (TException e) {
-              // the connection may be broken, close it to avoid it being reused
-              syncDataClient.getInputProtocol().getTransport().close();
-              throw e;
-            }
-          }
-        }
-        if (result != null) {
-          return result;
-        }
-      } catch (TException | IOException e) {
+        result = getUnregisteredSeriesListRemotelyForOneNode(node, seriesList, partitionGroup);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
         logger.error(
-            "{}: cannot getting unregistered {} and other {} paths from {}",
+            "{}: getting unregistered series list {} ... {} is interrupted from {}",
             metaGroupMember.getName(),
             seriesList.get(0),
             seriesList.get(seriesList.size() - 1),
             node,
             e);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
+      } catch (Exception e) {
         logger.error(
-            "{}: getting unregistered series list {} ... {} is interrupted from {}",
+            "{}: cannot getting unregistered {} and other {} paths from {}",
             metaGroupMember.getName(),
             seriesList.get(0),
             seriesList.get(seriesList.size() - 1),
             node,
             e);
       }
+      if (result != null) {
+        return result;
+      }
     }
     return Collections.emptyList();
   }
 
+  private List<String> getUnregisteredSeriesListRemotelyForOneNode(
+      Node node, List<String> seriesList, PartitionGroup partitionGroup)
+      throws IOException, TException, InterruptedException {
+    List<String> result;
+    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      AsyncDataClient client =
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+      result =
+          SyncClientAdaptor.getUnregisteredMeasurements(
+              client, partitionGroup.getHeader(), seriesList);
+    } else {
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+        result = syncDataClient.getUnregisteredTimeseries(partitionGroup.getHeader(), seriesList);
+      } catch (TException e) {
+        // the connection may be broken, close it to avoid it being reused
+        syncDataClient.close();
+        throw e;
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
+        }
+      }
+    }
+    return result;
+  }
+
   /**
    * Get all devices after removing wildcards in the path
    *
@@ -1067,21 +1077,23 @@ public class CMManager extends MManager {
     GetAllPathsResult result;
     if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
       AsyncDataClient client =
-          metaGroupMember
-              .getClientProvider()
-              .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
       result = SyncClientAdaptor.getAllPaths(client, header, pathsToQuery, withAlias);
     } else {
-      try (SyncDataClient syncDataClient =
-          metaGroupMember
-              .getClientProvider()
-              .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-        try {
-          result = syncDataClient.getAllPaths(header, pathsToQuery, withAlias);
-        } catch (TException e) {
-          // the connection may be broken, close it to avoid it being reused
-          syncDataClient.getInputProtocol().getTransport().close();
-          throw e;
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+        result = syncDataClient.getAllPaths(header, pathsToQuery, withAlias);
+      } catch (TException e) {
+        // the connection may be broken, close it to avoid it being reused
+        syncDataClient.close();
+        throw e;
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
         }
       }
     }
@@ -1093,7 +1105,7 @@ public class CMManager extends MManager {
       for (int i = 0; i < result.paths.size(); i++) {
         PartialPath matchedPath = getAssembledPathFromRequest(result.paths.get(i));
         partialPaths.add(matchedPath);
-        if (withAlias) {
+        if (withAlias && matchedPath != null) {
           matchedPath.setMeasurementAlias(result.aliasList.get(i));
         }
       }
@@ -1196,22 +1208,26 @@ public class CMManager extends MManager {
     Set<String> paths;
     if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
       AsyncDataClient client =
-          metaGroupMember
-              .getClientProvider()
-              .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
       paths = SyncClientAdaptor.getAllDevices(client, header, pathsToQuery);
     } else {
-      try (SyncDataClient syncDataClient =
-          metaGroupMember
-              .getClientProvider()
-              .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
         try {
           paths = syncDataClient.getAllDevices(header, pathsToQuery);
         } catch (TException e) {
           // the connection may be broken, close it to avoid it being reused
-          syncDataClient.getInputProtocol().getTransport().close();
+          syncDataClient.close();
           throw e;
         }
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
+        }
       }
     }
     return paths;
@@ -1265,6 +1281,9 @@ public class CMManager extends MManager {
       List<PartialPath> originalPaths) {
     ConcurrentSkipListSet<PartialPath> fullPaths = new ConcurrentSkipListSet<>();
     ConcurrentSkipListSet<PartialPath> nonExistPaths = new ConcurrentSkipListSet<>();
+    // TODO it is not suitable for register and deregister an Object to JMX to such a frequent
+    // function call.
+    // BUT is it suitable to create a thread pool for each calling??
     ExecutorService getAllPathsService =
         Executors.newFixedThreadPool(metaGroupMember.getPartitionTable().getGlobalGroups().size());
     for (PartialPath pathStr : originalPaths) {
@@ -1286,7 +1305,7 @@ public class CMManager extends MManager {
     getAllPathsService.shutdown();
     try {
       getAllPathsService.awaitTermination(
-          RaftServer.getReadOperationTimeoutMS(), TimeUnit.MILLISECONDS);
+          ClusterConstant.getReadOperationTimeoutMS(), TimeUnit.MILLISECONDS);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       logger.error("Unexpected interruption when waiting for get all paths services to stop", e);
@@ -1577,10 +1596,8 @@ public class CMManager extends MManager {
         if (resultBinary != null) {
           break;
         }
-      } catch (IOException e) {
+      } catch (IOException | TException e) {
         logger.error(LOG_FAIL_CONNECT, node, e);
-      } catch (TException e) {
-        logger.error("Error occurs when getting timeseries schemas in node {}.", node, e);
       } catch (InterruptedException e) {
         logger.error("Interrupted when getting timeseries schemas in node {}.", node, e);
         Thread.currentThread().interrupt();
@@ -1608,10 +1625,8 @@ public class CMManager extends MManager {
         if (resultBinary != null) {
           break;
         }
-      } catch (IOException e) {
+      } catch (IOException | TException e) {
         logger.error(LOG_FAIL_CONNECT, node, e);
-      } catch (TException e) {
-        logger.error("Error occurs when getting devices schemas in node {}.", node, e);
       } catch (InterruptedException e) {
         logger.error("Interrupted when getting devices schemas in node {}.", node, e);
         Thread.currentThread().interrupt();
@@ -1635,17 +1650,16 @@ public class CMManager extends MManager {
 
     if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
       AsyncDataClient client =
-          metaGroupMember
-              .getClientProvider()
-              .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
       resultBinary = SyncClientAdaptor.getAllMeasurementSchema(client, group.getHeader(), plan);
     } else {
+      SyncDataClient syncDataClient = null;
       try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-          DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
-          SyncDataClient syncDataClient =
-              metaGroupMember
-                  .getClientProvider()
-                  .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
+          DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
         try {
           plan.serialize(dataOutputStream);
           resultBinary =
@@ -1653,9 +1667,13 @@ public class CMManager extends MManager {
                   group.getHeader(), ByteBuffer.wrap(byteArrayOutputStream.toByteArray()));
         } catch (TException e) {
           // the connection may be broken, close it to avoid it being reused
-          syncDataClient.getInputProtocol().getTransport().close();
+          syncDataClient.close();
           throw e;
         }
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
+        }
       }
     }
     return resultBinary;
@@ -1666,26 +1684,27 @@ public class CMManager extends MManager {
     ByteBuffer resultBinary;
     if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
       AsyncDataClient client =
-          metaGroupMember
-              .getClientProvider()
-              .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
       resultBinary = SyncClientAdaptor.getDevices(client, group.getHeader(), plan);
     } else {
+      SyncDataClient syncDataClient = null;
       try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-          DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
-          SyncDataClient syncDataClient =
-              metaGroupMember
-                  .getClientProvider()
-                  .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-        try {
-          plan.serialize(dataOutputStream);
-          resultBinary =
-              syncDataClient.getDevices(
-                  group.getHeader(), ByteBuffer.wrap(byteArrayOutputStream.toByteArray()));
-        } catch (TException e) {
-          // the connection may be broken, close it to avoid it being reused
-          syncDataClient.getInputProtocol().getTransport().close();
-          throw e;
+          DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+        plan.serialize(dataOutputStream);
+        resultBinary =
+            syncDataClient.getDevices(
+                group.getHeader(), ByteBuffer.wrap(byteArrayOutputStream.toByteArray()));
+      } catch (TException e) {
+        // the connection may be broken, close it to avoid it being reused
+        syncDataClient.close();
+        throw e;
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
         }
       }
     }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/MetaPuller.java b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/MetaPuller.java
index 6715e23..66425df 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/MetaPuller.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/MetaPuller.java
@@ -19,9 +19,11 @@
 
 package org.apache.iotdb.cluster.metadata;
 
+import org.apache.iotdb.cluster.ClusterIoTDB;
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.partition.PartitionGroup;
@@ -30,7 +32,6 @@ import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.PullSchemaRequest;
 import org.apache.iotdb.cluster.rpc.thrift.PullSchemaResp;
 import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
-import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.cluster.utils.ClusterUtils;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
@@ -225,35 +226,37 @@ public class MetaPuller {
   }
 
   private List<IMeasurementSchema> pullMeasurementSchemas(Node node, PullSchemaRequest request)
-      throws TException, InterruptedException, IOException {
+      throws IOException, TException, InterruptedException {
     List<IMeasurementSchema> schemas;
     if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
       AsyncDataClient client =
-          metaGroupMember
-              .getClientProvider()
-              .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
       schemas = SyncClientAdaptor.pullMeasurementSchema(client, request);
     } else {
-      try (SyncDataClient syncDataClient =
-          metaGroupMember
-              .getClientProvider()
-              .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-        try {
-          // only need measurement name
-          PullSchemaResp pullSchemaResp = syncDataClient.pullMeasurementSchema(request);
-          ByteBuffer buffer = pullSchemaResp.schemaBytes;
-          int size = buffer.getInt();
-          schemas = new ArrayList<>(size);
-          for (int i = 0; i < size; i++) {
-            schemas.add(
-                buffer.get() == 0
-                    ? UnaryMeasurementSchema.partialDeserializeFrom(buffer)
-                    : VectorMeasurementSchema.partialDeserializeFrom(buffer));
-          }
-        } catch (TException e) {
-          // the connection may be broken, close it to avoid it being reused
-          syncDataClient.getInputProtocol().getTransport().close();
-          throw e;
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+        // only need measurement name
+        PullSchemaResp pullSchemaResp = syncDataClient.pullMeasurementSchema(request);
+        ByteBuffer buffer = pullSchemaResp.schemaBytes;
+        int size = buffer.getInt();
+        schemas = new ArrayList<>(size);
+        for (int i = 0; i < size; i++) {
+          schemas.add(
+              buffer.get() == 0
+                  ? UnaryMeasurementSchema.partialDeserializeFrom(buffer)
+                  : VectorMeasurementSchema.partialDeserializeFrom(buffer));
+        }
+      } catch (TException e) {
+        // the connection may be broken, close it to avoid it being reused
+        syncDataClient.close();
+        throw e;
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
         }
       }
     }
@@ -415,20 +418,19 @@ public class MetaPuller {
    * null if there was a timeout.
    */
   private List<TimeseriesSchema> pullTimeSeriesSchemas(Node node, PullSchemaRequest request)
-      throws TException, InterruptedException, IOException {
+      throws IOException, TException, InterruptedException {
     List<TimeseriesSchema> schemas;
     if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
       AsyncDataClient client =
-          metaGroupMember
-              .getClientProvider()
-              .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
       schemas = SyncClientAdaptor.pullTimeseriesSchema(client, request);
     } else {
-      try (SyncDataClient syncDataClient =
-          metaGroupMember
-              .getClientProvider()
-              .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
         PullSchemaResp pullSchemaResp = syncDataClient.pullTimeSeriesSchema(request);
         ByteBuffer buffer = pullSchemaResp.schemaBytes;
         int size = buffer.getInt();
@@ -436,6 +438,13 @@ public class MetaPuller {
         for (int i = 0; i < size; i++) {
           schemas.add(TimeseriesSchema.deserializeFrom(buffer));
         }
+      } catch (TException e) {
+        syncDataClient.close();
+        throw e;
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
+        }
       }
     }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionGroup.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionGroup.java
index 1ce653e..b86726b 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionGroup.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionGroup.java
@@ -102,4 +102,9 @@ public class PartitionGroup extends ArrayList<Node> {
   public void setId(int id) {
     this.id = id;
   }
+
+  @Override
+  public String toString() {
+    return super.toString() + ", id = " + id;
+  }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java
index ee62dc8..e62a1af 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java
@@ -19,9 +19,11 @@
 
 package org.apache.iotdb.cluster.query;
 
+import org.apache.iotdb.cluster.ClusterIoTDB;
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.metadata.CMManager;
@@ -30,7 +32,6 @@ import org.apache.iotdb.cluster.partition.slot.SlotPartitionTable;
 import org.apache.iotdb.cluster.query.filter.SlotSgFilter;
 import org.apache.iotdb.cluster.query.manage.QueryCoordinator;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
-import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -78,7 +79,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 public class ClusterPlanExecutor extends PlanExecutor {
 
   private static final Logger logger = LoggerFactory.getLogger(ClusterPlanExecutor.class);
-  private MetaGroupMember metaGroupMember;
+  private final MetaGroupMember metaGroupMember;
 
   public static final int THREAD_POOL_SIZE = 6;
   public static final String LOG_FAIL_CONNECT = "Failed to connect to node: {}";
@@ -213,32 +214,10 @@ public class ClusterPlanExecutor extends PlanExecutor {
       throws MetadataException {
     // choose the node with lowest latency or highest throughput
     List<Node> coordinatedNodes = QueryCoordinator.getINSTANCE().reorderNodes(partitionGroup);
+    Integer count;
     for (Node node : coordinatedNodes) {
       try {
-        Integer count;
-        if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-          AsyncDataClient client =
-              metaGroupMember
-                  .getClientProvider()
-                  .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
-          client.setTimeout(RaftServer.getReadOperationTimeoutMS());
-          count =
-              SyncClientAdaptor.getDeviceCount(client, partitionGroup.getHeader(), pathsToCount);
-        } else {
-          try (SyncDataClient syncDataClient =
-              metaGroupMember
-                  .getClientProvider()
-                  .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-            try {
-              syncDataClient.setTimeout(RaftServer.getReadOperationTimeoutMS());
-              count = syncDataClient.getDeviceCount(partitionGroup.getHeader(), pathsToCount);
-            } catch (TException e) {
-              // the connection may be broken, close it to avoid it being reused
-              syncDataClient.getInputProtocol().getTransport().close();
-              throw e;
-            }
-          }
-        }
+        count = getRemoteDeviceCountForOneNode(node, partitionGroup, pathsToCount);
         logger.debug(
             "{}: get device count of {} from {}, result {}",
             metaGroupMember.getName(),
@@ -255,10 +234,41 @@ public class ClusterPlanExecutor extends PlanExecutor {
         throw new MetadataException(e);
       }
     }
-    logger.warn("Cannot get devices count of {} from {}", pathsToCount, partitionGroup);
+    logger.warn("Cannot get devices of {} from {}", pathsToCount, partitionGroup);
     return 0;
   }
 
+  private Integer getRemoteDeviceCountForOneNode(
+      Node node, PartitionGroup partitionGroup, List<String> pathsToCount)
+      throws IOException, TException, InterruptedException {
+    Integer count;
+    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      AsyncDataClient client =
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+      client.setTimeout(ClusterConstant.getReadOperationTimeoutMS());
+      count = SyncClientAdaptor.getDeviceCount(client, partitionGroup.getHeader(), pathsToCount);
+    } else {
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+        syncDataClient.setTimeout(ClusterConstant.getReadOperationTimeoutMS());
+        count = syncDataClient.getDeviceCount(partitionGroup.getHeader(), pathsToCount);
+      } catch (TException e) {
+        // the connection may be broken, close it to avoid it being reused
+        syncDataClient.close();
+        throw e;
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
+        }
+      }
+    }
+    return count;
+  }
+
   @Override
   protected int getPathsNum(PartialPath path) throws MetadataException {
     return getNodesNumInGivenLevel(path, -1);
@@ -330,7 +340,7 @@ public class ClusterPlanExecutor extends PlanExecutor {
     if (groupPathMap.isEmpty()) {
       return result.get();
     }
-
+    // TODO: create a thread pool for each query calling.
     ExecutorService remoteQueryThreadPool = Executors.newFixedThreadPool(groupPathMap.size());
     List<Future<Void>> remoteFutures = new ArrayList<>();
     // query each data group separately
@@ -372,33 +382,10 @@ public class ClusterPlanExecutor extends PlanExecutor {
       throws MetadataException {
     // choose the node with lowest latency or highest throughput
     List<Node> coordinatedNodes = QueryCoordinator.getINSTANCE().reorderNodes(partitionGroup);
+    Integer count;
     for (Node node : coordinatedNodes) {
       try {
-        Integer count;
-        if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-          AsyncDataClient client =
-              metaGroupMember
-                  .getClientProvider()
-                  .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
-          client.setTimeout(RaftServer.getReadOperationTimeoutMS());
-          count =
-              SyncClientAdaptor.getPathCount(
-                  client, partitionGroup.getHeader(), pathsToQuery, level);
-        } else {
-          try (SyncDataClient syncDataClient =
-              metaGroupMember
-                  .getClientProvider()
-                  .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-            try {
-              syncDataClient.setTimeout(RaftServer.getReadOperationTimeoutMS());
-              count = syncDataClient.getPathCount(partitionGroup.getHeader(), pathsToQuery, level);
-            } catch (TException e) {
-              // the connection may be broken, close it to avoid it being reused
-              syncDataClient.getInputProtocol().getTransport().close();
-              throw e;
-            }
-          }
-        }
+        count = getRemotePathCountForOneNode(node, partitionGroup, pathsToQuery, level);
         logger.debug(
             "{}: get path count of {} from {}, result {}",
             metaGroupMember.getName(),
@@ -419,13 +406,45 @@ public class ClusterPlanExecutor extends PlanExecutor {
     return 0;
   }
 
+  private Integer getRemotePathCountForOneNode(
+      Node node, PartitionGroup partitionGroup, List<String> pathsToQuery, int level)
+      throws IOException, TException, InterruptedException {
+    Integer count;
+    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      AsyncDataClient client =
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+      client.setTimeout(ClusterConstant.getReadOperationTimeoutMS());
+      count =
+          SyncClientAdaptor.getPathCount(client, partitionGroup.getHeader(), pathsToQuery, level);
+    } else {
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+        syncDataClient.setTimeout(ClusterConstant.getReadOperationTimeoutMS());
+        count = syncDataClient.getPathCount(partitionGroup.getHeader(), pathsToQuery, level);
+      } catch (TException e) {
+        // the connection may be broken, close it to avoid it being reused
+        syncDataClient.close();
+        throw e;
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
+        }
+      }
+    }
+    return count;
+  }
+
   @Override
   protected List<PartialPath> getNodesList(PartialPath schemaPattern, int level)
       throws MetadataException {
-
     ConcurrentSkipListSet<PartialPath> nodeSet = new ConcurrentSkipListSet<>();
-    ExecutorService pool = Executors.newFixedThreadPool(THREAD_POOL_SIZE);
 
+    // TODO: create a thread pool for each query calling.
+    ExecutorService pool = Executors.newFixedThreadPool(THREAD_POOL_SIZE);
     List<Future<Void>> futureList = new ArrayList<>();
     for (PartitionGroup group : metaGroupMember.getPartitionTable().getGlobalGroups()) {
       futureList.add(
@@ -479,29 +498,7 @@ public class ClusterPlanExecutor extends PlanExecutor {
     List<String> paths = null;
     for (Node node : group) {
       try {
-        if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-          AsyncDataClient client =
-              metaGroupMember
-                  .getClientProvider()
-                  .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
-          paths =
-              SyncClientAdaptor.getNodeList(
-                  client, group.getHeader(), schemaPattern.getFullPath(), level);
-        } else {
-          try (SyncDataClient syncDataClient =
-              metaGroupMember
-                  .getClientProvider()
-                  .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-            try {
-              paths =
-                  syncDataClient.getNodeList(group.getHeader(), schemaPattern.getFullPath(), level);
-            } catch (TException e) {
-              // the connection may be broken, close it to avoid it being reused
-              syncDataClient.getInputProtocol().getTransport().close();
-              throw e;
-            }
-          }
-        }
+        paths = getRemoteNodesListForOneNode(node, group, schemaPattern, level);
         if (paths != null) {
           break;
         }
@@ -517,10 +514,42 @@ public class ClusterPlanExecutor extends PlanExecutor {
     return PartialPath.fromStringList(paths);
   }
 
+  private List<String> getRemoteNodesListForOneNode(
+      Node node, PartitionGroup group, PartialPath schemaPattern, int level)
+      throws TException, InterruptedException, IOException {
+    List<String> paths;
+    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      AsyncDataClient client =
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+      paths =
+          SyncClientAdaptor.getNodeList(
+              client, group.getHeader(), schemaPattern.getFullPath(), level);
+    } else {
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+        paths = syncDataClient.getNodeList(group.getHeader(), schemaPattern.getFullPath(), level);
+      } catch (TException e) {
+        // the connection may be broken, close it to avoid it being reused
+        syncDataClient.close();
+        throw e;
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
+        }
+      }
+    }
+    return paths;
+  }
+
   @Override
   protected Set<String> getNodeNextChildren(PartialPath path) throws MetadataException {
     ConcurrentSkipListSet<String> resultSet = new ConcurrentSkipListSet<>();
     List<PartitionGroup> globalGroups = metaGroupMember.getPartitionTable().getGlobalGroups();
+    // TODO: create a thread pool for each query calling.
     ExecutorService pool = Executors.newFixedThreadPool(THREAD_POOL_SIZE);
     List<Future<Void>> futureList = new ArrayList<>();
     for (PartitionGroup group : globalGroups) {
@@ -571,29 +600,7 @@ public class ClusterPlanExecutor extends PlanExecutor {
     Set<String> nextChildrenNodes = null;
     for (Node node : group) {
       try {
-        if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-          AsyncDataClient client =
-              metaGroupMember
-                  .getClientProvider()
-                  .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
-          nextChildrenNodes =
-              SyncClientAdaptor.getChildNodeInNextLevel(
-                  client, group.getHeader(), path.getFullPath());
-        } else {
-          try (SyncDataClient syncDataClient =
-              metaGroupMember
-                  .getClientProvider()
-                  .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-            try {
-              nextChildrenNodes =
-                  syncDataClient.getChildNodeInNextLevel(group.getHeader(), path.getFullPath());
-            } catch (TException e) {
-              // the connection may be broken, close it to avoid it being reused
-              syncDataClient.getInputProtocol().getTransport().close();
-              throw e;
-            }
-          }
-        }
+        nextChildrenNodes = getRemoteChildNodeInNextLevelForOneNode(node, group, path);
         if (nextChildrenNodes != null) {
           break;
         }
@@ -609,9 +616,41 @@ public class ClusterPlanExecutor extends PlanExecutor {
     return nextChildrenNodes;
   }
 
+  private Set<String> getRemoteChildNodeInNextLevelForOneNode(
+      Node node, PartitionGroup group, PartialPath path)
+      throws TException, InterruptedException, IOException {
+    Set<String> nextChildrenNodes;
+    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      AsyncDataClient client =
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+      nextChildrenNodes =
+          SyncClientAdaptor.getChildNodeInNextLevel(client, group.getHeader(), path.getFullPath());
+    } else {
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+        nextChildrenNodes =
+            syncDataClient.getChildNodeInNextLevel(group.getHeader(), path.getFullPath());
+      } catch (TException e) {
+        // the connection may be broken, close it to avoid it being reused
+        syncDataClient.close();
+        throw e;
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
+        }
+      }
+    }
+    return nextChildrenNodes;
+  }
+
   @Override
   protected Set<String> getPathNextChildren(PartialPath path) throws MetadataException {
     ConcurrentSkipListSet<String> resultSet = new ConcurrentSkipListSet<>();
+    // TODO: create a thread pool for each query calling.
     ExecutorService pool = Executors.newFixedThreadPool(THREAD_POOL_SIZE);
 
     List<Future<Void>> futureList = new ArrayList<>();
@@ -654,7 +693,7 @@ public class ClusterPlanExecutor extends PlanExecutor {
 
     pool.shutdown();
     try {
-      pool.awaitTermination(RaftServer.getReadOperationTimeoutMS(), TimeUnit.MILLISECONDS);
+      pool.awaitTermination(ClusterConstant.getReadOperationTimeoutMS(), TimeUnit.MILLISECONDS);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       logger.error("Unexpected interruption when waiting for {}", methodName, e);
@@ -686,28 +725,7 @@ public class ClusterPlanExecutor extends PlanExecutor {
     Set<String> nextChildren = null;
     for (Node node : group) {
       try {
-        if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-          AsyncDataClient client =
-              metaGroupMember
-                  .getClientProvider()
-                  .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
-          nextChildren =
-              SyncClientAdaptor.getNextChildren(client, group.getHeader(), path.getFullPath());
-        } else {
-          try (SyncDataClient syncDataClient =
-              metaGroupMember
-                  .getClientProvider()
-                  .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-            try {
-              nextChildren =
-                  syncDataClient.getChildNodePathInNextLevel(group.getHeader(), path.getFullPath());
-            } catch (TException e) {
-              // the connection may be broken, close it to avoid it being reused
-              syncDataClient.getInputProtocol().getTransport().close();
-              throw e;
-            }
-          }
-        }
+        nextChildren = getRemoteNextChildrenForOneNode(node, group, path);
         if (nextChildren != null) {
           break;
         }
@@ -723,6 +741,37 @@ public class ClusterPlanExecutor extends PlanExecutor {
     return nextChildren;
   }
 
+  private Set<String> getRemoteNextChildrenForOneNode(
+      Node node, PartitionGroup group, PartialPath path)
+      throws TException, InterruptedException, IOException {
+    Set<String> nextChildren;
+    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      AsyncDataClient client =
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+      nextChildren =
+          SyncClientAdaptor.getNextChildren(client, group.getHeader(), path.getFullPath());
+    } else {
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+        nextChildren =
+            syncDataClient.getChildNodePathInNextLevel(group.getHeader(), path.getFullPath());
+      } catch (TException e) {
+        // the connection may be broken, close it to avoid it being reused
+        syncDataClient.close();
+        throw e;
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
+        }
+      }
+    }
+    return nextChildren;
+  }
+
   @Override
   protected List<IStorageGroupMNode> getAllStorageGroupNodes() {
     try {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/aggregate/ClusterAggregator.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/aggregate/ClusterAggregator.java
index a27b8f3..39760b6 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/aggregate/ClusterAggregator.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/aggregate/ClusterAggregator.java
@@ -19,9 +19,11 @@
 
 package org.apache.iotdb.cluster.query.aggregate;
 
+import org.apache.iotdb.cluster.ClusterIoTDB;
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.exception.EmptyIntervalException;
@@ -32,7 +34,6 @@ import org.apache.iotdb.cluster.query.RemoteQueryContext;
 import org.apache.iotdb.cluster.query.manage.QueryCoordinator;
 import org.apache.iotdb.cluster.rpc.thrift.GetAggrResultRequest;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
-import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.db.exception.StorageEngineException;
@@ -256,25 +257,27 @@ public class ClusterAggregator {
 
   private List<ByteBuffer> getRemoteAggregateResult(Node node, GetAggrResultRequest request)
       throws IOException, TException, InterruptedException {
-    List<ByteBuffer> resultBuffers;
+    List<ByteBuffer> resultBuffers = null;
     if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
       AsyncDataClient client =
-          metaGroupMember
-              .getClientProvider()
-              .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
       // each buffer is an AggregationResult
       resultBuffers = SyncClientAdaptor.getAggrResult(client, request);
     } else {
-      try (SyncDataClient syncDataClient =
-          metaGroupMember
-              .getClientProvider()
-              .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-        try {
-          resultBuffers = syncDataClient.getAggrResult(request);
-        } catch (TException e) {
-          // the connection may be broken, close it to avoid it being reused
-          syncDataClient.getInputProtocol().getTransport().close();
-          throw e;
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+        resultBuffers = syncDataClient.getAggrResult(request);
+      } catch (TException e) {
+        // the connection may be broken, close it to avoid it being reused
+        syncDataClient.close();
+        throw e;
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
         }
       }
     }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/fill/ClusterPreviousFill.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/fill/ClusterPreviousFill.java
index cd78770..2e2171b 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/fill/ClusterPreviousFill.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/fill/ClusterPreviousFill.java
@@ -19,16 +19,17 @@
 
 package org.apache.iotdb.cluster.query.fill;
 
+import org.apache.iotdb.cluster.ClusterIoTDB;
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.exception.QueryTimeOutException;
 import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.PreviousFillRequest;
-import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.handlers.caller.PreviousFillHandler;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
@@ -59,6 +60,8 @@ public class ClusterPreviousFill extends PreviousFill {
   private static final Logger logger = LoggerFactory.getLogger(ClusterPreviousFill.class);
   private MetaGroupMember metaGroupMember;
   private TimeValuePair fillResult;
+  private static final String PREVIOUS_FILL_EXCEPTION_LOGGER_FORMAT =
+      "{}: Cannot perform previous fill of {} to {}";
 
   ClusterPreviousFill(PreviousFill fill, MetaGroupMember metaGroupMember) {
     super(fill.getDataType(), fill.getQueryStartTime(), fill.getBeforeRange());
@@ -120,7 +123,7 @@ public class ClusterPreviousFill extends PreviousFill {
     }
     CountDownLatch latch = new CountDownLatch(partitionGroups.size());
     PreviousFillHandler handler = new PreviousFillHandler(latch);
-
+    // TODO: create a thread pool for each query calling.
     ExecutorService fillService = Executors.newFixedThreadPool(partitionGroups.size());
     PreviousFillArguments arguments =
         new PreviousFillArguments(path, dataType, queryTime, beforeRange, deviceMeasurements);
@@ -130,7 +133,8 @@ public class ClusterPreviousFill extends PreviousFill {
     }
     fillService.shutdown();
     try {
-      fillService.awaitTermination(RaftServer.getReadOperationTimeoutMS(), TimeUnit.MILLISECONDS);
+      fillService.awaitTermination(
+          ClusterConstant.getReadOperationTimeoutMS(), TimeUnit.MILLISECONDS);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       logger.error("Unexpected interruption when waiting for fill pool to stop", e);
@@ -215,26 +219,16 @@ public class ClusterPreviousFill extends PreviousFill {
   private ByteBuffer remoteAsyncPreviousFill(
       Node node, PreviousFillRequest request, PreviousFillArguments arguments) {
     ByteBuffer byteBuffer = null;
-    AsyncDataClient asyncDataClient;
     try {
-      asyncDataClient =
-          metaGroupMember
-              .getClientProvider()
-              .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
+      AsyncDataClient asyncDataClient =
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+      byteBuffer = SyncClientAdaptor.previousFill(asyncDataClient, request);
     } catch (IOException e) {
       logger.warn("{}: Cannot connect to {} during previous fill", metaGroupMember, node);
-      return null;
-    }
-
-    try {
-      byteBuffer = SyncClientAdaptor.previousFill(asyncDataClient, request);
     } catch (Exception e) {
       logger.error(
-          "{}: Cannot perform previous fill of {} to {}",
-          metaGroupMember,
-          arguments.getPath(),
-          node,
-          e);
+          PREVIOUS_FILL_EXCEPTION_LOGGER_FORMAT, metaGroupMember, arguments.getPath(), node, e);
     }
     return byteBuffer;
   }
@@ -242,24 +236,33 @@ public class ClusterPreviousFill extends PreviousFill {
   private ByteBuffer remoteSyncPreviousFill(
       Node node, PreviousFillRequest request, PreviousFillArguments arguments) {
     ByteBuffer byteBuffer = null;
-    try (SyncDataClient syncDataClient =
-        metaGroupMember
-            .getClientProvider()
-            .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-      try {
-        byteBuffer = syncDataClient.previousFill(request);
-      } catch (TException e) {
-        // the connection may be broken, close it to avoid it being reused
-        syncDataClient.getInputProtocol().getTransport().close();
-        throw e;
-      }
+    SyncDataClient syncDataClient = null;
+    try {
+      syncDataClient =
+          ClusterIoTDB.getInstance()
+              .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+      byteBuffer = syncDataClient.previousFill(request);
+
+    } catch (TException e) {
+      // the connection may be broken, close it to avoid it being reused
+      syncDataClient.close();
+      logger.error(
+          PREVIOUS_FILL_EXCEPTION_LOGGER_FORMAT,
+          metaGroupMember.getName(),
+          arguments.getPath(),
+          node,
+          e);
     } catch (Exception e) {
       logger.error(
-          "{}: Cannot perform previous fill of {} to {}",
+          PREVIOUS_FILL_EXCEPTION_LOGGER_FORMAT,
           metaGroupMember.getName(),
           arguments.getPath(),
           node,
           e);
+    } finally {
+      if (syncDataClient != null) {
+        syncDataClient.returnSelf();
+      }
     }
     return byteBuffer;
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/RemoteGroupByExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/RemoteGroupByExecutor.java
index 99dfec6..9c6d0a5 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/RemoteGroupByExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/groupby/RemoteGroupByExecutor.java
@@ -19,14 +19,14 @@
 
 package org.apache.iotdb.cluster.query.groupby;
 
+import org.apache.iotdb.cluster.ClusterIoTDB;
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
-import org.apache.iotdb.cluster.server.RaftServer;
-import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
 import org.apache.iotdb.db.query.dataset.groupby.GroupByExecutor;
 import org.apache.iotdb.db.utils.SerializeUtils;
@@ -45,17 +45,14 @@ public class RemoteGroupByExecutor implements GroupByExecutor {
 
   private static final Logger logger = LoggerFactory.getLogger(RemoteGroupByExecutor.class);
 
-  private long executorId;
-  private MetaGroupMember metaGroupMember;
-  private Node source;
-  private RaftNode header;
+  private final long executorId;
+  private final Node source;
+  private final RaftNode header;
 
-  private List<AggregateResult> results = new ArrayList<>();
+  private final List<AggregateResult> results = new ArrayList<>();
 
-  public RemoteGroupByExecutor(
-      long executorId, MetaGroupMember metaGroupMember, Node source, RaftNode header) {
+  public RemoteGroupByExecutor(long executorId, Node source, RaftNode header) {
     this.executorId = executorId;
-    this.metaGroupMember = metaGroupMember;
     this.source = source;
     this.header = header;
   }
@@ -78,32 +75,34 @@ public class RemoteGroupByExecutor implements GroupByExecutor {
     try {
       if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
         AsyncDataClient client =
-            metaGroupMember
-                .getClientProvider()
-                .getAsyncDataClient(source, RaftServer.getReadOperationTimeoutMS());
+            ClusterIoTDB.getInstance()
+                .getAsyncDataClient(source, ClusterConstant.getReadOperationTimeoutMS());
         aggrBuffers =
             SyncClientAdaptor.getGroupByResult(
                 client, header, executorId, curStartTime, curEndTime);
       } else {
-        try (SyncDataClient syncDataClient =
-            metaGroupMember
-                .getClientProvider()
-                .getSyncDataClient(source, RaftServer.getReadOperationTimeoutMS())) {
-          try {
-            aggrBuffers =
-                syncDataClient.getGroupByResult(header, executorId, curStartTime, curEndTime);
-          } catch (TException e) {
-            // the connection may be broken, close it to avoid it being reused
-            syncDataClient.getInputProtocol().getTransport().close();
-            throw e;
+        SyncDataClient syncDataClient = null;
+        try {
+          syncDataClient =
+              ClusterIoTDB.getInstance()
+                  .getSyncDataClient(source, ClusterConstant.getReadOperationTimeoutMS());
+          aggrBuffers =
+              syncDataClient.getGroupByResult(header, executorId, curStartTime, curEndTime);
+        } catch (TException e) {
+          // the connection may be broken, close it to avoid it being reused
+          syncDataClient.close();
+          throw e;
+        } finally {
+          if (syncDataClient != null) {
+            syncDataClient.returnSelf();
           }
         }
       }
-    } catch (TException e) {
-      throw new IOException(e);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       throw new IOException(e);
+    } catch (Exception e) {
+      throw new IOException(e);
     }
     resetAggregateResults();
     if (aggrBuffers != null) {
@@ -128,24 +127,26 @@ public class RemoteGroupByExecutor implements GroupByExecutor {
     try {
       if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
         AsyncDataClient client =
-            metaGroupMember
-                .getClientProvider()
-                .getAsyncDataClient(source, RaftServer.getReadOperationTimeoutMS());
+            ClusterIoTDB.getInstance()
+                .getAsyncDataClient(source, ClusterConstant.getReadOperationTimeoutMS());
         aggrBuffer =
             SyncClientAdaptor.peekNextNotNullValue(
                 client, header, executorId, nextStartTime, nextEndTime);
       } else {
-        try (SyncDataClient syncDataClient =
-            metaGroupMember
-                .getClientProvider()
-                .getSyncDataClient(source, RaftServer.getReadOperationTimeoutMS())) {
-          try {
-            aggrBuffer =
-                syncDataClient.peekNextNotNullValue(header, executorId, nextStartTime, nextEndTime);
-          } catch (TException e) {
-            // the connection may be broken, close it to avoid it being reused
-            syncDataClient.getInputProtocol().getTransport().close();
-            throw e;
+        SyncDataClient syncDataClient = null;
+        try {
+          syncDataClient =
+              ClusterIoTDB.getInstance()
+                  .getSyncDataClient(source, ClusterConstant.getReadOperationTimeoutMS());
+          aggrBuffer =
+              syncDataClient.peekNextNotNullValue(header, executorId, nextStartTime, nextEndTime);
+        } catch (TException e) {
+          // the connection may be broken, close it to avoid it being reused
+          syncDataClient.close();
+          throw e;
+        } finally {
+          if (syncDataClient != null) {
+            syncDataClient.returnSelf();
           }
         }
       }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/last/ClusterLastQueryExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/last/ClusterLastQueryExecutor.java
index b42df93..1f038ef 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/last/ClusterLastQueryExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/last/ClusterLastQueryExecutor.java
@@ -19,18 +19,20 @@
 
 package org.apache.iotdb.cluster.query.last;
 
+import org.apache.iotdb.cluster.ClusterIoTDB;
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.rpc.thrift.LastQueryRequest;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
-import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.cluster.utils.ClusterQueryUtils;
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.PartialPath;
@@ -57,7 +59,6 @@ import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
 public class ClusterLastQueryExecutor extends LastQueryExecutor {
@@ -66,7 +67,8 @@ public class ClusterLastQueryExecutor extends LastQueryExecutor {
   private MetaGroupMember metaGroupMember;
 
   private static ExecutorService lastQueryPool =
-      Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
+      IoTDBThreadPoolFactory.newFixedThreadPool(
+          Runtime.getRuntime().availableProcessors(), "ClusterLastQuery");
 
   public ClusterLastQueryExecutor(LastQueryPlan lastQueryPlan, MetaGroupMember metaGroupMember) {
     super(lastQueryPlan);
@@ -223,7 +225,7 @@ public class ClusterLastQueryExecutor extends LastQueryExecutor {
             results.add(new Pair<>(true, pair));
           }
           return results;
-        } catch (TException e) {
+        } catch (IOException | TException e) {
           logger.warn("Query last of {} from {} errored", group, seriesPaths, e);
           return Collections.emptyList();
         } catch (InterruptedException e) {
@@ -236,17 +238,15 @@ public class ClusterLastQueryExecutor extends LastQueryExecutor {
     }
 
     private ByteBuffer lastAsync(Node node, QueryContext context)
-        throws TException, InterruptedException {
+        throws IOException, TException, InterruptedException {
       ByteBuffer buffer;
-      AsyncDataClient asyncDataClient;
-      try {
-        asyncDataClient =
-            metaGroupMember
-                .getClientProvider()
-                .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
-      } catch (IOException e) {
+      AsyncDataClient asyncDataClient =
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+      if (asyncDataClient == null) {
         return null;
       }
+
       buffer =
           SyncClientAdaptor.last(
               asyncDataClient,
@@ -258,26 +258,31 @@ public class ClusterLastQueryExecutor extends LastQueryExecutor {
       return buffer;
     }
 
-    private ByteBuffer lastSync(Node node, QueryContext context) throws TException {
+    private ByteBuffer lastSync(Node node, QueryContext context) throws IOException, TException {
       ByteBuffer res;
-      try (SyncDataClient syncDataClient =
-          metaGroupMember
-              .getClientProvider()
-              .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-        try {
-          res =
-              syncDataClient.last(
-                  new LastQueryRequest(
-                      PartialPath.toStringList(seriesPaths),
-                      dataTypeOrdinals,
-                      context.getQueryId(),
-                      queryPlan.getDeviceToMeasurements(),
-                      group.getHeader(),
-                      syncDataClient.getNode()));
-        } catch (TException e) {
-          // the connection may be broken, close it to avoid it being reused
-          syncDataClient.getInputProtocol().getTransport().close();
-          throw e;
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+        res =
+            syncDataClient.last(
+                new LastQueryRequest(
+                    PartialPath.toStringList(seriesPaths),
+                    dataTypeOrdinals,
+                    context.getQueryId(),
+                    queryPlan.getDeviceToMeasurements(),
+                    group.getHeader(),
+                    syncDataClient.getNode()));
+      } catch (IOException | TException e) {
+        // the connection may be broken, close it to avoid it being reused
+        if (syncDataClient != null) {
+          syncDataClient.close();
+        }
+        throw e;
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
         }
       }
       return res;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
index 1de13a2..078c95a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
@@ -19,9 +19,11 @@
 
 package org.apache.iotdb.cluster.query.reader;
 
+import org.apache.iotdb.cluster.ClusterIoTDB;
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
 import org.apache.iotdb.cluster.exception.EmptyIntervalException;
@@ -44,7 +46,6 @@ import org.apache.iotdb.cluster.rpc.thrift.MultSeriesQueryRequest;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.SingleSeriesQueryRequest;
-import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
 import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.cluster.utils.ClusterQueryUtils;
@@ -212,12 +213,7 @@ public class ClusterReaderFactory {
     List<Node> reorderedNodes = QueryCoordinator.getINSTANCE().reorderNodes(partitionGroup);
     DataSourceInfo dataSourceInfo =
         new DataSourceInfo(
-            partitionGroup,
-            dataType,
-            request,
-            (RemoteQueryContext) context,
-            metaGroupMember,
-            reorderedNodes);
+            partitionGroup, dataType, request, (RemoteQueryContext) context, reorderedNodes);
 
     // try building a reader from one of the nodes
     boolean hasClient = dataSourceInfo.hasNextDataClient(true, Long.MIN_VALUE);
@@ -243,9 +239,6 @@ public class ClusterReaderFactory {
    * @param valueFilter value filter
    * @param context query context
    * @param ascending asc or aesc
-   * @return
-   * @throws StorageEngineException
-   * @throws EmptyIntervalException
    */
   public List<AbstractMultPointReader> getMultSeriesReader(
       List<PartialPath> paths,
@@ -608,13 +601,7 @@ public class ClusterReaderFactory {
 
     MultDataSourceInfo dataSourceInfo =
         new MultDataSourceInfo(
-            partitionGroup,
-            paths,
-            dataType,
-            request,
-            (RemoteQueryContext) context,
-            metaGroupMember,
-            orderedNodes);
+            partitionGroup, paths, dataType, request, (RemoteQueryContext) context, orderedNodes);
 
     boolean hasClient = dataSourceInfo.hasNextDataClient(Long.MIN_VALUE);
     if (hasClient) {
@@ -669,12 +656,7 @@ public class ClusterReaderFactory {
 
     DataSourceInfo dataSourceInfo =
         new DataSourceInfo(
-            partitionGroup,
-            dataType,
-            request,
-            (RemoteQueryContext) context,
-            metaGroupMember,
-            orderedNodes);
+            partitionGroup, dataType, request, (RemoteQueryContext) context, orderedNodes);
 
     boolean hasClient = dataSourceInfo.hasNextDataClient(false, Long.MIN_VALUE);
     if (hasClient) {
@@ -901,8 +883,7 @@ public class ClusterReaderFactory {
               node);
           // create a remote executor with the return id
           RemoteGroupByExecutor remoteGroupByExecutor =
-              new RemoteGroupByExecutor(
-                  executorId, metaGroupMember, node, partitionGroup.getHeader());
+              new RemoteGroupByExecutor(executorId, node, partitionGroup.getHeader());
           for (Integer aggregationType : aggregationTypes) {
             remoteGroupByExecutor.addAggregateResult(
                 AggregateResultFactory.getAggrResultByType(
@@ -931,21 +912,23 @@ public class ClusterReaderFactory {
     Long executorId;
     if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
       AsyncDataClient client =
-          metaGroupMember
-              .getClientProvider()
-              .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
+          ClusterIoTDB.getInstance()
+              .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
       executorId = SyncClientAdaptor.getGroupByExecutor(client, request);
     } else {
-      try (SyncDataClient syncDataClient =
-          metaGroupMember
-              .getClientProvider()
-              .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-        try {
-          executorId = syncDataClient.getGroupByExecutor(request);
-        } catch (TException e) {
-          // the connection may be broken, close it to avoid it being reused
-          syncDataClient.getInputProtocol().getTransport().close();
-          throw e;
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+        executorId = syncDataClient.getGroupByExecutor(request);
+      } catch (TException e) {
+        // the connection may be broken, close it to avoid it being reused
+        syncDataClient.close();
+        throw e;
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
         }
       }
     }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/DataSourceInfo.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/DataSourceInfo.java
index 685f0fe..02be077 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/DataSourceInfo.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/DataSourceInfo.java
@@ -19,17 +19,17 @@
 
 package org.apache.iotdb.cluster.query.reader;
 
+import org.apache.iotdb.cluster.ClusterIoTDB;
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.sync.SyncClientAdaptor;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.query.RemoteQueryContext;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.rpc.thrift.SingleSeriesQueryRequest;
-import org.apache.iotdb.cluster.server.RaftServer;
-import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.db.utils.SerializeUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.TimeFilter;
@@ -58,7 +58,6 @@ public class DataSourceInfo {
   private TSDataType dataType;
   private SingleSeriesQueryRequest request;
   private RemoteQueryContext context;
-  private MetaGroupMember metaGroupMember;
   private List<Node> nodes;
   private int curPos;
   private boolean isNoData = false;
@@ -69,14 +68,12 @@ public class DataSourceInfo {
       TSDataType dataType,
       SingleSeriesQueryRequest request,
       RemoteQueryContext context,
-      MetaGroupMember metaGroupMember,
       List<Node> nodes) {
     this.readerId = -1;
     this.partitionGroup = group;
     this.dataType = dataType;
     this.request = request;
     this.context = context;
-    this.metaGroupMember = metaGroupMember;
     this.nodes = nodes;
     // set to the last node so after nextDataClient() is called it will scan from the first node
     this.curPos = nodes.size() - 1;
@@ -112,11 +109,11 @@ public class DataSourceInfo {
             return false;
           }
         }
-      } catch (TException | IOException e) {
-        logger.error("Cannot query {} from {}", this.request.path, node, e);
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         logger.error("Cannot query {} from {}", this.request.path, node, e);
+      } catch (Exception e) {
+        logger.error("Cannot query {} from {}", this.request.path, node, e);
       }
       nextNodePos = (nextNodePos + 1) % this.nodes.size();
       if (nextNodePos == this.curPos) {
@@ -131,7 +128,7 @@ public class DataSourceInfo {
   }
 
   private Long getReaderId(Node node, boolean byTimestamp, long timestamp)
-      throws TException, InterruptedException, IOException {
+      throws InterruptedException, TException, IOException {
     if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
       return applyForReaderIdAsync(node, byTimestamp, timestamp);
     }
@@ -139,12 +136,11 @@ public class DataSourceInfo {
   }
 
   private Long applyForReaderIdAsync(Node node, boolean byTimestamp, long timestamp)
-      throws TException, InterruptedException, IOException {
-    AsyncDataClient client =
-        this.metaGroupMember
-            .getClientProvider()
-            .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
+      throws IOException, TException, InterruptedException {
     Long newReaderId;
+    AsyncDataClient client =
+        ClusterIoTDB.getInstance()
+            .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
     if (byTimestamp) {
       newReaderId = SyncClientAdaptor.querySingleSeriesByTimestamp(client, request);
     } else {
@@ -154,35 +150,39 @@ public class DataSourceInfo {
   }
 
   private Long applyForReaderIdSync(Node node, boolean byTimestamp, long timestamp)
-      throws TException {
+      throws IOException, TException {
 
     Long newReaderId;
-    try (SyncDataClient client =
-        this.metaGroupMember
-            .getClientProvider()
-            .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
-
-      try {
-        if (byTimestamp) {
-          newReaderId = client.querySingleSeriesByTimestamp(request);
+    SyncDataClient client = null;
+    try {
+      client =
+          ClusterIoTDB.getInstance()
+              .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
+      if (byTimestamp) {
+        newReaderId = client.querySingleSeriesByTimestamp(request);
+      } else {
+        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(timestamp));
         } else {
-          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(timestamp));
-          } else {
-            newFilter = TimeFilter.gt(timestamp);
-          }
-          request.setTimeFilterBytes(SerializeUtils.serializeFilter(newFilter));
-          newReaderId = client.querySingleSeries(request);
+          newFilter = TimeFilter.gt(timestamp);
         }
-      } catch (TException e) {
-        // the connection may be broken, close it to avoid it being reused
-        client.getInputProtocol().getTransport().close();
-        throw e;
+        request.setTimeFilterBytes(SerializeUtils.serializeFilter(newFilter));
+        newReaderId = client.querySingleSeries(request);
       }
       return newReaderId;
+    } catch (IOException | TException e) {
+      // the connection may be broken, close it to avoid it being reused
+      if (client != null) {
+        client.close();
+      }
+      throw e;
+    } finally {
+      if (client != null) {
+        client.returnSelf();
+      }
     }
   }
 
@@ -205,13 +205,13 @@ public class DataSourceInfo {
   AsyncDataClient getCurAsyncClient(int timeout) throws IOException {
     return isNoClient
         ? null
-        : metaGroupMember.getClientProvider().getAsyncDataClient(this.curSource, timeout);
+        : ClusterIoTDB.getInstance().getAsyncDataClient(this.curSource, timeout);
   }
 
-  SyncDataClient getCurSyncClient(int timeout) throws TException {
+  SyncDataClient getCurSyncClient(int timeout) throws IOException {
     return isNoClient
         ? null
-        : metaGroupMember.getClientProvider().getSyncDataClient(this.curSource, timeout);
+        : ClusterIoTDB.getInstance().getSyncDataClient(this.curSource, timeout);
   }
 
   public boolean isNoData() {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/RemoteSeriesReaderByTimestamp.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/RemoteSeriesReaderByTimestamp.java
index e8b8d0e..1db57a9 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/RemoteSeriesReaderByTimestamp.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/RemoteSeriesReaderByTimestamp.java
@@ -20,10 +20,9 @@
 package org.apache.iotdb.cluster.query.reader;
 
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
-import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.handlers.caller.GenericHandler;
-import org.apache.iotdb.cluster.utils.ClientUtils;
 import org.apache.iotdb.db.query.reader.series.IReaderByTimestamp;
 import org.apache.iotdb.db.utils.SerializeUtils;
 
@@ -77,10 +76,10 @@ public class RemoteSeriesReaderByTimestamp implements IReaderByTimestamp {
       fetchResult.set(null);
       try {
         sourceInfo
-            .getCurAsyncClient(RaftServer.getReadOperationTimeoutMS())
+            .getCurAsyncClient(ClusterConstant.getReadOperationTimeoutMS())
             .fetchSingleSeriesByTimestamps(
                 sourceInfo.getHeader(), sourceInfo.getReaderId(), timestampList, handler);
-        fetchResult.wait(RaftServer.getReadOperationTimeoutMS());
+        fetchResult.wait(ClusterConstant.getReadOperationTimeoutMS());
       } catch (TException e) {
         // try other node
         if (!sourceInfo.switchNode(true, timestamps[0])) {
@@ -104,13 +103,11 @@ public class RemoteSeriesReaderByTimestamp implements IReaderByTimestamp {
       timestampList.add(timestamps[i]);
     }
     try {
-      curSyncClient = sourceInfo.getCurSyncClient(RaftServer.getReadOperationTimeoutMS());
+      curSyncClient = sourceInfo.getCurSyncClient(ClusterConstant.getReadOperationTimeoutMS());
       return curSyncClient.fetchSingleSeriesByTimestamps(
           sourceInfo.getHeader(), sourceInfo.getReaderId(), timestampList);
     } catch (TException e) {
-      if (curSyncClient != null) {
-        curSyncClient.getInputProtocol().getTransport().close();
-      }
+      curSyncClient.close();
       // try other node
       if (!sourceInfo.switchNode(true, timestamps[0])) {
         return null;
@@ -118,7 +115,7 @@ public class RemoteSeriesReaderByTimestamp implements IReaderByTimestamp {
       return fetchResultSync(timestamps, length);
     } finally {
       if (curSyncClient != null) {
-        ClientUtils.putBackSyncClient(curSyncClient);
+        curSyncClient.returnSelf();
       }
     }
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/RemoteSimpleSeriesReader.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/RemoteSimpleSeriesReader.java
index b85b53b..8ad9a41 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/RemoteSimpleSeriesReader.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/RemoteSimpleSeriesReader.java
@@ -20,10 +20,9 @@
 package org.apache.iotdb.cluster.query.reader;
 
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
-import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.handlers.caller.GenericHandler;
-import org.apache.iotdb.cluster.utils.ClientUtils;
 import org.apache.iotdb.db.utils.SerializeUtils;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
 import org.apache.iotdb.tsfile.read.common.BatchData;
@@ -125,9 +124,9 @@ public class RemoteSimpleSeriesReader implements IPointReader {
       fetchResult.set(null);
       try {
         sourceInfo
-            .getCurAsyncClient(RaftServer.getReadOperationTimeoutMS())
+            .getCurAsyncClient(ClusterConstant.getReadOperationTimeoutMS())
             .fetchSingleSeries(sourceInfo.getHeader(), sourceInfo.getReaderId(), handler);
-        fetchResult.wait(RaftServer.getReadOperationTimeoutMS());
+        fetchResult.wait(ClusterConstant.getReadOperationTimeoutMS());
       } catch (TException e) {
         // try other node
         if (!sourceInfo.switchNode(false, lastTimestamp)) {
@@ -146,12 +145,10 @@ public class RemoteSimpleSeriesReader implements IPointReader {
   private ByteBuffer fetchResultSync() throws IOException {
     SyncDataClient curSyncClient = null;
     try {
-      curSyncClient = sourceInfo.getCurSyncClient(RaftServer.getReadOperationTimeoutMS());
+      curSyncClient = sourceInfo.getCurSyncClient(ClusterConstant.getReadOperationTimeoutMS());
       return curSyncClient.fetchSingleSeries(sourceInfo.getHeader(), sourceInfo.getReaderId());
     } catch (TException e) {
-      if (curSyncClient != null) {
-        curSyncClient.getInputProtocol().getTransport().close();
-      }
+      curSyncClient.close();
       // try other node
       if (!sourceInfo.switchNode(false, lastTimestamp)) {
         return null;
@@ -159,7 +156,7 @@ public class RemoteSimpleSeriesReader implements IPointReader {
       return fetchResultSync();
     } finally {
       if (curSyncClient != null) {
-        ClientUtils.putBackSyncClient(curSyncClient);
+        curSyncClient.returnSelf();
       }
     }
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/MultDataSourceInfo.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/MultDataSourceInfo.java
index 73dee46..e849f04 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/MultDataSourceInfo.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/MultDataSourceInfo.java
@@ -19,17 +19,17 @@
 
 package org.apache.iotdb.cluster.query.reader.mult;
 
+import org.apache.iotdb.cluster.ClusterIoTDB;
 import org.apache.iotdb.cluster.client.async.AsyncDataClient;
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.query.RemoteQueryContext;
 import org.apache.iotdb.cluster.rpc.thrift.MultSeriesQueryRequest;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
-import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.handlers.caller.GenericHandler;
-import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.utils.SerializeUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -61,7 +61,6 @@ public class MultDataSourceInfo {
   private List<TSDataType> dataTypes;
   private MultSeriesQueryRequest request;
   private RemoteQueryContext context;
-  private MetaGroupMember metaGroupMember;
   private List<Node> nodes;
   private int curPos;
   private boolean isNoData = false;
@@ -73,7 +72,6 @@ public class MultDataSourceInfo {
       List<TSDataType> dataTypes,
       MultSeriesQueryRequest request,
       RemoteQueryContext context,
-      MetaGroupMember metaGroupMember,
       List<Node> nodes) {
     this.readerId = -1;
     this.partitionGroup = group;
@@ -81,7 +79,6 @@ public class MultDataSourceInfo {
     this.dataTypes = dataTypes;
     this.request = request;
     this.context = context;
-    this.metaGroupMember = metaGroupMember;
     this.nodes = nodes;
     // set to the last node so after nextDataClient() is called it will scan from the first node
     this.curPos = nodes.size() - 1;
@@ -117,11 +114,11 @@ public class MultDataSourceInfo {
             return false;
           }
         }
-      } catch (TException | IOException e) {
-        logger.error("Cannot query {} from {}", this.request.path, node, e);
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         logger.error("Cannot query {} from {}", this.request.path, node, e);
+      } catch (Exception e) {
+        logger.error("Cannot query {} from {}", this.request.path, node, e);
       }
       nextNodePos = (nextNodePos + 1) % this.nodes.size();
       if (nextNodePos == this.curPos) {
@@ -148,11 +145,10 @@ public class MultDataSourceInfo {
   }
 
   private Long applyForReaderIdAsync(Node node, long timestamp)
-      throws TException, InterruptedException, IOException {
+      throws IOException, TException, InterruptedException {
     AsyncDataClient client =
-        this.metaGroupMember
-            .getClientProvider()
-            .getAsyncDataClient(node, RaftServer.getReadOperationTimeoutMS());
+        ClusterIoTDB.getInstance()
+            .getAsyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
     AtomicReference<Long> result = new AtomicReference<>();
     GenericHandler<Long> handler = new GenericHandler<>(client.getNode(), result);
     Filter newFilter;
@@ -167,20 +163,20 @@ public class MultDataSourceInfo {
     client.queryMultSeries(request, handler);
     synchronized (result) {
       if (result.get() == null && handler.getException() == null) {
-        result.wait(RaftServer.getReadOperationTimeoutMS());
+        result.wait(ClusterConstant.getReadOperationTimeoutMS());
       }
     }
     return result.get();
   }
 
-  private Long applyForReaderIdSync(Node node, long timestamp) throws TException {
-
-    Long newReaderId;
-    try (SyncDataClient client =
-        this.metaGroupMember
-            .getClientProvider()
-            .getSyncDataClient(node, RaftServer.getReadOperationTimeoutMS())) {
+  private Long applyForReaderIdSync(Node node, long timestamp) throws IOException, TException {
 
+    long newReaderId;
+    SyncDataClient client = null;
+    try {
+      client =
+          ClusterIoTDB.getInstance()
+              .getSyncDataClient(node, ClusterConstant.getReadOperationTimeoutMS());
       Filter newFilter;
       // add timestamp to as a timeFilter to skip the data which has been read
       if (request.isSetTimeFilterBytes()) {
@@ -190,14 +186,16 @@ public class MultDataSourceInfo {
         newFilter = TimeFilter.gt(timestamp);
       }
       request.setTimeFilterBytes(SerializeUtils.serializeFilter(newFilter));
-      try {
-        newReaderId = client.queryMultSeries(request);
-      } catch (TException e) {
-        // the connection may be broken, close it to avoid it being reused
-        client.getInputProtocol().getTransport().close();
-        throw e;
-      }
+      newReaderId = client.queryMultSeries(request);
       return newReaderId;
+    } catch (TException e) {
+      // the connection may be broken, close it to avoid it being reused
+      client.close();
+      throw e;
+    } finally {
+      if (client != null) {
+        client.returnSelf();
+      }
     }
   }
 
@@ -216,13 +214,13 @@ public class MultDataSourceInfo {
   AsyncDataClient getCurAsyncClient(int timeout) throws IOException {
     return isNoClient
         ? null
-        : metaGroupMember.getClientProvider().getAsyncDataClient(this.curSource, timeout);
+        : ClusterIoTDB.getInstance().getAsyncDataClient(this.curSource, timeout);
   }
 
-  SyncDataClient getCurSyncClient(int timeout) throws TException {
+  SyncDataClient getCurSyncClient(int timeout) throws IOException {
     return isNoClient
         ? null
-        : metaGroupMember.getClientProvider().getSyncDataClient(this.curSource, timeout);
+        : ClusterIoTDB.getInstance().getSyncDataClient(this.curSource, timeout);
   }
 
   public boolean isNoData() {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/RemoteMultSeriesReader.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/RemoteMultSeriesReader.java
index e80fc1e..1314341 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/RemoteMultSeriesReader.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/mult/RemoteMultSeriesReader.java
@@ -20,8 +20,8 @@
 package org.apache.iotdb.cluster.query.reader.mult;
 
 import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
-import org.apache.iotdb.cluster.server.RaftServer;
 import org.apache.iotdb.cluster.server.handlers.caller.GenericHandler;
 import org.apache.iotdb.db.utils.SerializeUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -91,10 +91,7 @@ public class RemoteMultSeriesReader extends AbstractMultPointReader {
 
   private boolean checkPathBatchData(String fullPath) {
     BatchData batchData = cachedBatchs.get(fullPath).peek();
-    if (batchData != null && !batchData.isEmpty()) {
-      return true;
-    }
-    return false;
+    return batchData != null && !batchData.isEmpty();
   }
 
   @Override
@@ -169,9 +166,9 @@ public class RemoteMultSeriesReader extends AbstractMultPointReader {
       fetchResult.set(null);
       try {
         sourceInfo
-            .getCurAsyncClient(RaftServer.getReadOperationTimeoutMS())
+            .getCurAsyncClient(ClusterConstant.getReadOperationTimeoutMS())
             .fetchMultSeries(sourceInfo.getHeader(), sourceInfo.getReaderId(), paths, handler);
-        fetchResult.wait(RaftServer.getReadOperationTimeoutMS());
+        fetchResult.wait(ClusterConstant.getReadOperationTimeoutMS());
       } catch (TException | InterruptedException e) {
         logger.error("Failed to fetch result async, connect to {}", sourceInfo, e);
         return null;
@@ -181,20 +178,18 @@ public class RemoteMultSeriesReader extends AbstractMultPointReader {
   }
 
   private Map<String, ByteBuffer> fetchResultSync(List<String> paths) throws IOException {
-
-    try (SyncDataClient curSyncClient =
-        sourceInfo.getCurSyncClient(RaftServer.getReadOperationTimeoutMS()); ) {
-      try {
-        return curSyncClient.fetchMultSeries(
-            sourceInfo.getHeader(), sourceInfo.getReaderId(), paths);
-      } catch (TException e) {
-        // the connection may be broken, close it to avoid it being reused
-        curSyncClient.getInputProtocol().getTransport().close();
-        throw e;
-      }
+    SyncDataClient curSyncClient = null;
+    try {
+      curSyncClient = sourceInfo.getCurSyncClient(ClusterConstant.getReadOperationTimeoutMS());
+      return curSyncClient.fetchMultSeries(sourceInfo.getHeader(), sourceInfo.getReaderId(), paths);
     } catch (TException e) {
+      curSyncClient.close();
       logger.error("Failed to fetch result sync, connect to {}", sourceInfo, e);
       return null;
+    } finally {
+      if (curSyncClient != null) {
+        curSyncClient.returnSelf();
+      }
     }
   }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/ClientServer.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/ClientServer.java
deleted file mode 100644
index 86e5fd2..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/ClientServer.java
+++ /dev/null
@@ -1,311 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.cluster.server;
-
-import org.apache.iotdb.cluster.client.async.AsyncDataClient;
-import org.apache.iotdb.cluster.client.sync.SyncDataClient;
-import org.apache.iotdb.cluster.config.ClusterConfig;
-import org.apache.iotdb.cluster.config.ClusterDescriptor;
-import org.apache.iotdb.cluster.coordinator.Coordinator;
-import org.apache.iotdb.cluster.query.ClusterPlanExecutor;
-import org.apache.iotdb.cluster.query.ClusterPlanner;
-import org.apache.iotdb.cluster.query.RemoteQueryContext;
-import org.apache.iotdb.cluster.rpc.thrift.Node;
-import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
-import org.apache.iotdb.cluster.server.handlers.caller.GenericHandler;
-import org.apache.iotdb.cluster.server.member.MetaGroupMember;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.sys.FlushPlan;
-import org.apache.iotdb.db.qp.physical.sys.SetSystemModePlan;
-import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.service.TSServiceImpl;
-import org.apache.iotdb.db.utils.CommonUtils;
-import org.apache.iotdb.rpc.RpcTransportFactory;
-import org.apache.iotdb.rpc.RpcUtils;
-import org.apache.iotdb.rpc.TSStatusCode;
-import org.apache.iotdb.service.rpc.thrift.TSIService.Processor;
-import org.apache.iotdb.service.rpc.thrift.TSStatus;
-
-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.protocol.TProtocolFactory;
-import org.apache.thrift.server.ServerContext;
-import org.apache.thrift.server.TServer;
-import org.apache.thrift.server.TServerEventHandler;
-import org.apache.thrift.server.TThreadPoolServer;
-import org.apache.thrift.transport.TServerSocket;
-import org.apache.thrift.transport.TServerTransport;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * ClientServer is the cluster version of TSServiceImpl, which is responsible for the processing of
- * the user requests (sqls and session api). It inherits the basic procedures from TSServiceImpl,
- * but redirect the queries of data and metadata to a MetaGroupMember of the local node.
- */
-public class ClientServer extends TSServiceImpl {
-
-  private static final Logger logger = LoggerFactory.getLogger(ClientServer.class);
-  /**
-   * The Coordinator of the local node. Through this node ClientServer queries data and meta from
-   * the cluster and performs data manipulations to the cluster.
-   */
-  private Coordinator coordinator;
-
-  public void setCoordinator(Coordinator coordinator) {
-    this.coordinator = coordinator;
-  }
-
-  /** The single thread pool that runs poolServer to unblock the main thread. */
-  private ExecutorService serverService;
-
-  /**
-   * Using the poolServer, ClientServer will listen to a socket to accept thrift requests like an
-   * HttpServer.
-   */
-  private TServer poolServer;
-
-  /** The socket poolServer will listen to. Async service requires nonblocking socket */
-  private TServerTransport serverTransport;
-
-  /**
-   * queryId -> queryContext map. When a query ends either normally or accidentally, the resources
-   * used by the query can be found in the context and then released.
-   */
-  private Map<Long, RemoteQueryContext> queryContextMap = new ConcurrentHashMap<>();
-
-  public ClientServer(MetaGroupMember metaGroupMember) throws QueryProcessException {
-    super();
-    this.processor = new ClusterPlanner();
-    this.executor = new ClusterPlanExecutor(metaGroupMember);
-  }
-
-  /**
-   * Create a thrift server to listen to the client port and accept requests from clients. This
-   * server is run in a separate thread. Calling the method twice does not induce side effects.
-   *
-   * @throws TTransportException
-   */
-  public void start() throws TTransportException {
-    if (serverService != null) {
-      return;
-    }
-
-    serverService = Executors.newSingleThreadExecutor(r -> new Thread(r, "ClusterClientServer"));
-    ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
-
-    // this defines how thrift parse the requests bytes to a request
-    TProtocolFactory protocolFactory;
-    if (IoTDBDescriptor.getInstance().getConfig().isRpcThriftCompressionEnable()) {
-      protocolFactory = new TCompactProtocol.Factory();
-    } else {
-      protocolFactory = new TBinaryProtocol.Factory();
-    }
-    serverTransport =
-        new TServerSocket(
-            new InetSocketAddress(
-                IoTDBDescriptor.getInstance().getConfig().getRpcAddress(),
-                config.getClusterRpcPort()));
-    // async service also requires nonblocking server, and HsHaServer is basically more efficient a
-    // nonblocking server
-    int maxConcurrentClientNum =
-        Math.max(CommonUtils.getCpuCores(), config.getMaxConcurrentClientNum());
-    TThreadPoolServer.Args poolArgs =
-        new TThreadPoolServer.Args(serverTransport)
-            .maxWorkerThreads(maxConcurrentClientNum)
-            .minWorkerThreads(CommonUtils.getCpuCores());
-    poolArgs.executorService(
-        new ThreadPoolExecutor(
-            poolArgs.minWorkerThreads,
-            poolArgs.maxWorkerThreads,
-            poolArgs.stopTimeoutVal,
-            poolArgs.stopTimeoutUnit,
-            new SynchronousQueue<>(),
-            new ThreadFactory() {
-              private AtomicLong threadIndex = new AtomicLong(0);
-
-              @Override
-              public Thread newThread(Runnable r) {
-                return new Thread(r, "ClusterClient-" + threadIndex.incrementAndGet());
-              }
-            }));
-    // ClientServer will do the following processing when the HsHaServer has parsed a request
-    poolArgs.processor(new Processor<>(this));
-    poolArgs.protocolFactory(protocolFactory);
-    // nonblocking server requests FramedTransport
-    poolArgs.transportFactory(RpcTransportFactory.INSTANCE);
-
-    poolServer = new TThreadPoolServer(poolArgs);
-    // mainly for handling client exit events
-    poolServer.setServerEventHandler(new EventHandler());
-
-    serverService.submit(() -> poolServer.serve());
-    logger.info("Client service is set up");
-  }
-
-  /**
-   * Stop the thrift server, close the socket and shutdown the thread pool. Calling the method twice
-   * does not induce side effects.
-   */
-  public void stop() {
-    if (serverService == null) {
-      return;
-    }
-
-    poolServer.stop();
-    serverService.shutdownNow();
-    serverTransport.close();
-  }
-
-  /**
-   * Redirect the plan to the local Coordinator so that it will be processed cluster-wide.
-   *
-   * @param plan
-   * @return
-   */
-  @Override
-  protected TSStatus executeNonQueryPlan(PhysicalPlan plan) {
-    try {
-      plan.checkIntegrity();
-      if (!(plan instanceof SetSystemModePlan)
-          && !(plan instanceof FlushPlan)
-          && IoTDBDescriptor.getInstance().getConfig().isReadOnly()) {
-        throw new QueryProcessException(
-            "Current system mode is read-only, does not support non-query operation");
-      }
-    } catch (QueryProcessException e) {
-      logger.warn("Illegal plan detected๏ผš {}", plan);
-      return RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage());
-    }
-    return coordinator.executeNonQueryPlan(plan);
-  }
-
-  /**
-   * EventHandler handles the preprocess and postprocess of the thrift requests, but it currently
-   * only release resources when a client disconnects.
-   */
-  class EventHandler implements TServerEventHandler {
-
-    @Override
-    public void preServe() {
-      // do nothing
-    }
-
-    @Override
-    public ServerContext createContext(TProtocol input, TProtocol output) {
-      return null;
-    }
-
-    @Override
-    public void deleteContext(ServerContext serverContext, TProtocol input, TProtocol output) {
-      ClientServer.this.handleClientExit();
-    }
-
-    @Override
-    public void processContext(
-        ServerContext serverContext, TTransport inputTransport, TTransport outputTransport) {
-      // do nothing
-    }
-  }
-
-  /**
-   * Generate and cache a QueryContext using "queryId". In the distributed version, the QueryContext
-   * is a RemoteQueryContext.
-   *
-   * @param queryId
-   * @return a RemoteQueryContext using queryId
-   */
-  @Override
-  protected QueryContext genQueryContext(
-      long queryId, boolean debug, long startTime, String statement, long timeout) {
-    RemoteQueryContext context =
-        new RemoteQueryContext(queryId, debug, startTime, statement, timeout);
-    queryContextMap.put(queryId, context);
-    return context;
-  }
-
-  /**
-   * Release the local and remote resources used by a query.
-   *
-   * @param queryId
-   * @throws StorageEngineException
-   */
-  @Override
-  protected void releaseQueryResource(long queryId) throws StorageEngineException {
-    // release resources locally
-    super.releaseQueryResource(queryId);
-    // release resources remotely
-    RemoteQueryContext context = queryContextMap.remove(queryId);
-    if (context != null) {
-      // release the resources in every queried node
-      for (Entry<RaftNode, Set<Node>> headerEntry : context.getQueriedNodesMap().entrySet()) {
-        RaftNode header = headerEntry.getKey();
-        Set<Node> queriedNodes = headerEntry.getValue();
-
-        for (Node queriedNode : queriedNodes) {
-          GenericHandler<Void> handler = new GenericHandler<>(queriedNode, new AtomicReference<>());
-          try {
-            if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-              AsyncDataClient client =
-                  coordinator.getAsyncDataClient(
-                      queriedNode, RaftServer.getReadOperationTimeoutMS());
-              client.endQuery(header, coordinator.getThisNode(), queryId, handler);
-            } else {
-              try (SyncDataClient syncDataClient =
-                  coordinator.getSyncDataClient(
-                      queriedNode, RaftServer.getReadOperationTimeoutMS())) {
-                try {
-                  syncDataClient.endQuery(header, coordinator.getThisNode(), queryId);
-                } catch (TException e) {
-                  // the connection may be broken, close it to avoid it being reused
-                  syncDataClient.getInputProtocol().getTransport().close();
-                  throw e;
-                }
-              }
-            }
-          } catch (IOException | TException e) {
-            logger.error("Cannot end query {} in {}", queryId, queriedNode);
-          }
-        }
-      }
-    }
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/service/RPCService.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/ClusterRPCService.java
similarity index 62%
copy from server/src/main/java/org/apache/iotdb/db/service/RPCService.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/server/ClusterRPCService.java
index 5bbddec..fa2a2c9 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/RPCService.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/ClusterRPCService.java
@@ -16,45 +16,47 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.service;
 
+package org.apache.iotdb.cluster.server;
+
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.db.concurrent.ThreadName;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.runtime.RPCServiceException;
+import org.apache.iotdb.db.service.RPCServiceThriftHandler;
+import org.apache.iotdb.db.service.ServiceType;
 import org.apache.iotdb.db.service.thrift.ThriftService;
 import org.apache.iotdb.db.service.thrift.ThriftServiceThread;
 import org.apache.iotdb.service.rpc.thrift.TSIService.Processor;
 
-/** A service to handle jdbc request from client. */
-public class RPCService extends ThriftService implements RPCServiceMBean {
+public class ClusterRPCService extends ThriftService implements ClusterRPCServiceMBean {
 
-  private TSServiceImpl impl;
+  private ClusterTSServiceImpl impl;
 
-  private RPCService() {}
+  private ClusterRPCService() {}
 
-  public static RPCService getInstance() {
-    return RPCServiceHolder.INSTANCE;
+  @Override
+  public ThriftService getImplementation() {
+    return ClusterRPCServiceHolder.INSTANCE;
   }
 
   @Override
-  public int getRPCPort() {
-    IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
-    return config.getRpcPort();
+  public ServiceType getID() {
+    return ServiceType.CLUSTER_RPC_SERVICE;
   }
 
   @Override
-  public ThriftService getImplementation() {
-    return getInstance();
+  public void initSyncedServiceImpl(Object serviceImpl) {
+    impl = (ClusterTSServiceImpl) serviceImpl;
+    super.initSyncedServiceImpl(serviceImpl);
   }
 
   @Override
-  public void initTProcessor()
-      throws ClassNotFoundException, IllegalAccessException, InstantiationException {
-    impl =
-        (TSServiceImpl)
-            Class.forName(IoTDBDescriptor.getInstance().getConfig().getRpcImplClassName())
-                .newInstance();
+  public void initTProcessor() throws InstantiationException {
+    if (impl == null) {
+      throw new InstantiationException("ClusterTSServiceImpl is null");
+    }
     processor = new Processor<>(impl);
   }
 
@@ -66,9 +68,9 @@ public class RPCService extends ThriftService implements RPCServiceMBean {
           new ThriftServiceThread(
               processor,
               getID().getName(),
-              ThreadName.RPC_CLIENT.getName(),
-              config.getRpcAddress(),
-              config.getRpcPort(),
+              ThreadName.CLUSTER_RPC_CLIENT.getName(),
+              getBindIP(),
+              getBindPort(),
               config.getRpcMaxConcurrentClientNum(),
               config.getThriftServerAwaitTimeForStopService(),
               new RPCServiceThriftHandler(impl),
@@ -76,7 +78,7 @@ public class RPCService extends ThriftService implements RPCServiceMBean {
     } catch (RPCServiceException e) {
       throw new IllegalAccessException(e.getMessage());
     }
-    thriftServiceThread.setName(ThreadName.RPC_SERVICE.getName());
+    thriftServiceThread.setName(ThreadName.CLUSTER_RPC_SERVICE.getName());
   }
 
   @Override
@@ -86,18 +88,22 @@ public class RPCService extends ThriftService implements RPCServiceMBean {
 
   @Override
   public int getBindPort() {
-    return IoTDBDescriptor.getInstance().getConfig().getRpcPort();
+    return ClusterDescriptor.getInstance().getConfig().getClusterRpcPort();
   }
 
   @Override
-  public ServiceType getID() {
-    return ServiceType.RPC_SERVICE;
+  public int getRPCPort() {
+    return getBindPort();
+  }
+
+  public static ClusterRPCService getInstance() {
+    return ClusterRPCServiceHolder.INSTANCE;
   }
 
-  private static class RPCServiceHolder {
+  private static class ClusterRPCServiceHolder {
 
-    private static final RPCService INSTANCE = new RPCService();
+    private static final ClusterRPCService INSTANCE = new ClusterRPCService();
 
-    private RPCServiceHolder() {}
+    private ClusterRPCServiceHolder() {}
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/ClusterRPCServiceMBean.java
similarity index 56%
copy from cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/server/ClusterRPCServiceMBean.java
index 6bfe9003..3717bcd 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/ClusterRPCServiceMBean.java
@@ -17,24 +17,19 @@
  * under the License.
  */
 
-package org.apache.iotdb.cluster.client.sync;
+package org.apache.iotdb.cluster.server;
 
-import org.apache.iotdb.cluster.rpc.thrift.Node;
-import org.apache.iotdb.cluster.rpc.thrift.RaftService;
+import org.apache.iotdb.db.exception.StartupException;
 
-import org.apache.thrift.transport.TTransportException;
+public interface ClusterRPCServiceMBean {
 
-import java.io.IOException;
+  String getRPCServiceStatus();
 
-public interface SyncClientFactory {
+  int getRPCPort();
 
-  /**
-   * 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;
+  void startService() throws StartupException;
+
+  void restartService() throws StartupException;
+
+  void stopService();
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/ClusterTSServiceImpl.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/ClusterTSServiceImpl.java
new file mode 100644
index 0000000..fc52b6f
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/ClusterTSServiceImpl.java
@@ -0,0 +1,172 @@
+/*
+ * 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.server;
+
+import org.apache.iotdb.cluster.ClusterIoTDB;
+import org.apache.iotdb.cluster.client.async.AsyncDataClient;
+import org.apache.iotdb.cluster.client.sync.SyncDataClient;
+import org.apache.iotdb.cluster.config.ClusterConstant;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.coordinator.Coordinator;
+import org.apache.iotdb.cluster.query.ClusterPlanExecutor;
+import org.apache.iotdb.cluster.query.RemoteQueryContext;
+import org.apache.iotdb.cluster.rpc.thrift.Node;
+import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
+import org.apache.iotdb.cluster.server.handlers.caller.GenericHandler;
+import org.apache.iotdb.cluster.server.member.MetaGroupMember;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.FlushPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetSystemModePlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.service.TSServiceImpl;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * ClusterTSServiceImpl is the cluster version of TSServiceImpl, which is responsible for the
+ * processing of the user requests (sqls and session api). It inherits the basic procedures from
+ * TSServiceImpl, but redirect the queries of data and metadata to a MetaGroupMember of the local
+ * node.
+ */
+public class ClusterTSServiceImpl extends TSServiceImpl {
+
+  private static final Logger logger = LoggerFactory.getLogger(ClusterTSServiceImpl.class);
+  /**
+   * The Coordinator of the local node. Through this node queries data and meta from the cluster and
+   * performs data manipulations to the cluster.
+   */
+  private Coordinator coordinator;
+
+  /**
+   * queryId -> queryContext map. When a query ends either normally or accidentally, the resources
+   * used by the query can be found in the context and then released.
+   */
+  private Map<Long, RemoteQueryContext> queryContextMap = new ConcurrentHashMap<>();
+
+  public ClusterTSServiceImpl() throws QueryProcessException {}
+
+  public void setExecutor(MetaGroupMember metaGroupMember) throws QueryProcessException {
+    executor = new ClusterPlanExecutor(metaGroupMember);
+  }
+
+  public void setCoordinator(Coordinator coordinator) {
+    this.coordinator = coordinator;
+  }
+
+  /** Redirect the plan to the local Coordinator so that it will be processed cluster-wide. */
+  @Override
+  protected TSStatus executeNonQueryPlan(PhysicalPlan plan) {
+    try {
+      plan.checkIntegrity();
+      if (!(plan instanceof SetSystemModePlan)
+          && !(plan instanceof FlushPlan)
+          && IoTDBDescriptor.getInstance().getConfig().isReadOnly()) {
+        throw new QueryProcessException(
+            "Current system mode is read-only, does not support non-query operation");
+      }
+    } catch (QueryProcessException e) {
+      logger.warn("Illegal plan detected๏ผš {}", plan);
+      return RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+
+    return coordinator.executeNonQueryPlan(plan);
+  }
+
+  /**
+   * Generate and cache a QueryContext using "queryId". In the distributed version, the QueryContext
+   * is a RemoteQueryContext.
+   *
+   * @return a RemoteQueryContext using queryId
+   */
+  @Override
+  protected QueryContext genQueryContext(
+      long queryId, boolean debug, long startTime, String statement, long timeout) {
+    RemoteQueryContext context =
+        new RemoteQueryContext(queryId, debug, startTime, statement, timeout);
+    queryContextMap.put(queryId, context);
+    return context;
+  }
+
+  /** Release the local and remote resources used by a query. */
+  @Override
+  protected void releaseQueryResource(long queryId) throws StorageEngineException {
+    // release resources locally
+    super.releaseQueryResource(queryId);
+    // release resources remotely
+    RemoteQueryContext context = queryContextMap.remove(queryId);
+    if (context != null) {
+      // release the resources in every queried node
+      for (Entry<RaftNode, Set<Node>> headerEntry : context.getQueriedNodesMap().entrySet()) {
+        RaftNode header = headerEntry.getKey();
+        Set<Node> queriedNodes = headerEntry.getValue();
+        for (Node queriedNode : queriedNodes) {
+          releaseQueryResourceForOneNode(queryId, header, queriedNode);
+        }
+      }
+    }
+  }
+
+  protected void releaseQueryResourceForOneNode(long queryId, RaftNode header, Node queriedNode) {
+    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
+      GenericHandler<Void> handler = new GenericHandler<>(queriedNode, new AtomicReference<>());
+      try {
+        AsyncDataClient client =
+            ClusterIoTDB.getInstance()
+                .getAsyncDataClient(queriedNode, ClusterConstant.getReadOperationTimeoutMS());
+        client.endQuery(header, coordinator.getThisNode(), queryId, handler);
+      } catch (IOException | TException e) {
+        logger.error("Cannot end query {} in {}", queryId, queriedNode);
+      }
+    } else {
+      SyncDataClient syncDataClient = null;
+      try {
+        syncDataClient =
+            ClusterIoTDB.getInstance()
+                .getSyncDataClient(queriedNode, ClusterConstant.getReadOperationTimeoutMS());
+        syncDataClient.endQuery(header, coordinator.getThisNode(), queryId);
+      } catch (IOException | TException e) {
+        // the connection may be broken, close it to avoid it being reused
+        if (syncDataClient != null) {
+          syncDataClient.close();
+        }
+        logger.error("Cannot end query {} in {}", queryId, queriedNode);
+      } finally {
+        if (syncDataClient != null) {
+          syncDataClient.returnSelf();
+        }
+      }
+    }
+  }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/DataClusterServer.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/DataClusterServer.java
deleted file mode 100644
index 35a8fe9..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/DataClusterServer.java
+++ /dev/null
@@ -1,1109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.cluster.server;
-
-import org.apache.iotdb.cluster.config.ClusterDescriptor;
-import org.apache.iotdb.cluster.exception.CheckConsistencyException;
-import org.apache.iotdb.cluster.exception.NoHeaderNodeException;
-import org.apache.iotdb.cluster.exception.NotInSameGroupException;
-import org.apache.iotdb.cluster.exception.PartitionTableUnavailableException;
-import org.apache.iotdb.cluster.log.logtypes.AddNodeLog;
-import org.apache.iotdb.cluster.log.logtypes.RemoveNodeLog;
-import org.apache.iotdb.cluster.partition.NodeAdditionResult;
-import org.apache.iotdb.cluster.partition.NodeRemovalResult;
-import org.apache.iotdb.cluster.partition.PartitionGroup;
-import org.apache.iotdb.cluster.partition.PartitionTable;
-import org.apache.iotdb.cluster.partition.slot.SlotPartitionTable;
-import org.apache.iotdb.cluster.rpc.thrift.AppendEntriesRequest;
-import org.apache.iotdb.cluster.rpc.thrift.AppendEntryRequest;
-import org.apache.iotdb.cluster.rpc.thrift.ElectionRequest;
-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.HeartBeatRequest;
-import org.apache.iotdb.cluster.rpc.thrift.HeartBeatResponse;
-import org.apache.iotdb.cluster.rpc.thrift.LastQueryRequest;
-import org.apache.iotdb.cluster.rpc.thrift.MultSeriesQueryRequest;
-import org.apache.iotdb.cluster.rpc.thrift.Node;
-import org.apache.iotdb.cluster.rpc.thrift.PreviousFillRequest;
-import org.apache.iotdb.cluster.rpc.thrift.PullSchemaRequest;
-import org.apache.iotdb.cluster.rpc.thrift.PullSchemaResp;
-import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotRequest;
-import org.apache.iotdb.cluster.rpc.thrift.PullSnapshotResp;
-import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
-import org.apache.iotdb.cluster.rpc.thrift.RequestCommitIndexResponse;
-import org.apache.iotdb.cluster.rpc.thrift.SendSnapshotRequest;
-import org.apache.iotdb.cluster.rpc.thrift.SingleSeriesQueryRequest;
-import org.apache.iotdb.cluster.rpc.thrift.TSDataService;
-import org.apache.iotdb.cluster.rpc.thrift.TSDataService.AsyncProcessor;
-import org.apache.iotdb.cluster.rpc.thrift.TSDataService.Processor;
-import org.apache.iotdb.cluster.server.member.DataGroupMember;
-import org.apache.iotdb.cluster.server.member.MetaGroupMember;
-import org.apache.iotdb.cluster.server.monitor.NodeReport.DataMemberReport;
-import org.apache.iotdb.cluster.server.service.DataAsyncService;
-import org.apache.iotdb.cluster.server.service.DataSyncService;
-import org.apache.iotdb.cluster.utils.IOUtils;
-import org.apache.iotdb.service.rpc.thrift.TSStatus;
-
-import org.apache.thrift.TException;
-import org.apache.thrift.TProcessor;
-import org.apache.thrift.async.AsyncMethodCallback;
-import org.apache.thrift.transport.TNonblockingServerSocket;
-import org.apache.thrift.transport.TServerSocket;
-import org.apache.thrift.transport.TServerTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-public class DataClusterServer extends RaftServer
-    implements TSDataService.AsyncIface, TSDataService.Iface {
-
-  private static final Logger logger = LoggerFactory.getLogger(DataClusterServer.class);
-
-  // key: the header of a data group, value: the member representing this node in this group and
-  // it is currently at service
-  private Map<RaftNode, DataGroupMember> headerGroupMap = new ConcurrentHashMap<>();
-  private Map<RaftNode, DataAsyncService> asyncServiceMap = new ConcurrentHashMap<>();
-  private Map<RaftNode, DataSyncService> syncServiceMap = new ConcurrentHashMap<>();
-  // key: the header of a data group, value: the member representing this node in this group but
-  // it is out of service because another node has joined the group and expelled this node, or
-  // the node itself is removed, but it is still stored to provide snapshot for other nodes
-  private StoppedMemberManager stoppedMemberManager;
-  private PartitionTable partitionTable;
-  private DataGroupMember.Factory dataMemberFactory;
-  private MetaGroupMember metaGroupMember;
-
-  public DataClusterServer(
-      Node thisNode, DataGroupMember.Factory dataMemberFactory, MetaGroupMember metaGroupMember) {
-    super(thisNode);
-    this.dataMemberFactory = dataMemberFactory;
-    this.metaGroupMember = metaGroupMember;
-    this.stoppedMemberManager = new StoppedMemberManager(dataMemberFactory, thisNode);
-  }
-
-  @Override
-  public void stop() {
-    closeLogManagers();
-    for (DataGroupMember member : headerGroupMap.values()) {
-      member.stop();
-    }
-    super.stop();
-  }
-
-  /**
-   * Add a DataGroupMember into this server, if a member with the same header exists, the old member
-   * will be stopped and replaced by the new one.
-   *
-   * @param dataGroupMember
-   */
-  public DataGroupMember addDataGroupMember(DataGroupMember dataGroupMember, RaftNode header) {
-    synchronized (headerGroupMap) {
-      if (headerGroupMap.containsKey(header)) {
-        logger.debug("Group {} already exist.", dataGroupMember.getAllNodes());
-        return headerGroupMap.get(header);
-      }
-      stoppedMemberManager.remove(header);
-      headerGroupMap.put(header, dataGroupMember);
-
-      dataGroupMember.start();
-    }
-    logger.info("Add group {} successfully.", dataGroupMember.getName());
-    resetServiceCache(header); // avoid dead-lock
-
-    return dataGroupMember;
-  }
-
-  private void resetServiceCache(RaftNode header) {
-    asyncServiceMap.remove(header);
-    syncServiceMap.remove(header);
-  }
-
-  private <T> DataAsyncService getDataAsyncService(
-      RaftNode header, AsyncMethodCallback<T> resultHandler, Object request) {
-    return asyncServiceMap.computeIfAbsent(
-        header,
-        h -> {
-          DataGroupMember dataMember = getDataMember(header, resultHandler, request);
-          return dataMember != null ? new DataAsyncService(dataMember) : null;
-        });
-  }
-
-  private DataSyncService getDataSyncService(RaftNode header) {
-    return syncServiceMap.computeIfAbsent(
-        header,
-        h -> {
-          DataGroupMember dataMember = getDataMember(header, null, null);
-          return dataMember != null ? new DataSyncService(dataMember) : null;
-        });
-  }
-
-  /**
-   * @param header the header of the group which the local node is in
-   * @param resultHandler can be set to null if the request is an internal request
-   * @param request the toString() of this parameter should explain what the request is and it is
-   *     only used in logs for tracing
-   * @return
-   */
-  public <T> DataGroupMember getDataMember(
-      RaftNode header, AsyncMethodCallback<T> resultHandler, Object request) {
-    // if the resultHandler is not null, then the request is a external one and must be with a
-    // header
-    if (header.getNode() == null) {
-      if (resultHandler != null) {
-        resultHandler.onError(new NoHeaderNodeException());
-      }
-      return null;
-    }
-    DataGroupMember member = stoppedMemberManager.get(header);
-    if (member != null) {
-      return member;
-    }
-
-    // avoid creating two members for a header
-    Exception ex = null;
-    member = headerGroupMap.get(header);
-    if (member != null) {
-      return member;
-    }
-    logger.info("Received a request \"{}\" from unregistered header {}", request, header);
-    if (partitionTable != null) {
-      try {
-        member = createNewMember(header);
-      } catch (NotInSameGroupException | CheckConsistencyException e) {
-        ex = e;
-      }
-    } else {
-      logger.info("Partition is not ready, cannot create member");
-      ex = new PartitionTableUnavailableException(thisNode);
-    }
-    if (ex != null && resultHandler != null) {
-      resultHandler.onError(ex);
-    }
-    return member;
-  }
-
-  /**
-   * @param header
-   * @return A DataGroupMember representing this node in the data group of the header.
-   * @throws NotInSameGroupException If this node is not in the group of the header.
-   */
-  private DataGroupMember createNewMember(RaftNode header)
-      throws NotInSameGroupException, CheckConsistencyException {
-    PartitionGroup partitionGroup;
-    partitionGroup = partitionTable.getHeaderGroup(header);
-    if (partitionGroup == null || !partitionGroup.contains(thisNode)) {
-      // if the partition table is old, this node may have not been moved to the new group
-      metaGroupMember.syncLeaderWithConsistencyCheck(true);
-      partitionGroup = partitionTable.getHeaderGroup(header);
-    }
-    DataGroupMember member;
-    synchronized (headerGroupMap) {
-      member = headerGroupMap.get(header);
-      if (member != null) {
-        return member;
-      }
-      if (partitionGroup != null && partitionGroup.contains(thisNode)) {
-        // the two nodes are in the same group, create a new data member
-        member = dataMemberFactory.create(partitionGroup, thisNode);
-        headerGroupMap.put(header, member);
-        stoppedMemberManager.remove(header);
-        logger.info("Created a member for header {}, group is {}", header, partitionGroup);
-        member.start();
-      } else {
-        // the member may have been stopped after syncLeader
-        member = stoppedMemberManager.get(header);
-        if (member != null) {
-          return member;
-        }
-        logger.info(
-            "This node {} does not belong to the group {}, header {}",
-            thisNode,
-            partitionGroup,
-            header);
-        throw new NotInSameGroupException(partitionGroup, thisNode);
-      }
-    }
-    return member;
-  }
-
-  // Forward requests. Find the DataGroupMember that is in the group of the header of the
-  // request, and forward the request to it. See methods in DataGroupMember for details.
-
-  @Override
-  public void sendHeartbeat(
-      HeartBeatRequest request, AsyncMethodCallback<HeartBeatResponse> resultHandler) {
-    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    if (service != null) {
-      service.sendHeartbeat(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void startElection(ElectionRequest request, AsyncMethodCallback<Long> resultHandler) {
-    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    if (service != null) {
-      service.startElection(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void appendEntries(AppendEntriesRequest request, AsyncMethodCallback<Long> resultHandler) {
-    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    if (service != null) {
-      service.appendEntries(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void appendEntry(AppendEntryRequest request, AsyncMethodCallback<Long> resultHandler) {
-    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    if (service != null) {
-      service.appendEntry(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void sendSnapshot(SendSnapshotRequest request, AsyncMethodCallback<Void> resultHandler) {
-    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    if (service != null) {
-      service.sendSnapshot(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void pullSnapshot(
-      PullSnapshotRequest request, AsyncMethodCallback<PullSnapshotResp> resultHandler) {
-    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    if (service != null) {
-      service.pullSnapshot(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void executeNonQueryPlan(
-      ExecutNonQueryReq request, AsyncMethodCallback<TSStatus> resultHandler) {
-    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    if (service != null) {
-      service.executeNonQueryPlan(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void requestCommitIndex(
-      RaftNode header, AsyncMethodCallback<RequestCommitIndexResponse> resultHandler) {
-    DataAsyncService service = getDataAsyncService(header, resultHandler, "Request commit index");
-    if (service != null) {
-      service.requestCommitIndex(header, resultHandler);
-    }
-  }
-
-  @Override
-  public void readFile(
-      String filePath, long offset, int length, AsyncMethodCallback<ByteBuffer> resultHandler) {
-    try {
-      resultHandler.onComplete(IOUtils.readFile(filePath, offset, length));
-    } catch (IOException e) {
-      resultHandler.onError(e);
-    }
-  }
-
-  @Override
-  public void querySingleSeries(
-      SingleSeriesQueryRequest request, AsyncMethodCallback<Long> resultHandler) {
-    DataAsyncService service =
-        getDataAsyncService(
-            request.getHeader(), resultHandler, "Query series:" + request.getPath());
-    if (service != null) {
-      service.querySingleSeries(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void queryMultSeries(
-      MultSeriesQueryRequest request, AsyncMethodCallback<Long> resultHandler) throws TException {
-    DataAsyncService service =
-        getDataAsyncService(
-            request.getHeader(), resultHandler, "Query series:" + request.getPath());
-    if (service != null) {
-      service.queryMultSeries(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void fetchSingleSeries(
-      RaftNode header, long readerId, AsyncMethodCallback<ByteBuffer> resultHandler) {
-    DataAsyncService service =
-        getDataAsyncService(header, resultHandler, "Fetch reader:" + readerId);
-    if (service != null) {
-      service.fetchSingleSeries(header, readerId, resultHandler);
-    }
-  }
-
-  @Override
-  public void fetchMultSeries(
-      RaftNode header,
-      long readerId,
-      List<String> paths,
-      AsyncMethodCallback<Map<String, ByteBuffer>> resultHandler)
-      throws TException {
-    DataAsyncService service =
-        getDataAsyncService(header, resultHandler, "Fetch reader:" + readerId);
-    if (service != null) {
-      service.fetchMultSeries(header, readerId, paths, resultHandler);
-    }
-  }
-
-  @Override
-  public void getAllPaths(
-      RaftNode header,
-      List<String> paths,
-      boolean withAlias,
-      AsyncMethodCallback<GetAllPathsResult> resultHandler) {
-    DataAsyncService service = getDataAsyncService(header, resultHandler, "Find path:" + paths);
-    if (service != null) {
-      service.getAllPaths(header, paths, withAlias, resultHandler);
-    }
-  }
-
-  @Override
-  public void endQuery(
-      RaftNode header, Node thisNode, long queryId, AsyncMethodCallback<Void> resultHandler) {
-    DataAsyncService service = getDataAsyncService(header, resultHandler, "End query");
-    if (service != null) {
-      service.endQuery(header, thisNode, queryId, resultHandler);
-    }
-  }
-
-  @Override
-  public void querySingleSeriesByTimestamp(
-      SingleSeriesQueryRequest request, AsyncMethodCallback<Long> resultHandler) {
-    DataAsyncService service =
-        getDataAsyncService(
-            request.getHeader(),
-            resultHandler,
-            "Query by timestamp:"
-                + request.getQueryId()
-                + "#"
-                + request.getPath()
-                + " of "
-                + request.getRequester());
-    if (service != null) {
-      service.querySingleSeriesByTimestamp(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void fetchSingleSeriesByTimestamps(
-      RaftNode header,
-      long readerId,
-      List<Long> timestamps,
-      AsyncMethodCallback<ByteBuffer> resultHandler) {
-    DataAsyncService service =
-        getDataAsyncService(header, resultHandler, "Fetch by timestamp:" + readerId);
-    if (service != null) {
-      service.fetchSingleSeriesByTimestamps(header, readerId, timestamps, resultHandler);
-    }
-  }
-
-  @Override
-  public void pullTimeSeriesSchema(
-      PullSchemaRequest request, AsyncMethodCallback<PullSchemaResp> resultHandler) {
-    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    if (service != null) {
-      service.pullTimeSeriesSchema(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void pullMeasurementSchema(
-      PullSchemaRequest request, AsyncMethodCallback<PullSchemaResp> resultHandler) {
-    DataAsyncService service =
-        getDataAsyncService(request.getHeader(), resultHandler, "Pull measurement schema");
-    if (service != null) {
-      service.pullMeasurementSchema(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void getAllDevices(
-      RaftNode header, List<String> paths, AsyncMethodCallback<Set<String>> resultHandler) {
-    DataAsyncService service = getDataAsyncService(header, resultHandler, "Get all devices");
-    if (service != null) {
-      service.getAllDevices(header, paths, resultHandler);
-    }
-  }
-
-  @Override
-  public void getDevices(
-      RaftNode header, ByteBuffer planBinary, AsyncMethodCallback<ByteBuffer> resultHandler)
-      throws TException {
-    DataAsyncService service = getDataAsyncService(header, resultHandler, "get devices");
-    if (service != null) {
-      service.getDevices(header, planBinary, resultHandler);
-    }
-  }
-
-  @Override
-  public void getNodeList(
-      RaftNode header,
-      String path,
-      int nodeLevel,
-      AsyncMethodCallback<List<String>> resultHandler) {
-    DataAsyncService service = getDataAsyncService(header, resultHandler, "Get node list");
-    if (service != null) {
-      service.getNodeList(header, path, nodeLevel, resultHandler);
-    }
-  }
-
-  @Override
-  public void getChildNodeInNextLevel(
-      RaftNode header, String path, AsyncMethodCallback<Set<String>> resultHandler)
-      throws TException {
-    DataAsyncService service =
-        getDataAsyncService(header, resultHandler, "Get child node in next level");
-    if (service != null) {
-      service.getChildNodeInNextLevel(header, path, resultHandler);
-    }
-  }
-
-  @Override
-  public void getChildNodePathInNextLevel(
-      RaftNode header, String path, AsyncMethodCallback<Set<String>> resultHandler) {
-    DataAsyncService service =
-        getDataAsyncService(header, resultHandler, "Get child node path in next level");
-    if (service != null) {
-      service.getChildNodePathInNextLevel(header, path, resultHandler);
-    }
-  }
-
-  @Override
-  public void getAllMeasurementSchema(
-      RaftNode header, ByteBuffer planBytes, AsyncMethodCallback<ByteBuffer> resultHandler) {
-    DataAsyncService service =
-        getDataAsyncService(header, resultHandler, "Get all measurement schema");
-    if (service != null) {
-      service.getAllMeasurementSchema(header, planBytes, resultHandler);
-    }
-  }
-
-  @Override
-  public void getAggrResult(
-      GetAggrResultRequest request, AsyncMethodCallback<List<ByteBuffer>> resultHandler) {
-    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    if (service != null) {
-      service.getAggrResult(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void getUnregisteredTimeseries(
-      RaftNode header,
-      List<String> timeseriesList,
-      AsyncMethodCallback<List<String>> resultHandler) {
-    DataAsyncService service =
-        getDataAsyncService(header, resultHandler, "Check if measurements are registered");
-    if (service != null) {
-      service.getUnregisteredTimeseries(header, timeseriesList, resultHandler);
-    }
-  }
-
-  @Override
-  public void getGroupByExecutor(GroupByRequest request, AsyncMethodCallback<Long> resultHandler) {
-    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    if (service != null) {
-      service.getGroupByExecutor(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void getGroupByResult(
-      RaftNode header,
-      long executorId,
-      long startTime,
-      long endTime,
-      AsyncMethodCallback<List<ByteBuffer>> resultHandler) {
-    DataAsyncService service = getDataAsyncService(header, resultHandler, "Fetch group by");
-    if (service != null) {
-      service.getGroupByResult(header, executorId, startTime, endTime, resultHandler);
-    }
-  }
-
-  @Override
-  TProcessor getProcessor() {
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-      return new AsyncProcessor<>(this);
-    } else {
-      return new Processor<>(this);
-    }
-  }
-
-  @Override
-  TServerTransport getServerSocket() throws TTransportException {
-    logger.info(
-        "[{}] Cluster node will listen {}:{}",
-        getServerClientName(),
-        config.getInternalIp(),
-        config.getInternalDataPort());
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-      return new TNonblockingServerSocket(
-          new InetSocketAddress(config.getInternalIp(), thisNode.getDataPort()),
-          getConnectionTimeoutInMS());
-    } else {
-      return new TServerSocket(
-          new InetSocketAddress(config.getInternalIp(), thisNode.getDataPort()));
-    }
-  }
-
-  @Override
-  String getClientThreadPrefix() {
-    return "DataClientThread-";
-  }
-
-  @Override
-  String getServerClientName() {
-    return "DataServerThread-";
-  }
-
-  public void preAddNodeForDataGroup(AddNodeLog log, DataGroupMember targetDataGroupMember) {
-
-    // Make sure the previous add/remove node log has applied
-    metaGroupMember.syncLocalApply(log.getMetaLogIndex() - 1, false);
-
-    // Check the validity of the partition table
-    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
-      return;
-    }
-
-    targetDataGroupMember.preAddNode(log.getNewNode());
-  }
-
-  /**
-   * Try adding the node into the group of each DataGroupMember, and if the DataGroupMember no
-   * longer stays in that group, also remove and stop it. If the new group contains this node, also
-   * create and add a new DataGroupMember for it.
-   *
-   * @param node
-   * @param result
-   */
-  public void addNode(Node node, NodeAdditionResult result) {
-    // If the node executed adding itself to the cluster, it's unnecessary to add new groups because
-    // they already exist.
-    if (node.equals(thisNode)) {
-      return;
-    }
-    Iterator<Entry<RaftNode, DataGroupMember>> entryIterator = headerGroupMap.entrySet().iterator();
-    synchronized (headerGroupMap) {
-      while (entryIterator.hasNext()) {
-        Entry<RaftNode, DataGroupMember> entry = entryIterator.next();
-        DataGroupMember dataGroupMember = entry.getValue();
-        // the member may be extruded from the group, remove and stop it if so
-        boolean shouldLeave = dataGroupMember.addNode(node, result);
-        if (shouldLeave) {
-          logger.info("This node does not belong to {} any more", dataGroupMember.getAllNodes());
-          removeMember(entry.getKey(), entry.getValue(), false);
-          entryIterator.remove();
-        }
-      }
-
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "Data cluster server: start to handle new groups when adding new node {}", node);
-      }
-      for (PartitionGroup newGroup : result.getNewGroupList()) {
-        if (newGroup.contains(thisNode)) {
-          RaftNode header = newGroup.getHeader();
-          logger.info("Adding this node into a new group {}", newGroup);
-          DataGroupMember dataGroupMember = dataMemberFactory.create(newGroup, thisNode);
-          dataGroupMember = addDataGroupMember(dataGroupMember, header);
-          dataGroupMember.pullNodeAdditionSnapshots(
-              ((SlotPartitionTable) partitionTable).getNodeSlots(header), node);
-        }
-      }
-    }
-  }
-
-  /**
-   * When the node joins a cluster, it also creates a new data group and a corresponding member
-   * which has no data. This is to make that member pull data from other nodes.
-   */
-  public void pullSnapshots() {
-    for (int raftId = 0;
-        raftId < ClusterDescriptor.getInstance().getConfig().getMultiRaftFactor();
-        raftId++) {
-      RaftNode raftNode = new RaftNode(thisNode, raftId);
-      List<Integer> slots = ((SlotPartitionTable) partitionTable).getNodeSlots(raftNode);
-      DataGroupMember dataGroupMember = headerGroupMap.get(raftNode);
-      dataGroupMember.pullNodeAdditionSnapshots(slots, thisNode);
-    }
-  }
-
-  /**
-   * Make sure the group will not receive new raft logs
-   *
-   * @param header
-   * @param dataGroupMember
-   */
-  private void removeMember(
-      RaftNode header, DataGroupMember dataGroupMember, boolean removedGroup) {
-    dataGroupMember.setReadOnly();
-    if (!removedGroup) {
-      dataGroupMember.stop();
-    } else {
-      if (dataGroupMember.getCharacter() != NodeCharacter.LEADER) {
-        new Thread(
-                () -> {
-                  try {
-                    dataGroupMember.syncLeader(null);
-                    dataGroupMember.stop();
-                  } catch (CheckConsistencyException e) {
-                    logger.warn("Failed to check consistency.", e);
-                  }
-                })
-            .start();
-      }
-    }
-    stoppedMemberManager.put(header, dataGroupMember);
-    logger.info(
-        "Data group member has removed, header {}, group is {}.",
-        header,
-        dataGroupMember.getAllNodes());
-  }
-
-  /**
-   * Set the partition table as the in-use one and build a DataGroupMember for each local group (the
-   * group which the local node is in) and start them.
-   *
-   * @param partitionTable
-   * @throws TTransportException
-   */
-  @SuppressWarnings("java:S1135")
-  public void buildDataGroupMembers(PartitionTable partitionTable) {
-    setPartitionTable(partitionTable);
-    // TODO-Cluster: if there are unchanged members, do not stop and restart them
-    // clear previous members if the partition table is reloaded
-    for (DataGroupMember value : headerGroupMap.values()) {
-      value.stop();
-    }
-
-    for (DataGroupMember value : headerGroupMap.values()) {
-      value.setUnchanged(false);
-    }
-
-    List<PartitionGroup> partitionGroups = partitionTable.getLocalGroups();
-    for (PartitionGroup partitionGroup : partitionGroups) {
-      RaftNode header = partitionGroup.getHeader();
-      DataGroupMember prevMember = headerGroupMap.get(header);
-      if (prevMember == null || !prevMember.getAllNodes().equals(partitionGroup)) {
-        logger.info("Building member of data group: {}", partitionGroup);
-        // no previous member or member changed
-        DataGroupMember dataGroupMember = dataMemberFactory.create(partitionGroup, thisNode);
-        // the previous member will be replaced here
-        addDataGroupMember(dataGroupMember, header);
-        dataGroupMember.setUnchanged(true);
-      } else {
-        prevMember.setUnchanged(true);
-      }
-    }
-
-    // remove out-dated members of this node
-    headerGroupMap.entrySet().removeIf(e -> !e.getValue().isUnchanged());
-
-    logger.info("Data group members are ready");
-  }
-
-  public void preRemoveNodeForDataGroup(RemoveNodeLog log, DataGroupMember targetDataGroupMember) {
-
-    // Make sure the previous add/remove node log has applied
-    metaGroupMember.syncLocalApply(log.getMetaLogIndex() - 1, false);
-
-    // Check the validity of the partition table
-    if (!metaGroupMember.getPartitionTable().deserialize(log.getPartitionTable())) {
-      return;
-    }
-
-    logger.debug(
-        "Pre removing a node {} from {}",
-        log.getRemovedNode(),
-        targetDataGroupMember.getAllNodes());
-    targetDataGroupMember.preRemoveNode(log.getRemovedNode());
-  }
-
-  /**
-   * Try removing a node from the groups of each DataGroupMember. If the node is the header of some
-   * group, set the member to read only so that it can still provide data for other nodes that has
-   * not yet pulled its data. Otherwise, just change the node list of the member and pull new data.
-   * And create a new DataGroupMember if this node should join a new group because of this removal.
-   *
-   * @param node
-   * @param removalResult cluster changes due to the node removal
-   */
-  public void removeNode(Node node, NodeRemovalResult removalResult) {
-    Iterator<Entry<RaftNode, DataGroupMember>> entryIterator = headerGroupMap.entrySet().iterator();
-    synchronized (headerGroupMap) {
-      while (entryIterator.hasNext()) {
-        Entry<RaftNode, DataGroupMember> entry = entryIterator.next();
-        DataGroupMember dataGroupMember = entry.getValue();
-        if (dataGroupMember.getHeader().getNode().equals(node) || node.equals(thisNode)) {
-          entryIterator.remove();
-          removeMember(
-              entry.getKey(), dataGroupMember, dataGroupMember.getHeader().getNode().equals(node));
-        } else {
-          // the group should be updated
-          dataGroupMember.removeNode(node);
-        }
-      }
-
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "Data cluster server: start to handle new groups and pulling data when removing node {}",
-            node);
-      }
-      // if the removed group contains the local node, the local node should join a new group to
-      // preserve the replication number
-      for (PartitionGroup group : partitionTable.getLocalGroups()) {
-        RaftNode header = group.getHeader();
-        if (!headerGroupMap.containsKey(header)) {
-          logger.info("{} should join a new group {}", thisNode, group);
-          DataGroupMember dataGroupMember = dataMemberFactory.create(group, thisNode);
-          addDataGroupMember(dataGroupMember, header);
-        }
-        // pull new slots from the removed node
-        headerGroupMap.get(header).pullSlots(removalResult);
-      }
-    }
-  }
-
-  public void setPartitionTable(PartitionTable partitionTable) {
-    this.partitionTable = partitionTable;
-  }
-
-  /** @return The reports of every DataGroupMember in this node. */
-  public List<DataMemberReport> genMemberReports() {
-    List<DataMemberReport> dataMemberReports = new ArrayList<>();
-    for (DataGroupMember value : headerGroupMap.values()) {
-
-      dataMemberReports.add(value.genReport());
-    }
-    return dataMemberReports;
-  }
-
-  @Override
-  public void previousFill(
-      PreviousFillRequest request, AsyncMethodCallback<ByteBuffer> resultHandler) {
-    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, request);
-    if (service != null) {
-      service.previousFill(request, resultHandler);
-    }
-  }
-
-  public void closeLogManagers() {
-    for (DataGroupMember member : headerGroupMap.values()) {
-      member.closeLogManager();
-    }
-  }
-
-  public Map<RaftNode, DataGroupMember> getHeaderGroupMap() {
-    return headerGroupMap;
-  }
-
-  @Override
-  public void matchTerm(
-      long index, long term, RaftNode header, AsyncMethodCallback<Boolean> resultHandler) {
-    DataAsyncService service = getDataAsyncService(header, resultHandler, "Match term");
-    if (service != null) {
-      service.matchTerm(index, term, header, resultHandler);
-    }
-  }
-
-  @Override
-  public void last(LastQueryRequest request, AsyncMethodCallback<ByteBuffer> resultHandler) {
-    DataAsyncService service = getDataAsyncService(request.getHeader(), resultHandler, "last");
-    if (service != null) {
-      service.last(request, resultHandler);
-    }
-  }
-
-  @Override
-  public void getPathCount(
-      RaftNode header,
-      List<String> pathsToQuery,
-      int level,
-      AsyncMethodCallback<Integer> resultHandler) {
-    DataAsyncService service = getDataAsyncService(header, resultHandler, "count path");
-    if (service != null) {
-      service.getPathCount(header, pathsToQuery, level, resultHandler);
-    }
-  }
-
-  @Override
-  public void getDeviceCount(
-      RaftNode header, List<String> pathsToQuery, AsyncMethodCallback<Integer> resultHandler)
-      throws TException {
-    DataAsyncService service = getDataAsyncService(header, resultHandler, "count device");
-    if (service != null) {
-      service.getDeviceCount(header, pathsToQuery, resultHandler);
-    }
-  }
-
-  @Override
-  public void onSnapshotApplied(
-      RaftNode header, List<Integer> slots, AsyncMethodCallback<Boolean> resultHandler) {
-    DataAsyncService service = getDataAsyncService(header, resultHandler, "Snapshot applied");
-    if (service != null) {
-      service.onSnapshotApplied(header, slots, resultHandler);
-    }
-  }
-
-  @Override
-  public long querySingleSeries(SingleSeriesQueryRequest request) throws TException {
-    return getDataSyncService(request.getHeader()).querySingleSeries(request);
-  }
-
-  @Override
-  public long queryMultSeries(MultSeriesQueryRequest request) throws TException {
-    return getDataSyncService(request.getHeader()).queryMultSeries(request);
-  }
-
-  @Override
-  public ByteBuffer fetchSingleSeries(RaftNode header, long readerId) throws TException {
-    return getDataSyncService(header).fetchSingleSeries(header, readerId);
-  }
-
-  @Override
-  public Map<String, ByteBuffer> fetchMultSeries(RaftNode header, long readerId, List<String> paths)
-      throws TException {
-    return getDataSyncService(header).fetchMultSeries(header, readerId, paths);
-  }
-
-  @Override
-  public long querySingleSeriesByTimestamp(SingleSeriesQueryRequest request) throws TException {
-    return getDataSyncService(request.getHeader()).querySingleSeriesByTimestamp(request);
-  }
-
-  @Override
-  public ByteBuffer fetchSingleSeriesByTimestamps(
-      RaftNode header, long readerId, List<Long> timestamps) throws TException {
-    return getDataSyncService(header).fetchSingleSeriesByTimestamps(header, readerId, timestamps);
-  }
-
-  @Override
-  public void endQuery(RaftNode header, Node thisNode, long queryId) throws TException {
-    getDataSyncService(header).endQuery(header, thisNode, queryId);
-  }
-
-  @Override
-  public GetAllPathsResult getAllPaths(RaftNode header, List<String> path, boolean withAlias)
-      throws TException {
-    return getDataSyncService(header).getAllPaths(header, path, withAlias);
-  }
-
-  @Override
-  public Set<String> getAllDevices(RaftNode header, List<String> path) throws TException {
-    return getDataSyncService(header).getAllDevices(header, path);
-  }
-
-  @Override
-  public List<String> getNodeList(RaftNode header, String path, int nodeLevel) throws TException {
-    return getDataSyncService(header).getNodeList(header, path, nodeLevel);
-  }
-
-  @Override
-  public Set<String> getChildNodeInNextLevel(RaftNode header, String path) throws TException {
-    return getDataSyncService(header).getChildNodeInNextLevel(header, path);
-  }
-
-  @Override
-  public Set<String> getChildNodePathInNextLevel(RaftNode header, String path) throws TException {
-    return getDataSyncService(header).getChildNodePathInNextLevel(header, path);
-  }
-
-  @Override
-  public ByteBuffer getAllMeasurementSchema(RaftNode header, ByteBuffer planBinary)
-      throws TException {
-    return getDataSyncService(header).getAllMeasurementSchema(header, planBinary);
-  }
-
-  @Override
-  public ByteBuffer getDevices(RaftNode header, ByteBuffer planBinary) throws TException {
-    return getDataSyncService(header).getDevices(header, planBinary);
-  }
-
-  @Override
-  public List<ByteBuffer> getAggrResult(GetAggrResultRequest request) throws TException {
-    return getDataSyncService(request.getHeader()).getAggrResult(request);
-  }
-
-  @Override
-  public List<String> getUnregisteredTimeseries(RaftNode header, List<String> timeseriesList)
-      throws TException {
-    return getDataSyncService(header).getUnregisteredTimeseries(header, timeseriesList);
-  }
-
-  @Override
-  public PullSnapshotResp pullSnapshot(PullSnapshotRequest request) throws TException {
-    return getDataSyncService(request.getHeader()).pullSnapshot(request);
-  }
-
-  @Override
-  public long getGroupByExecutor(GroupByRequest request) throws TException {
-    return getDataSyncService(request.getHeader()).getGroupByExecutor(request);
-  }
-
-  @Override
-  public List<ByteBuffer> getGroupByResult(
-      RaftNode header, long executorId, long startTime, long endTime) throws TException {
-    return getDataSyncService(header).getGroupByResult(header, executorId, startTime, endTime);
-  }
-
-  @Override
-  public PullSchemaResp pullTimeSeriesSchema(PullSchemaRequest request) throws TException {
-    return getDataSyncService(request.getHeader()).pullTimeSeriesSchema(request);
-  }
-
-  @Override
-  public PullSchemaResp pullMeasurementSchema(PullSchemaRequest request) throws TException {
-    return getDataSyncService(request.getHeader()).pullMeasurementSchema(request);
-  }
-
-  @Override
-  public ByteBuffer previousFill(PreviousFillRequest request) throws TException {
-    return getDataSyncService(request.getHeader()).previousFill(request);
-  }
-
-  @Override
-  public ByteBuffer last(LastQueryRequest request) throws TException {
-    return getDataSyncService(request.getHeader()).last(request);
-  }
-
-  @Override
-  public int getPathCount(RaftNode header, List<String> pathsToQuery, int level) throws TException {
-    return getDataSyncService(header).getPathCount(header, pathsToQuery, level);
-  }
-
-  @Override
-  public int getDeviceCount(RaftNode header, List<String> pathsToQuery) throws TException {
-    return getDataSyncService(header).getDeviceCount(header, pathsToQuery);
-  }
-
-  @Override
-  public boolean onSnapshotApplied(RaftNode header, List<Integer> slots) {
-    return getDataSyncService(header).onSnapshotApplied(header, slots);
-  }
-
-  @Override
-  public HeartBeatResponse sendHeartbeat(HeartBeatRequest request) {
-    return getDataSyncService(request.getHeader()).sendHeartbeat(request);
-  }
-
-  @Override
-  public long startElection(ElectionRequest request) {
-    return getDataSyncService(request.getHeader()).startElection(request);
-  }
-
-  @Override
-  public long appendEntries(AppendEntriesRequest request) throws TException {
-    return getDataSyncService(request.getHeader()).appendEntries(request);
-  }
-
-  @Override
-  public long appendEntry(AppendEntryRequest request) throws TException {
-    return getDataSyncService(request.getHeader()).appendEntry(request);
-  }
-
-  @Override
-  public void sendSnapshot(SendSnapshotRequest request) throws TException {
-    getDataSyncService(request.getHeader()).sendSnapshot(request);
-  }
-
-  @Override
-  public TSStatus executeNonQueryPlan(ExecutNonQueryReq request) throws TException {
-    return getDataSyncService(request.getHeader()).executeNonQueryPlan(request);
-  }
-
-  @Override
-  public RequestCommitIndexResponse requestCommitIndex(RaftNode header) throws TException {
-    return getDataSyncService(header).requestCommitIndex(header);
-  }
-
-  @Override
-  public ByteBuffer readFile(String filePath, long offset, int length) throws TException {
-    try {
-      return IOUtils.readFile(filePath, offset, length);
-    } catch (IOException e) {
-      throw new TException(e);
-    }
-  }
-
-  @Override
-  public boolean matchTerm(long index, long term, RaftNode header) {
-    return getDataSyncService(header).matchTerm(index, term, header);
-  }
-
-  @Override
-  public ByteBuffer peekNextNotNullValue(
-      RaftNode header, long executorId, long startTime, long endTime) throws TException {
-    return getDataSyncService(header).peekNextNotNullValue(header, executorId, startTime, endTime);
-  }
-
-  @Override
-  public void peekNextNotNullValue(
-      RaftNode header,
-      long executorId,
-      long startTime,
-      long endTime,
-      AsyncMethodCallback<ByteBuffer> resultHandler)
-      throws TException {
-    resultHandler.onComplete(
-        getDataSyncService(header).peekNextNotNullValue(header, executorId, startTime, endTime));
-  }
-
-  @Override
-  public void removeHardLink(String hardLinkPath) throws TException {
-    try {
-      Files.deleteIfExists(new File(hardLinkPath).toPath());
-    } catch (IOException e) {
-      throw new TException(e);
-    }
-  }
-
-  @Override
-  public void removeHardLink(String hardLinkPath, AsyncMethodCallback<Void> resultHandler) {
-    try {
-      Files.deleteIfExists(new File(hardLinkPath).toPath());
-      resultHandler.onComplete(null);
-    } catch (IOException e) {
-      resultHandler.onError(e);
-    }
-  }
-}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/MetaClusterServer.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/MetaClusterServer.java
deleted file mode 100644
index ec19cad..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/MetaClusterServer.java
+++ /dev/null
@@ -1,379 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.cluster.server;
-
-import org.apache.iotdb.cluster.config.ClusterDescriptor;
-import org.apache.iotdb.cluster.coordinator.Coordinator;
-import org.apache.iotdb.cluster.exception.ConfigInconsistentException;
-import org.apache.iotdb.cluster.exception.StartUpCheckFailureException;
-import org.apache.iotdb.cluster.metadata.CMManager;
-import org.apache.iotdb.cluster.metadata.MetaPuller;
-import org.apache.iotdb.cluster.rpc.thrift.*;
-import org.apache.iotdb.cluster.rpc.thrift.TSMetaService.AsyncProcessor;
-import org.apache.iotdb.cluster.rpc.thrift.TSMetaService.Processor;
-import org.apache.iotdb.cluster.server.heartbeat.MetaHeartbeatServer;
-import org.apache.iotdb.cluster.server.member.MetaGroupMember;
-import org.apache.iotdb.cluster.server.service.MetaAsyncService;
-import org.apache.iotdb.cluster.server.service.MetaSyncService;
-import org.apache.iotdb.cluster.utils.nodetool.ClusterMonitor;
-import org.apache.iotdb.db.exception.StartupException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.service.RegisterManager;
-import org.apache.iotdb.db.utils.TestOnly;
-import org.apache.iotdb.service.rpc.thrift.TSStatus;
-
-import org.apache.thrift.TException;
-import org.apache.thrift.TProcessor;
-import org.apache.thrift.async.AsyncMethodCallback;
-import org.apache.thrift.transport.TNonblockingServerSocket;
-import org.apache.thrift.transport.TServerSocket;
-import org.apache.thrift.transport.TServerTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-
-/**
- * MetaCluster manages the whole cluster's metadata, such as what nodes are in the cluster and the
- * data partition. Each node has one MetaClusterServer instance, the single-node IoTDB instance is
- * started-up at the same time.
- */
-public class MetaClusterServer extends RaftServer
-    implements TSMetaService.AsyncIface, TSMetaService.Iface {
-  private static Logger logger = LoggerFactory.getLogger(MetaClusterServer.class);
-
-  // each node only contains one MetaGroupMember
-  private MetaGroupMember member;
-  private Coordinator coordinator;
-  // the single-node IoTDB instance
-  private IoTDB ioTDB;
-  // to register the ClusterMonitor that helps monitoring the cluster
-  private RegisterManager registerManager = new RegisterManager();
-  private MetaAsyncService asyncService;
-  private MetaSyncService syncService;
-  private MetaHeartbeatServer metaHeartbeatServer;
-
-  public MetaClusterServer() throws QueryProcessException {
-    super();
-    metaHeartbeatServer = new MetaHeartbeatServer(thisNode, this);
-    coordinator = new Coordinator();
-    member = new MetaGroupMember(protocolFactory, thisNode, coordinator);
-    coordinator.setMetaGroupMember(member);
-    asyncService = new MetaAsyncService(member);
-    syncService = new MetaSyncService(member);
-    MetaPuller.getInstance().init(member);
-  }
-
-  /**
-   * Besides the standard RaftServer start-up, the IoTDB instance, a MetaGroupMember and the
-   * ClusterMonitor are also started.
-   *
-   * @throws TTransportException
-   * @throws StartupException
-   */
-  @Override
-  public void start() throws TTransportException, StartupException {
-    super.start();
-    metaHeartbeatServer.start();
-    ioTDB = new IoTDB();
-    IoTDB.setMetaManager(CMManager.getInstance());
-    IoTDB.setClusterMode();
-    ((CMManager) IoTDB.metaManager).setMetaGroupMember(member);
-    ((CMManager) IoTDB.metaManager).setCoordinator(coordinator);
-    ioTDB.active();
-    member.start();
-    // JMX based DBA API
-    registerManager.register(ClusterMonitor.INSTANCE);
-  }
-
-  /** Also stops the IoTDB instance, the MetaGroupMember and the ClusterMonitor. */
-  @Override
-  public void stop() {
-    if (ioTDB == null) {
-      return;
-    }
-    metaHeartbeatServer.stop();
-    super.stop();
-    ioTDB.stop();
-    ioTDB = null;
-    member.stop();
-    registerManager.deregisterAll();
-  }
-
-  /** Build a initial cluster with other nodes on the seed list. */
-  public void buildCluster() throws ConfigInconsistentException, StartUpCheckFailureException {
-    member.buildCluster();
-  }
-
-  /**
-   * Pick up a node from the seed list and send a join request to it.
-   *
-   * @return whether the node has joined the cluster.
-   */
-  public void joinCluster() throws ConfigInconsistentException, StartUpCheckFailureException {
-    member.joinCluster();
-  }
-
-  /**
-   * MetaClusterServer uses the meta port to create the socket.
-   *
-   * @return the TServerTransport
-   * @throws TTransportException if create the socket fails
-   */
-  @Override
-  TServerTransport getServerSocket() throws TTransportException {
-    logger.info(
-        "[{}] Cluster node will listen {}:{}",
-        getServerClientName(),
-        config.getInternalIp(),
-        config.getInternalMetaPort());
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-      return new TNonblockingServerSocket(
-          new InetSocketAddress(config.getInternalIp(), config.getInternalMetaPort()),
-          getConnectionTimeoutInMS());
-    } else {
-      return new TServerSocket(
-          new InetSocketAddress(config.getInternalIp(), config.getInternalMetaPort()));
-    }
-  }
-
-  @Override
-  String getClientThreadPrefix() {
-    return "MetaClientThread-";
-  }
-
-  @Override
-  String getServerClientName() {
-    return "MetaServerThread-";
-  }
-
-  @Override
-  TProcessor getProcessor() {
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-      return new AsyncProcessor<>(this);
-    } else {
-      return new Processor<>(this);
-    }
-  }
-
-  // Request forwarding. There is only one MetaGroupMember each node, so all requests will be
-  // directly sent to that member. See the methods in MetaGroupMember for details
-
-  @Override
-  public void addNode(Node node, StartUpStatus startUpStatus, AsyncMethodCallback resultHandler) {
-    asyncService.addNode(node, startUpStatus, resultHandler);
-  }
-
-  @Override
-  public void sendHeartbeat(HeartBeatRequest request, AsyncMethodCallback resultHandler) {
-    asyncService.sendHeartbeat(request, resultHandler);
-  }
-
-  @Override
-  public void startElection(ElectionRequest electionRequest, AsyncMethodCallback resultHandler) {
-    asyncService.startElection(electionRequest, resultHandler);
-  }
-
-  @Override
-  public void appendEntries(AppendEntriesRequest request, AsyncMethodCallback resultHandler) {
-    asyncService.appendEntries(request, resultHandler);
-  }
-
-  @Override
-  public void appendEntry(AppendEntryRequest request, AsyncMethodCallback resultHandler) {
-    asyncService.appendEntry(request, resultHandler);
-  }
-
-  @Override
-  public void sendSnapshot(SendSnapshotRequest request, AsyncMethodCallback resultHandler) {
-    asyncService.sendSnapshot(request, resultHandler);
-  }
-
-  @Override
-  public void executeNonQueryPlan(
-      ExecutNonQueryReq request, AsyncMethodCallback<TSStatus> resultHandler) {
-    asyncService.executeNonQueryPlan(request, resultHandler);
-  }
-
-  @Override
-  public void requestCommitIndex(
-      RaftNode header, AsyncMethodCallback<RequestCommitIndexResponse> resultHandler) {
-    asyncService.requestCommitIndex(header, resultHandler);
-  }
-
-  @Override
-  public void checkAlive(AsyncMethodCallback<Node> resultHandler) {
-    asyncService.checkAlive(resultHandler);
-  }
-
-  @Override
-  public void collectMigrationStatus(AsyncMethodCallback<ByteBuffer> resultHandler) {
-    asyncService.collectMigrationStatus(resultHandler);
-  }
-
-  @Override
-  public void readFile(
-      String filePath, long offset, int length, AsyncMethodCallback<ByteBuffer> resultHandler) {
-    asyncService.readFile(filePath, offset, length, resultHandler);
-  }
-
-  @Override
-  public void queryNodeStatus(AsyncMethodCallback<TNodeStatus> resultHandler) {
-    asyncService.queryNodeStatus(resultHandler);
-  }
-
-  public MetaGroupMember getMember() {
-    return member;
-  }
-
-  @Override
-  public void checkStatus(
-      StartUpStatus startUpStatus, AsyncMethodCallback<CheckStatusResponse> resultHandler) {
-    asyncService.checkStatus(startUpStatus, resultHandler);
-  }
-
-  @Override
-  public void removeNode(Node node, AsyncMethodCallback<Long> resultHandler) {
-    asyncService.removeNode(node, resultHandler);
-  }
-
-  @Override
-  public void exile(ByteBuffer removeNodeLog, AsyncMethodCallback<Void> resultHandler) {
-    asyncService.exile(removeNodeLog, resultHandler);
-  }
-
-  @Override
-  public void matchTerm(
-      long index, long term, RaftNode header, AsyncMethodCallback<Boolean> resultHandler) {
-    asyncService.matchTerm(index, term, header, resultHandler);
-  }
-
-  @Override
-  public AddNodeResponse addNode(Node node, StartUpStatus startUpStatus) throws TException {
-    return syncService.addNode(node, startUpStatus);
-  }
-
-  @Override
-  public CheckStatusResponse checkStatus(StartUpStatus startUpStatus) {
-    return syncService.checkStatus(startUpStatus);
-  }
-
-  @Override
-  public long removeNode(Node node) throws TException {
-    return syncService.removeNode(node);
-  }
-
-  @Override
-  public void exile(ByteBuffer removeNodeLog) {
-    syncService.exile(removeNodeLog);
-  }
-
-  @Override
-  public TNodeStatus queryNodeStatus() {
-    return syncService.queryNodeStatus();
-  }
-
-  @Override
-  public Node checkAlive() {
-    return syncService.checkAlive();
-  }
-
-  @Override
-  public ByteBuffer collectMigrationStatus() {
-    return syncService.collectMigrationStatus();
-  }
-
-  @Override
-  public HeartBeatResponse sendHeartbeat(HeartBeatRequest request) {
-    return syncService.sendHeartbeat(request);
-  }
-
-  @Override
-  public long startElection(ElectionRequest request) {
-    return syncService.startElection(request);
-  }
-
-  @Override
-  public long appendEntries(AppendEntriesRequest request) throws TException {
-    return syncService.appendEntries(request);
-  }
-
-  @Override
-  public long appendEntry(AppendEntryRequest request) throws TException {
-    return syncService.appendEntry(request);
-  }
-
-  @Override
-  public void sendSnapshot(SendSnapshotRequest request) throws TException {
-    syncService.sendSnapshot(request);
-  }
-
-  @Override
-  public TSStatus executeNonQueryPlan(ExecutNonQueryReq request) throws TException {
-    return syncService.executeNonQueryPlan(request);
-  }
-
-  @Override
-  public RequestCommitIndexResponse requestCommitIndex(RaftNode header) throws TException {
-    return syncService.requestCommitIndex(header);
-  }
-
-  @Override
-  public ByteBuffer readFile(String filePath, long offset, int length) throws TException {
-    return syncService.readFile(filePath, offset, length);
-  }
-
-  @Override
-  public boolean matchTerm(long index, long term, RaftNode header) {
-    return syncService.matchTerm(index, term, header);
-  }
-
-  @Override
-  public void removeHardLink(String hardLinkPath) throws TException {
-    syncService.removeHardLink(hardLinkPath);
-  }
-
-  @Override
-  public void removeHardLink(String hardLinkPath, AsyncMethodCallback<Void> resultHandler) {
-    asyncService.removeHardLink(hardLinkPath, resultHandler);
-  }
-
-  @Override
-  public void handshake(Node sender) {
-    syncService.handshake(sender);
-  }
-
-  @Override
-  public void handshake(Node sender, AsyncMethodCallback<Void> resultHandler) {
-    asyncService.handshake(sender, resultHandler);
-  }
-
-  @TestOnly
-  public void setMetaGroupMember(MetaGroupMember metaGroupMember) {
-    this.member = metaGroupMember;
-  }
-
-  @TestOnly
-  public IoTDB getIoTDB() {
-    return ioTDB;
-  }
-}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/PullSnapshotHintService.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/PullSnapshotHintService.java
index c137027..d594d05 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/PullSnapshotHintService.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/PullSnapshotHintService.java
@@ -28,6 +28,7 @@ import org.apache.iotdb.cluster.partition.PartitionGroup;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.rpc.thrift.RaftNode;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
 
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
@@ -36,7 +37,6 @@ import org.slf4j.LoggerFactory;
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.ConcurrentLinkedDeque;
-import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
@@ -56,7 +56,7 @@ public class PullSnapshotHintService {
   }
 
   public void start() {
-    this.service = Executors.newScheduledThreadPool(1);
+    this.service = IoTDBThreadPoolFactory.newScheduledThreadPool(1, "PullSnapshotHint");
     this.service.scheduleAtFixedRate(this::sendHints, 0, 10, TimeUnit.MILLISECONDS);
   }
 
@@ -135,15 +135,27 @@ public class PullSnapshotHintService {
   private boolean sendHintsAsync(Node receiver, PullSnapshotHint hint)
       throws TException, InterruptedException {
     AsyncDataClient asyncDataClient = (AsyncDataClient) member.getAsyncClient(receiver);
+    if (asyncDataClient == null) {
+      return false;
+    }
     return SyncClientAdaptor.onSnapshotApplied(asyncDataClient, hint.getHeader(), hint.slots);
   }
 
   private boolean sendHintSync(Node receiver, PullSnapshotHint hint) throws TException {
-    try (SyncDataClient syncDataClient = (SyncDataClient) member.getSyncClient(receiver)) {
+    SyncDataClient syncDataClient = null;
+    try {
+      syncDataClient = (SyncDataClient) member.getSyncClient(receiver);
       if (syncDataClient == null) {
         return false;
       }
       return syncDataClient.onSnapshotApplied(hint.getHeader(), hint.slots);
+    } catch (TException e) {
+      syncDataClient.close();
+      throw e;
+    } finally {
+      if (syncDataClient != null) {
+        syncDataClient.returnSelf();
+      }
     }
   }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/RaftServer.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/RaftServer.java
deleted file mode 100644
index 91358d4..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/RaftServer.java
+++ /dev/null
@@ -1,285 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.cluster.server;
-
-import org.apache.iotdb.cluster.config.ClusterConfig;
-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.iotdb.cluster.utils.ClusterUtils;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.StartupException;
-import org.apache.iotdb.db.utils.CommonUtils;
-import org.apache.iotdb.db.utils.TestOnly;
-import org.apache.iotdb.rpc.RpcTransportFactory;
-
-import org.apache.thrift.TProcessor;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.protocol.TProtocolFactory;
-import org.apache.thrift.server.TServer;
-import org.apache.thrift.server.TThreadedSelectorServer;
-import org.apache.thrift.transport.TNonblockingServerTransport;
-import org.apache.thrift.transport.TServerTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ConcurrentModificationException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.atomic.AtomicLong;
-
-/**
- * RaftServer works as a broker (network and protocol layer) that sends the requests to the proper
- * RaftMembers to process.
- */
-public abstract class RaftServer implements RaftService.AsyncIface, RaftService.Iface {
-
-  private static final Logger logger = LoggerFactory.getLogger(RaftServer.class);
-
-  // Heartbeat client connection timeout should not be larger than heartbeat interval, otherwise
-  // the thread pool of sending heartbeats or requesting votes may be used up by waiting for
-  // establishing connection with some slow or dead nodes.
-  private static final int heartbeatClientConnTimeoutMs =
-      Math.min((int) RaftServer.getHeartbeatIntervalMs(), RaftServer.getConnectionTimeoutInMS());
-
-  private static int connectionTimeoutInMS =
-      ClusterDescriptor.getInstance().getConfig().getConnectionTimeoutInMS();
-  private static int readOperationTimeoutMS =
-      ClusterDescriptor.getInstance().getConfig().getReadOperationTimeoutMS();
-  private static int writeOperationTimeoutMS =
-      ClusterDescriptor.getInstance().getConfig().getWriteOperationTimeoutMS();
-  private static int syncLeaderMaxWaitMs = 20 * 1000;
-  private static long heartbeatIntervalMs =
-      ClusterDescriptor.getInstance().getConfig().getHeartbeatIntervalMs();
-  private static long electionTimeoutMs =
-      ClusterDescriptor.getInstance().getConfig().getElectionTimeoutMs();
-
-  ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
-  // the socket poolServer will listen to
-  private TServerTransport socket;
-  // RPC processing server
-  private TServer poolServer;
-  Node thisNode;
-
-  TProtocolFactory protocolFactory =
-      config.isRpcThriftCompressionEnabled()
-          ? new TCompactProtocol.Factory()
-          : new TBinaryProtocol.Factory();
-
-  // this thread pool is to run the thrift server (poolServer above)
-  private ExecutorService clientService;
-
-  RaftServer() {
-    thisNode = new Node();
-    // set internal rpc ip and ports
-    thisNode.setInternalIp(config.getInternalIp());
-    thisNode.setMetaPort(config.getInternalMetaPort());
-    thisNode.setDataPort(config.getInternalDataPort());
-    // set client rpc ip and ports
-    thisNode.setClientPort(config.getClusterRpcPort());
-    thisNode.setClientIp(IoTDBDescriptor.getInstance().getConfig().getRpcAddress());
-  }
-
-  RaftServer(Node thisNode) {
-    this.thisNode = thisNode;
-  }
-
-  public static int getConnectionTimeoutInMS() {
-    return connectionTimeoutInMS;
-  }
-
-  public static void setConnectionTimeoutInMS(int connectionTimeoutInMS) {
-    RaftServer.connectionTimeoutInMS = connectionTimeoutInMS;
-  }
-
-  public static int getReadOperationTimeoutMS() {
-    return readOperationTimeoutMS;
-  }
-
-  public static int getWriteOperationTimeoutMS() {
-    return writeOperationTimeoutMS;
-  }
-
-  public static int getSyncLeaderMaxWaitMs() {
-    return syncLeaderMaxWaitMs;
-  }
-
-  public static void setSyncLeaderMaxWaitMs(int syncLeaderMaxWaitMs) {
-    RaftServer.syncLeaderMaxWaitMs = syncLeaderMaxWaitMs;
-  }
-
-  public static long getHeartbeatIntervalMs() {
-    return heartbeatIntervalMs;
-  }
-
-  public static void setHeartbeatIntervalMs(long heartbeatIntervalMs) {
-    RaftServer.heartbeatIntervalMs = heartbeatIntervalMs;
-  }
-
-  public static long getElectionTimeoutMs() {
-    return electionTimeoutMs;
-  }
-
-  public static void setElectionTimeoutMs(long electionTimeoutMs) {
-    RaftServer.electionTimeoutMs = electionTimeoutMs;
-  }
-
-  public static int getHeartbeatClientConnTimeoutMs() {
-    return heartbeatClientConnTimeoutMs;
-  }
-
-  /**
-   * Establish a thrift server with the configurations in ClusterConfig to listen to and respond to
-   * thrift RPCs. Calling the method twice does not induce side effects.
-   *
-   * @throws TTransportException
-   */
-  @SuppressWarnings("java:S1130") // thrown in override method
-  public void start() throws TTransportException, StartupException {
-    if (poolServer != null) {
-      return;
-    }
-
-    establishServer();
-  }
-
-  /**
-   * Stop the thrift server, close the socket and interrupt all in progress RPCs. Calling the method
-   * twice does not induce side effects.
-   */
-  public void stop() {
-    if (poolServer == null) {
-      return;
-    }
-
-    try {
-      poolServer.stop();
-    } catch (ConcurrentModificationException e) {
-      // ignore
-    }
-    socket.close();
-    clientService.shutdownNow();
-    socket = null;
-    poolServer = null;
-  }
-
-  /**
-   * @return An AsyncProcessor that contains the extended interfaces of a non-abstract subclass of
-   *     RaftService (DataService or MetaService).
-   */
-  abstract TProcessor getProcessor();
-
-  /**
-   * @return A socket that will be used to establish a thrift server to listen to RPC requests.
-   *     DataServer and MetaServer use different port, so this is to be determined.
-   * @throws TTransportException
-   */
-  abstract TServerTransport getServerSocket() throws TTransportException;
-
-  /**
-   * Each thrift RPC request will be processed in a separate thread and this will return the name
-   * prefix of such threads. This is used to fast distinguish DataServer and MetaServer in the logs
-   * for the sake of debug.
-   *
-   * @return name prefix of RPC processing threads.
-   */
-  abstract String getClientThreadPrefix();
-
-  /**
-   * The thrift server will be run in a separate thread, and this will be its name. It help you
-   * locate the desired logs quickly when debugging.
-   *
-   * @return The name of the thread running the thrift server.
-   */
-  abstract String getServerClientName();
-
-  private TServer createAsyncServer() throws TTransportException {
-    socket = getServerSocket();
-    TThreadedSelectorServer.Args poolArgs =
-        new TThreadedSelectorServer.Args((TNonblockingServerTransport) socket);
-    poolArgs.maxReadBufferBytes = IoTDBDescriptor.getInstance().getConfig().getThriftMaxFrameSize();
-    poolArgs.selectorThreads(CommonUtils.getCpuCores());
-    int maxConcurrentClientNum =
-        Math.max(CommonUtils.getCpuCores(), config.getMaxConcurrentClientNum());
-    poolArgs.executorService(
-        new ThreadPoolExecutor(
-            CommonUtils.getCpuCores(),
-            maxConcurrentClientNum,
-            poolArgs.getStopTimeoutVal(),
-            poolArgs.getStopTimeoutUnit(),
-            new SynchronousQueue<>(),
-            new ThreadFactory() {
-              private AtomicLong threadIndex = new AtomicLong(0);
-
-              @Override
-              public Thread newThread(Runnable r) {
-                return new Thread(r, getClientThreadPrefix() + threadIndex.incrementAndGet());
-              }
-            }));
-    poolArgs.processor(getProcessor());
-    poolArgs.protocolFactory(protocolFactory);
-    // async service requires FramedTransport
-    poolArgs.transportFactory(RpcTransportFactory.INSTANCE);
-
-    // run the thrift server in a separate thread so that the main thread is not blocked
-    return new TThreadedSelectorServer(poolArgs);
-  }
-
-  private TServer createSyncServer() throws TTransportException {
-    socket = getServerSocket();
-    return ClusterUtils.createTThreadPoolServer(
-        socket, getClientThreadPrefix(), getProcessor(), protocolFactory);
-  }
-
-  private void establishServer() throws TTransportException {
-    logger.info(
-        "[{}] Cluster node {} begins to set up with {} mode",
-        getServerClientName(),
-        thisNode,
-        ClusterDescriptor.getInstance().getConfig().isUseAsyncServer() ? "Async" : "Sync");
-
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-      poolServer = createAsyncServer();
-    } else {
-      poolServer = createSyncServer();
-    }
-
-    clientService = Executors.newSingleThreadExecutor(r -> new Thread(r, getServerClientName()));
-
-    clientService.submit(() -> poolServer.serve());
-
-    logger.info("[{}] Cluster node {} is up", getServerClientName(), thisNode);
-  }
-
-  @TestOnly
-  public static void setReadOperationTimeoutMS(int readOperationTimeoutMS) {
-    RaftServer.readOperationTimeoutMS = readOperationTimeoutMS;
-  }
-
-  @TestOnly
-  public static void setWriteOperationTimeoutMS(int writeOperationTimeoutMS) {
-    RaftServer.writeOperationTimeoutMS = writeOperationTimeoutMS;
-  }
-}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/Response.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/Response.java
index 006eec1..387549d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/Response.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/Response.java
@@ -53,6 +53,9 @@ public class Response {
   // the request is not executed locally anc should be forwarded
   public static final long RESPONSE_NULL = Long.MIN_VALUE;
 
+  // the meta engine is not ready (except for the partitionTable is ready)
+  public static final long RESPONSE_META_NOT_READY = -12;
+
   private Response() {
     // enum-like class
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/StoppedMemberManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/StoppedMemberManager.java
index 788e83b..a4abde2 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/StoppedMemberManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/StoppedMemberManager.java
@@ -58,11 +58,9 @@ public class StoppedMemberManager {
 
   private Map<RaftNode, DataGroupMember> removedMemberMap = new HashMap<>();
   private DataGroupMember.Factory memberFactory;
-  private Node thisNode;
 
-  StoppedMemberManager(Factory memberFactory, Node thisNode) {
+  public StoppedMemberManager(Factory memberFactory) {
     this.memberFactory = memberFactory;
-    this.thisNode = thisNode;
     recover();
   }
 
@@ -70,9 +68,8 @@ public class StoppedMemberManager {
    * When a DataGroupMember is removed, add it here and record this removal, so in next start-up we
    * can recover it as a data source for data transfers.
    *
-   * @param header When a DataGroupMember is removed, add it here and record this removal, so in
+   * @param raftNode When a DataGroupMember is removed, add it here and record this removal, so in
    *     next start-up we can recover it as a data source for data transfers.
-   * @param raftNode
    * @param dataGroupMember
    */
   public synchronized void put(RaftNode raftNode, DataGroupMember dataGroupMember) {
@@ -150,7 +147,7 @@ public class StoppedMemberManager {
       Node node = ClusterUtils.stringToNode(split[i]);
       partitionGroup.add(node);
     }
-    DataGroupMember member = memberFactory.create(partitionGroup, thisNode);
+    DataGroupMember member = memberFactory.create(partitionGroup);
     member.setReadOnly();
     removedMemberMap.put(partitionGroup.getHeader(), member);
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/clusterinfo/ClusterInfoServer.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/clusterinfo/ClusterInfoServer.java
index bf08e7d..39619b6 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/clusterinfo/ClusterInfoServer.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/clusterinfo/ClusterInfoServer.java
@@ -49,6 +49,7 @@ public class ClusterInfoServer extends ThriftService implements ClusterInfoServe
 
   @Override
   public void initTProcessor() {
+    initSyncedServiceImpl(null);
     serviceImpl = new ClusterInfoServiceImpl();
     processor = new Processor<>(serviceImpl);
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/AppendNodeEntryHandler.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/AppendNodeEntryHandler.java
index fa744d1..2945726 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/AppendNodeEntryHandler.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/AppendNodeEntryHandler.java
@@ -77,7 +77,8 @@ public class AppendNodeEntryHandler implements AsyncMethodCallback<Long> {
       // the request already failed
       return;
     }
-    logger.debug("{}: Append response {} from {}", member.getName(), response, receiver);
+    logger.debug(
+        "{}: Append response {} from {} for log {}", member.getName(), response, receiver, log);
     if (leaderShipStale.get()) {
       // someone has rejected this log because the leadership is stale
       return;
@@ -106,11 +107,12 @@ public class AppendNodeEntryHandler implements AsyncMethodCallback<Long> {
         // the leader ship is stale, wait for the new leader's heartbeat
         long prevReceiverTerm = receiverTerm.get();
         logger.debug(
-            "{}: Received a rejection from {} because term is stale: {}/{}",
+            "{}: Received a rejection from {} because term is stale: {}/{} for log {}",
             member.getName(),
             receiver,
             prevReceiverTerm,
-            resp);
+            resp,
+            log);
         if (resp > prevReceiverTerm) {
           receiverTerm.set(resp);
         }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/HeartbeatHandler.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/HeartbeatHandler.java
index ba26259..bccb16f 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/HeartbeatHandler.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/HeartbeatHandler.java
@@ -54,7 +54,11 @@ public class HeartbeatHandler implements AsyncMethodCallback<HeartBeatResponse>
   public void onComplete(HeartBeatResponse resp) {
     long followerTerm = resp.getTerm();
     if (logger.isDebugEnabled()) {
-      logger.debug("{}: Received a heartbeat response {}", memberName, followerTerm);
+      logger.debug(
+          "{}: Received a heartbeat response {} for last log index {}",
+          memberName,
+          followerTerm,
+          resp.getLastLogIndex());
     }
     if (followerTerm == RESPONSE_AGREE) {
       // current leadership is still valid
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/DataHeartbeatServer.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/DataHeartbeatServer.java
deleted file mode 100644
index d38670f..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/DataHeartbeatServer.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.cluster.server.heartbeat;
-
-import org.apache.iotdb.cluster.config.ClusterDescriptor;
-import org.apache.iotdb.cluster.rpc.thrift.Node;
-import org.apache.iotdb.cluster.rpc.thrift.TSDataService.AsyncProcessor;
-import org.apache.iotdb.cluster.rpc.thrift.TSDataService.Processor;
-import org.apache.iotdb.cluster.server.DataClusterServer;
-import org.apache.iotdb.cluster.utils.ClusterUtils;
-
-import org.apache.thrift.TProcessor;
-import org.apache.thrift.transport.TNonblockingServerSocket;
-import org.apache.thrift.transport.TServerSocket;
-import org.apache.thrift.transport.TServerTransport;
-import org.apache.thrift.transport.TTransportException;
-
-import java.net.InetSocketAddress;
-
-public class DataHeartbeatServer extends HeartbeatServer {
-
-  private DataClusterServer dataClusterServer;
-
-  /** Do not use this method for initialization */
-  private DataHeartbeatServer() {}
-
-  public DataHeartbeatServer(Node thisNode, DataClusterServer dataClusterServer) {
-    super(thisNode);
-    this.dataClusterServer = dataClusterServer;
-  }
-
-  @Override
-  TProcessor getProcessor() {
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-      return new AsyncProcessor<>(dataClusterServer);
-    } else {
-      return new Processor<>(dataClusterServer);
-    }
-  }
-
-  @Override
-  TServerTransport getHeartbeatServerSocket() throws TTransportException {
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-      return new TNonblockingServerSocket(
-          new InetSocketAddress(
-              config.getInternalIp(),
-              thisNode.getDataPort() + ClusterUtils.DATA_HEARTBEAT_PORT_OFFSET),
-          getConnectionTimeoutInMS());
-    } else {
-      return new TServerSocket(
-          new InetSocketAddress(
-              config.getInternalIp(),
-              thisNode.getDataPort() + ClusterUtils.DATA_HEARTBEAT_PORT_OFFSET));
-    }
-  }
-
-  @Override
-  String getClientThreadPrefix() {
-    return "DataHeartbeatClientThread-";
-  }
-
-  @Override
-  String getServerClientName() {
-    return "DataHeartbeatServerThread-";
-  }
-}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/HeartbeatServer.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/HeartbeatServer.java
deleted file mode 100644
index 63fc2c5..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/HeartbeatServer.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.cluster.server.heartbeat;
-
-import org.apache.iotdb.cluster.config.ClusterConfig;
-import org.apache.iotdb.cluster.config.ClusterDescriptor;
-import org.apache.iotdb.cluster.rpc.thrift.Node;
-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.utils.CommonUtils;
-import org.apache.iotdb.rpc.RpcTransportFactory;
-
-import org.apache.thrift.TProcessor;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.protocol.TProtocolFactory;
-import org.apache.thrift.server.THsHaServer;
-import org.apache.thrift.server.THsHaServer.Args;
-import org.apache.thrift.server.TServer;
-import org.apache.thrift.transport.TNonblockingServerTransport;
-import org.apache.thrift.transport.TServerTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.atomic.AtomicLong;
-
-/**
- * HeartbeatServer works as a broker (network and protocol layer) that sends and receive the
- * heartbeat requests to the proper RaftMembers to process.
- */
-public abstract class HeartbeatServer {
-
-  private static final Logger logger = LoggerFactory.getLogger(HeartbeatServer.class);
-  private static int connectionTimeoutInMS =
-      ClusterDescriptor.getInstance().getConfig().getConnectionTimeoutInMS();
-
-  ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
-  /** the heartbeat socket poolServer will listen to */
-  private TServerTransport heartbeatSocket;
-
-  /** the heartbeat RPC processing server */
-  private TServer heartbeatPoolServer;
-
-  Node thisNode;
-
-  private TProtocolFactory heartbeatProtocolFactory =
-      config.isRpcThriftCompressionEnabled()
-          ? new TCompactProtocol.Factory()
-          : new TBinaryProtocol.Factory();
-
-  /** This thread pool is to run the thrift server (heartbeatPoolServer above) */
-  private ExecutorService heartbeatClientService;
-
-  HeartbeatServer() {
-    thisNode = new Node();
-    thisNode.setInternalIp(config.getInternalIp());
-    thisNode.setMetaPort(config.getInternalMetaPort());
-    thisNode.setDataPort(config.getInternalDataPort());
-    thisNode.setClientIp(IoTDBDescriptor.getInstance().getConfig().getRpcAddress());
-  }
-
-  HeartbeatServer(Node thisNode) {
-    this.thisNode = thisNode;
-  }
-
-  public static int getConnectionTimeoutInMS() {
-    return connectionTimeoutInMS;
-  }
-
-  /**
-   * Establish a thrift server with the configurations in ClusterConfig to listen to and respond to
-   * thrift RPCs. Calling the method twice does not induce side effects.
-   *
-   * @throws TTransportException
-   */
-  @SuppressWarnings("java:S1130") // thrown in override method
-  public void start() throws TTransportException, StartupException {
-    if (heartbeatPoolServer != null) {
-      return;
-    }
-
-    establishHeartbeatServer();
-  }
-
-  /**
-   * Stop the thrift server, close the socket and interrupt all in progress RPCs. Calling the method
-   * twice does not induce side effects.
-   */
-  public void stop() {
-    if (heartbeatPoolServer == null) {
-      return;
-    }
-
-    heartbeatPoolServer.stop();
-    heartbeatSocket.close();
-    heartbeatClientService.shutdownNow();
-    heartbeatSocket = null;
-    heartbeatPoolServer = null;
-  }
-
-  /**
-   * An AsyncProcessor that contains the extended interfaces of a non-abstract subclass of
-   * RaftService (DataHeartbeatService or MetaHeartbeatService).
-   *
-   * @return the TProcessor
-   */
-  abstract TProcessor getProcessor();
-
-  /**
-   * A socket that will be used to establish a thrift server to listen to RPC heartbeat requests.
-   * DataHeartServer and MetaHeartServer use different port, so this is to be determined.
-   *
-   * @return TServerTransport
-   * @throws TTransportException
-   */
-  abstract TServerTransport getHeartbeatServerSocket() throws TTransportException;
-
-  /**
-   * Each thrift RPC request will be processed in a separate thread and this will return the name
-   * prefix of such threads. This is used to fast distinguish DataHeartbeatServer and
-   * MetaHeartbeatServer in the logs for the sake of debug.
-   *
-   * @return name prefix of RPC processing threads.
-   */
-  abstract String getClientThreadPrefix();
-
-  /**
-   * The thrift server will be run in a separate thread, and this will be its name. It help you
-   * locate the desired logs quickly when debugging.
-   *
-   * @return The name of the thread running the thrift server.
-   */
-  abstract String getServerClientName();
-
-  private TServer getSyncHeartbeatServer() throws TTransportException {
-    heartbeatSocket = getHeartbeatServerSocket();
-    return ClusterUtils.createTThreadPoolServer(
-        heartbeatSocket, getClientThreadPrefix(), getProcessor(), heartbeatProtocolFactory);
-  }
-
-  private TServer getAsyncHeartbeatServer() throws TTransportException {
-    heartbeatSocket = getHeartbeatServerSocket();
-    int maxConcurrentClientNum =
-        Math.max(CommonUtils.getCpuCores(), config.getMaxConcurrentClientNum());
-    Args poolArgs =
-        new Args((TNonblockingServerTransport) heartbeatSocket)
-            .maxWorkerThreads(maxConcurrentClientNum)
-            .minWorkerThreads(CommonUtils.getCpuCores());
-
-    poolArgs.executorService(
-        new ThreadPoolExecutor(
-            poolArgs.minWorkerThreads,
-            poolArgs.maxWorkerThreads,
-            poolArgs.getStopTimeoutVal(),
-            poolArgs.getStopTimeoutUnit(),
-            new SynchronousQueue<>(),
-            new ThreadFactory() {
-              private AtomicLong threadIndex = new AtomicLong(0);
-
-              @Override
-              public Thread newThread(Runnable r) {
-                return new Thread(r, getClientThreadPrefix() + threadIndex.incrementAndGet());
-              }
-            }));
-    poolArgs.processor(getProcessor());
-    poolArgs.protocolFactory(heartbeatProtocolFactory);
-    // async service requires FramedTransport
-    poolArgs.transportFactory(RpcTransportFactory.INSTANCE);
-
-    return new THsHaServer(poolArgs);
-  }
-
-  private void establishHeartbeatServer() throws TTransportException {
-    logger.info("Cluster node's heartbeat {} begins to set up", thisNode);
-
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-      heartbeatPoolServer = getAsyncHeartbeatServer();
-    } else {
-      heartbeatPoolServer = getSyncHeartbeatServer();
-    }
-
-    heartbeatClientService =
-        Executors.newSingleThreadExecutor(r -> new Thread(r, getServerClientName()));
-    heartbeatClientService.submit(() -> heartbeatPoolServer.serve());
-
-    logger.info("[{}] Cluster node's heartbeat {} is up", getServerClientName(), thisNode);
-  }
-}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/HeartbeatThread.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/HeartbeatThread.java
index bc5e8d0..38a314d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/HeartbeatThread.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/HeartbeatThread.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.cluster.server.heartbeat;
 
+import org.apache.iotdb.cluster.ClusterIoTDB;
 import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.rpc.thrift.ElectionRequest;
@@ -28,11 +29,9 @@ 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.ElectionHandler;
 import org.apache.iotdb.cluster.server.handlers.caller.HeartbeatHandler;
 import org.apache.iotdb.cluster.server.member.RaftMember;
-import org.apache.iotdb.cluster.utils.ClientUtils;
 
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransportException;
@@ -84,7 +83,7 @@ public class HeartbeatThread implements Runnable {
             // send heartbeats to the followers
             sendHeartbeats();
             synchronized (localMember.getHeartBeatWaitObject()) {
-              localMember.getHeartBeatWaitObject().wait(RaftServer.getHeartbeatIntervalMs());
+              localMember.getHeartBeatWaitObject().wait(ClusterConstant.getHeartbeatIntervalMs());
             }
             hasHadLeader = true;
             break;
@@ -92,8 +91,9 @@ public class HeartbeatThread implements Runnable {
             // check if heartbeat times out
             long heartbeatInterval =
                 System.currentTimeMillis() - localMember.getLastHeartbeatReceivedTime();
+
             long randomElectionTimeout =
-                RaftServer.getElectionTimeoutMs() + getElectionRandomWaitMs();
+                ClusterConstant.getElectionTimeoutMs() + getElectionRandomWaitMs();
             if (heartbeatInterval >= randomElectionTimeout) {
               // the leader is considered dead, an election will be started in the next loop
               logger.info("{}: The leader {} timed out", memberName, localMember.getLeader());
@@ -234,18 +234,19 @@ public class HeartbeatThread implements Runnable {
                   HeartBeatResponse heartBeatResponse = client.sendHeartbeat(req);
                   heartbeatHandler.onComplete(heartBeatResponse);
                 } catch (TTransportException e) {
-                  logger.warn(
-                      memberName
-                          + ": Cannot send heartbeat to node "
-                          + node.toString()
-                          + " due to network",
-                      e);
+                  if (ClusterIoTDB.getInstance().shouldPrintClientConnectionErrorStack()) {
+                    logger.warn(
+                        "{}: Cannot send heartbeat to node {} due to network", memberName, node, e);
+                  } else {
+                    logger.warn(
+                        "{}: Cannot send heartbeat to node {} due to network", memberName, node);
+                  }
                   client.getInputProtocol().getTransport().close();
                 } catch (Exception e) {
                   logger.warn(
                       memberName + ": Cannot send heart beat to node " + node.toString(), e);
                 } finally {
-                  ClientUtils.putBackSyncHeartbeatClient(client);
+                  localMember.returnSyncClient(client);
                 }
               }
             });
@@ -329,8 +330,8 @@ public class HeartbeatThread implements Runnable {
         logger.info(
             "{}: Wait for {}ms until election time out",
             memberName,
-            RaftServer.getElectionTimeoutMs());
-        localMember.getTerm().wait(RaftServer.getElectionTimeoutMs());
+            ClusterConstant.getElectionTimeoutMs());
+        localMember.getTerm().wait(ClusterConstant.getElectionTimeoutMs());
       } catch (InterruptedException e) {
         logger.info(
             "{}: Unexpected interruption when waiting the result of election {}",
@@ -430,7 +431,7 @@ public class HeartbeatThread implements Runnable {
                 } catch (Exception e) {
                   handler.onError(e);
                 } finally {
-                  ClientUtils.putBackSyncHeartbeatClient(client);
+                  localMember.returnSyncClient(client);
                 }
               }
             });
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatServer.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatServer.java
deleted file mode 100644
index ed99c3d..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatServer.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.cluster.server.heartbeat;
-
-import org.apache.iotdb.cluster.config.ClusterDescriptor;
-import org.apache.iotdb.cluster.rpc.thrift.Node;
-import org.apache.iotdb.cluster.rpc.thrift.TSMetaService.AsyncProcessor;
-import org.apache.iotdb.cluster.rpc.thrift.TSMetaService.Processor;
-import org.apache.iotdb.cluster.server.MetaClusterServer;
-import org.apache.iotdb.cluster.utils.ClusterUtils;
-
-import org.apache.thrift.TProcessor;
-import org.apache.thrift.transport.TNonblockingServerSocket;
-import org.apache.thrift.transport.TServerSocket;
-import org.apache.thrift.transport.TServerTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.InetSocketAddress;
-
-public class MetaHeartbeatServer extends HeartbeatServer {
-  private static Logger logger = LoggerFactory.getLogger(MetaHeartbeatServer.class);
-
-  private MetaClusterServer metaClusterServer;
-
-  /** Do not use this method for initialization */
-  private MetaHeartbeatServer() {}
-
-  public MetaHeartbeatServer(Node thisNode, MetaClusterServer metaClusterServer) {
-    super(thisNode);
-    this.metaClusterServer = metaClusterServer;
-  }
-
-  @Override
-  TProcessor getProcessor() {
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-      return new AsyncProcessor<>(metaClusterServer);
-    } else {
-      return new Processor<>(metaClusterServer);
-    }
-  }
-
-  @Override
-  TServerTransport getHeartbeatServerSocket() throws TTransportException {
-    logger.info(
-        "[{}] Cluster node will listen {}:{}",
-        getServerClientName(),
-        config.getInternalIp(),
-        config.getInternalMetaPort() + ClusterUtils.META_HEARTBEAT_PORT_OFFSET);
-    if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
-      return new TNonblockingServerSocket(
-          new InetSocketAddress(
-              config.getInternalIp(),
-              config.getInternalMetaPort() + ClusterUtils.META_HEARTBEAT_PORT_OFFSET),
-          getConnectionTimeoutInMS());
-    } else {
-      return new TServerSocket(
-          new InetSocketAddress(
-              config.getInternalIp(),
-              config.getInternalMetaPort() + ClusterUtils.META_HEARTBEAT_PORT_OFFSET));
-    }
-  }
-
-  @Override
-  String getClientThreadPrefix() {
-    return "MetaHeartbeatClientThread-";
-  }
-
-  @Override
-  String getServerClientName() {
-    return "MetaHeartbeatServerThread-";
-  }
-}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatThread.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatThread.java
index 4036244..f8ce0b8 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatThread.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/heartbeat/MetaHeartbeatThread.java
@@ -77,6 +77,9 @@ public class MetaHeartbeatThread extends HeartbeatThread {
     super.startElection();
 
     if (localMetaMember.getCharacter() == NodeCharacter.LEADER) {
+      // if the node becomes the leader,
+      localMetaMember.buildMetaEngineServiceIfNotReady();
+
       // A new raft leader needs to have at least one log in its term for committing logs with older
       // terms.
       // In the meta group, log frequency is very low. When the leader is changed whiling changing
@@ -85,6 +88,9 @@ public class MetaHeartbeatThread extends HeartbeatThread {
       localMetaMember
           .getAppendLogThreadPool()
           .submit(() -> localMetaMember.processEmptyContentLog());
+      // this is a risk that (1) put a task into a pool
+      // and (2) the task puts more sub-tasks into the same pool, especially the task can only
+      // terminal when all sub-tasks finish.
     }
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
index f4a750b..f611683 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
@@ -19,13 +19,8 @@
 
 package org.apache.iotdb.cluster.server.member;
 
-import org.apache.iotdb.cluster.client.async.AsyncClientPool;
-import org.apache.iotdb.cluster.client.async.AsyncDataClient;
-import org.apache.iotdb.cluster.client.async.AsyncDataClient.SingleManagerFactory;
-import org.apache.iotdb.cluster.client.async.AsyncDataHeartbeatClient;
-import org.apache.iotdb.cluster.client.sync.SyncClientPool;
-import org.apache.iotdb.cluster.client.sync.SyncDataClient;
-import org.apache.iotdb.cluster.client.sync.SyncDataHeartbeatClient;
+import org.apache.iotdb.cluster.client.ClientCategory;
+import org.apache.iotdb.cluster.client.ClientManager;
 import org.apache.iotdb.cluster.config.ClusterConstant;
 import org.apache.iotdb.cluster.config.ClusterDescriptor;
 import org.apache.iotdb.cluster.exception.CheckConsistencyException;
@@ -76,6 +71,8 @@ import org.apache.iotdb.cluster.server.monitor.Timer;
 import org.apache.iotdb.cluster.server.monitor.Timer.Statistic;
 import org.apache.iotdb.cluster.utils.IOUtils;
 import org.apache.iotdb.cluster.utils.StatusUtils;
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor.TimePartitionFilter;
@@ -93,6 +90,7 @@ import org.apache.iotdb.db.qp.physical.crud.*;
 import org.apache.iotdb.db.qp.physical.sys.FlushPlan;
 import org.apache.iotdb.db.qp.physical.sys.LogPlan;
 import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.service.JMXService;
 import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.service.rpc.thrift.EndPoint;
@@ -122,12 +120,13 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
 import static org.apache.iotdb.cluster.config.ClusterConstant.THREAD_POLL_WAIT_TERMINATION_TIME_S;
 
-public class DataGroupMember extends RaftMember {
+public class DataGroupMember extends RaftMember implements DataGroupMemberMBean {
+
+  private final String mbeanName;
 
   private static final Logger logger = LoggerFactory.getLogger(DataGroupMember.class);
 
@@ -173,33 +172,49 @@ public class DataGroupMember extends RaftMember {
   @TestOnly
   public DataGroupMember(PartitionGroup nodes) {
     // constructor for test
+    this.name =
+        "Data-"
+            + nodes.getHeader().getNode().getInternalIp()
+            + "-"
+            + nodes.getHeader().getNode().getDataPort()
+            + "-raftId-"
+            + nodes.getId()
+            + "";
     allNodes = nodes;
+    mbeanName =
+        String.format(
+            "%s:%s=%s%d",
+            "org.apache.iotdb.cluster.service",
+            IoTDBConstant.JMX_TYPE,
+            "DataMember",
+            getRaftGroupId());
     setQueryManager(new ClusterQueryManager());
     localQueryExecutor = new LocalQueryExecutor(this);
     lastAppliedPartitionTableVersion = new LastAppliedPatitionTableVersion(getMemberDir());
   }
 
-  DataGroupMember(
-      TProtocolFactory factory,
-      PartitionGroup nodes,
-      Node thisNode,
-      MetaGroupMember metaGroupMember) {
+  DataGroupMember(TProtocolFactory factory, PartitionGroup nodes, MetaGroupMember metaGroupMember) {
+    // The name is used in JMX, so we have to avoid to use "(" "," "=" ")"
     super(
-        "Data("
+        "Data-"
             + nodes.getHeader().getNode().getInternalIp()
-            + ":"
-            + nodes.getHeader().getNode().getMetaPort()
-            + ", raftId="
+            + "-"
+            + nodes.getHeader().getNode().getDataPort()
+            + "-raftId-"
             + nodes.getId()
-            + ")",
-        new AsyncClientPool(new AsyncDataClient.FactoryAsync(factory)),
-        new SyncClientPool(new SyncDataClient.FactorySync(factory)),
-        new AsyncClientPool(new AsyncDataHeartbeatClient.FactoryAsync(factory)),
-        new SyncClientPool(new SyncDataHeartbeatClient.FactorySync(factory)),
-        new AsyncClientPool(new SingleManagerFactory(factory)));
-    this.thisNode = thisNode;
+            + "",
+        new ClientManager(
+            ClusterDescriptor.getInstance().getConfig().isUseAsyncServer(),
+            ClientManager.Type.DataGroupClient));
     this.metaGroupMember = metaGroupMember;
     allNodes = nodes;
+    mbeanName =
+        String.format(
+            "%s:%s=%s%d",
+            "org.apache.iotdb.cluster.service",
+            IoTDBConstant.JMX_TYPE,
+            "DataMember",
+            getRaftGroupId());
     setQueryManager(new ClusterQueryManager());
     slotManager = new SlotManager(ClusterConstant.SLOT_NUM, getMemberDir(), getName());
     dataLogApplier = new DataLogApplier(metaGroupMember, this);
@@ -226,9 +241,13 @@ public class DataGroupMember extends RaftMember {
     if (heartBeatService != null) {
       return;
     }
+    logger.info("Starting DataGroupMember {}... RaftGroupID: {}", name, getRaftGroupId());
+    JMXService.registerMBean(this, mbeanName);
     super.start();
     heartBeatService.submit(new DataHeartbeatThread(this));
-    pullSnapshotService = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
+    pullSnapshotService =
+        IoTDBThreadPoolFactory.newFixedThreadPool(
+            Runtime.getRuntime().availableProcessors(), "pullSnapshot");
     pullSnapshotHintService = new PullSnapshotHintService(this);
     pullSnapshotHintService.start();
     resumePullSnapshotTasks();
@@ -240,7 +259,8 @@ public class DataGroupMember extends RaftMember {
    */
   @Override
   public void stop() {
-    logger.info("{}: stopping...", name);
+    logger.info("Stopping DataGroupMember {}... RaftGroupID: {}", name, getRaftGroupId());
+    JMXService.deregisterMBean(mbeanName);
     super.stop();
     if (pullSnapshotService != null) {
       pullSnapshotService.shutdownNow();
@@ -301,13 +321,13 @@ public class DataGroupMember extends RaftMember {
     private TProtocolFactory protocolFactory;
     private MetaGroupMember metaGroupMember;
 
-    Factory(TProtocolFactory protocolFactory, MetaGroupMember metaGroupMember) {
+    public Factory(TProtocolFactory protocolFactory, MetaGroupMember metaGroupMember) {
       this.protocolFactory = protocolFactory;
       this.metaGroupMember = metaGroupMember;
     }
 
-    public DataGroupMember create(PartitionGroup partitionGroup, Node thisNode) {
-      return new DataGroupMember(protocolFactory, partitionGroup, thisNode, metaGroupMember);
+    public DataGroupMember create(PartitionGroup partitionGroup) {
+      return new DataGroupMember(protocolFactory, partitionGroup, metaGroupMember);
     }
   }
 
@@ -743,6 +763,16 @@ public class DataGroupMember extends RaftMember {
     }
   }
 
+  @Override
+  ClientCategory getClientCategory() {
+    return ClientCategory.DATA;
+  }
+
+  @Override
+  public String getMBeanName() {
+    return mbeanName;
+  }
+
   private void handleChangeMembershipLogWithoutRaft(Log log) {
     if (log instanceof AddNodeLog) {
       if (!metaGroupMember
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMemberMBean.java
similarity index 56%
copy from cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMemberMBean.java
index 6bfe9003..964b5c4 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientFactory.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMemberMBean.java
@@ -17,24 +17,9 @@
  * under the License.
  */
 
-package org.apache.iotdb.cluster.client.sync;
+package org.apache.iotdb.cluster.server.member;
 
-import org.apache.iotdb.cluster.rpc.thrift.Node;
-import org.apache.iotdb.cluster.rpc.thrift.RaftService;
+public interface DataGroupMemberMBean extends RaftMemberMBean {
 
-import org.apache.thrift.transport.TTransportException;
-
-import java.io.IOException;
-
-public interface SyncClientFactory {
-
-  /**
-   * 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;
+  String getCharacterAsString();
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
index bacacab..87e5aeb 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
... 11441 lines suppressed ...