You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2022/04/20 14:47:39 UTC

[iotdb] branch master updated: [IOTDB-2942] [IOTDB-2944] Delete duplicate data structure in node-commons and extend TDataNodeLocation (#5592)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 7ce6033609 [IOTDB-2942] [IOTDB-2944] Delete duplicate data structure in node-commons and extend TDataNodeLocation (#5592)
7ce6033609 is described below

commit 7ce6033609f0df3c7b809dfc048c1bfcb47aefb7
Author: YongzaoDan <33...@users.noreply.github.com>
AuthorDate: Wed Apr 20 22:47:33 2022 +0800

    [IOTDB-2942] [IOTDB-2944] Delete duplicate data structure in node-commons and extend TDataNodeLocation (#5592)
---
 client-cpp/pom.xml                                 |   7 +-
 client-py/pom.xml                                  |   8 +
 confignode/pom.xml                                 |  10 -
 .../resources/conf/iotdb-confignode.properties     |  18 +-
 .../iotdb/confignode/cli/TemporaryClient.java      |  54 ++++--
 .../iotdb/confignode/conf/ConfigNodeConf.java      |  59 +++---
 .../confignode/conf/ConfigNodeDescriptor.java      |  22 ++-
 .../response/DataNodeConfigurationDataSet.java     |   6 +-
 ...oDataSet.java => DataNodeLocationsDataSet.java} |  34 +---
 .../consensus/response/DataPartitionDataSet.java   |  53 +-----
 .../consensus/response/SchemaPartitionDataSet.java |  31 +--
 .../response/StorageGroupSchemaDataSet.java        |  11 +-
 .../iotdb/confignode/manager/ConfigManager.java    |  10 +-
 .../iotdb/confignode/manager/ConsensusManager.java |   6 +-
 .../iotdb/confignode/manager/DataNodeManager.java  |  25 +--
 .../iotdb/confignode/manager/PartitionManager.java |  48 ++---
 .../iotdb/confignode/manager/RegionManager.java    |  85 +++++----
 .../confignode/partition/StorageGroupSchema.java   | 121 ------------
 .../persistence/DataNodeInfoPersistence.java       |  39 ++--
 .../persistence/PartitionInfoPersistence.java      |  22 +--
 .../persistence/RegionInfoPersistence.java         |  54 +++---
 .../physical/crud/CreateDataPartitionPlan.java     |  44 ++---
 .../physical/crud/CreateRegionsPlan.java           |  21 ++-
 .../physical/crud/CreateSchemaPartitionPlan.java   |  27 +--
 .../crud/GetOrCreateDataPartitionPlan.java         |  40 ++--
 .../crud/GetOrCreateSchemaPartitionPlan.java       |  22 ++-
 .../iotdb/confignode/physical/sys/AuthorPlan.java  |  24 +--
 .../physical/sys/RegisterDataNodePlan.java         |  32 ++--
 .../physical/sys/SetStorageGroupPlan.java          |  17 +-
 .../server/ConfigNodeRPCServerProcessor.java       |  50 +++--
 .../iotdb/confignode/cli/TemporaryClientDemo.java  |   4 +-
 .../confignode/consensus/RatisConsensusDemo.java   | 151 ---------------
 .../manager/ConfigManagerManualTest.java           | 133 -------------
 ...serializeUT.java => PhysicalPlanSerDeTest.java} | 110 +++++++----
 .../server/ConfigNodeRPCServerProcessorTest.java   | 209 +++++++++------------
 consensus/pom.xml                                  |   5 -
 .../apache/iotdb/consensus/ConsensusFactory.java   |   7 +-
 .../org/apache/iotdb/consensus/common/Peer.java    |   8 +-
 .../iotdb/consensus/ratis/RatisConsensus.java      |   7 +-
 .../iotdb/consensus/ratis/RequestMessage.java      |   2 +-
 .../org/apache/iotdb/consensus/ratis/Utils.java    |  16 +-
 .../consensus/standalone/StandAloneConsensus.java  |   6 +-
 .../iotdb/consensus/ratis/RatisConsensusTest.java  |   8 +-
 .../standalone/StandAloneConsensusTest.java        |  31 +--
 node-commons/pom.xml                               |  10 +
 .../iotdb/commons/cluster/DataNodeLocation.java    |  81 --------
 .../org/apache/iotdb/commons/cluster/Endpoint.java |  92 ---------
 .../iotdb/commons/consensus/ConsensusGroupId.java  |  44 +++--
 .../iotdb/commons/consensus/DataRegionId.java      |  21 +--
 .../apache/iotdb/commons/consensus/GroupType.java  |  27 ---
 .../iotdb/commons/consensus/PartitionRegionId.java |  21 +--
 .../iotdb/commons/consensus/SchemaRegionId.java    |  21 +--
 .../iotdb/commons/partition/DataPartition.java     |  69 +++----
 .../commons/partition/DataPartitionQueryParam.java |   8 +-
 .../apache/iotdb/commons/partition/Partition.java  |   3 +-
 .../iotdb/commons/partition/RegionReplicaSet.java  | 141 --------------
 .../iotdb/commons/partition/SchemaPartition.java   |  31 +--
 .../commons/partition/SeriesPartitionSlot.java     |  59 ------
 .../iotdb/commons/partition/TimePartitionSlot.java |  64 -------
 .../executor/SeriesPartitionExecutor.java          |   4 +-
 .../partition/executor/hash/APHashExecutor.java    |   6 +-
 .../partition/executor/hash/BKDRHashExecutor.java  |   6 +-
 .../partition/executor/hash/JSHashExecutor.java    |   6 +-
 .../partition/executor/hash/SDBMHashExecutor.java  |   6 +-
 .../commons/utils/BasicStructureSerDeUtil.java     |   6 +-
 .../apache/iotdb/commons/utils/CommonUtils.java    |  12 +-
 .../commons/utils/ThriftCommonsSerDeUtils.java     | 144 ++++++++++++++
 .../commons/utils/ThriftConfigNodeSerDeUtils.java  |  87 +++++++++
 .../apache/iotdb/commons/ConsensusGroupIdTest.java |  23 +--
 .../commons/utils/BasicStructureSerDeUtilTest.java |  18 +-
 .../commons/utils/ThriftCommonsSerDeUtilsTest.java | 116 ++++++++++++
 .../utils/ThriftConfigNodeSerDeUtilsTest.java      |  67 +++++++
 pom.xml                                            |   2 +
 server/pom.xml                                     |  15 +-
 .../apache/iotdb/db/client/ConfigNodeClient.java   |  23 +--
 .../apache/iotdb/db/consensus/ConsensusImpl.java   |   4 +-
 .../org/apache/iotdb/db/engine/StorageEngine.java  |   6 +-
 .../iotdb/db/mpp/buffer/DataBlockManager.java      |   6 +-
 .../mpp/buffer/DataBlockServiceClientFactory.java  |   6 +-
 .../iotdb/db/mpp/buffer/IDataBlockManager.java     |   6 +-
 .../iotdb/db/mpp/common/MPPQueryContext.java       |   8 +-
 .../iotdb/db/mpp/common/ResultNodeContext.java     |   8 +-
 .../apache/iotdb/db/mpp/execution/Coordinator.java |   8 +-
 .../iotdb/db/mpp/execution/QueryExecution.java     |   4 +-
 .../mpp/execution/config/SetStorageGroupTask.java  |   6 +-
 .../scheduler/AbstractFragInsStateTracker.java     |   4 +-
 .../scheduler/InternalServiceClientFactory.java    |   6 +-
 .../scheduler/SimpleFragInstanceDispatcher.java    |  12 +-
 .../execution/scheduler/SimpleQueryTerminator.java |  10 +-
 .../apache/iotdb/db/mpp/sql/analyze/Analysis.java  |   4 +-
 .../mpp/sql/analyze/ClusterPartitionFetcher.java   |  26 ++-
 .../db/mpp/sql/analyze/ClusterSchemaFetcher.java   |   6 +-
 .../mpp/sql/analyze/FakePartitionFetcherImpl.java  | 114 ++++++-----
 .../db/mpp/sql/planner/DistributionPlanner.java    |  28 +--
 .../db/mpp/sql/planner/LocalExecutionPlanner.java  |  10 +-
 .../db/mpp/sql/planner/plan/FragmentInstance.java  |  33 ++--
 .../db/mpp/sql/planner/plan/PlanFragment.java      |   8 +-
 .../plan/SimpleFragmentParallelPlanner.java        |   4 +-
 .../sql/planner/plan/node/PlanGraphPrinter.java    |   2 +-
 .../mpp/sql/planner/plan/node/WritePlanNode.java   |   4 +-
 .../plan/node/metedata/read/SchemaScanNode.java    |   8 +-
 .../node/metedata/write/CreateTimeSeriesNode.java  |  10 +-
 .../planner/plan/node/process/ExchangeNode.java    |  13 +-
 .../planner/plan/node/sink/FragmentSinkNode.java   |  13 +-
 .../plan/node/source/SeriesAggregateScanNode.java  |  13 +-
 .../planner/plan/node/source/SeriesScanNode.java   |  16 +-
 .../sql/planner/plan/node/source/SourceNode.java   |   6 +-
 .../plan/node/write/InsertMultiTabletsNode.java    |   6 +-
 .../sql/planner/plan/node/write/InsertNode.java    |  10 +-
 .../sql/planner/plan/node/write/InsertRowNode.java |   4 +-
 .../planner/plan/node/write/InsertRowsNode.java    |   6 +-
 .../plan/node/write/InsertRowsOfOneDeviceNode.java |   6 +-
 .../planner/plan/node/write/InsertTabletNode.java  |  14 +-
 .../mpp/sql/statement/crud/InsertRowStatement.java |   4 +-
 .../crud/InsertRowsOfOneDeviceStatement.java       |   6 +-
 .../sql/statement/crud/InsertTabletStatement.java  |   8 +-
 .../java/org/apache/iotdb/db/service/DataNode.java |  23 ++-
 .../service/thrift/impl/InternalServiceImpl.java   |  35 ++--
 .../db/mpp/sql/plan/DistributionPlannerTest.java   | 155 ++++++++-------
 .../db/mpp/sql/plan/FragmentInstanceSerdeTest.java |  48 +++--
 .../iotdb/db/mpp/sql/plan/QueryPlannerTest.java    |   5 +-
 .../read/DeviceSchemaScanNodeSerdeTest.java        |   6 +-
 .../read/TimeSeriesSchemaScanNodeSerdeTest.java    |   6 +-
 .../plan/node/process/AggregateNodeSerdeTest.java  |   8 +-
 .../node/process/DeviceMergeNodeSerdeTest.java     |   8 +-
 .../plan/node/process/ExchangeNodeSerdeTest.java   |  14 +-
 .../sql/plan/node/process/FillNodeSerdeTest.java   |   8 +-
 .../sql/plan/node/process/FilterNodeSerdeTest.java |   8 +-
 .../plan/node/process/FilterNullNodeSerdeTest.java |   8 +-
 .../node/process/GroupByLevelNodeSerdeTest.java    |   8 +-
 .../sql/plan/node/process/LimitNodeSerdeTest.java  |   8 +-
 .../sql/plan/node/process/OffsetNodeSerdeTest.java |  17 +-
 .../sql/plan/node/process/SortNodeSerdeTest.java   |   8 +-
 .../plan/node/process/TimeJoinNodeSerdeTest.java   |   8 +-
 .../plan/node/sink/FragmentSinkNodeSerdeTest.java  |   4 +-
 .../source/SeriesAggregateScanNodeSerdeTest.java   |   8 +-
 .../plan/node/source/SeriesScanNodeSerdeTest.java  |   8 +-
 .../iotdb/db/service/InternalServiceImplTest.java  |  57 +++---
 thrift-cluster/pom.xml                             |   5 +
 {thrift-cluster => thrift-commons}/pom.xml         |  11 +-
 .../src/main/thrift/common.thrift                  |  29 ++-
 thrift-confignode/pom.xml                          |   2 +-
 .../src/main/thrift/confignode.thrift              |  38 ++--
 thrift/pom.xml                                     |   5 +
 thrift/src/main/thrift/mpp.thrift                  |   7 +-
 145 files changed, 1790 insertions(+), 2171 deletions(-)

diff --git a/client-cpp/pom.xml b/client-cpp/pom.xml
index f5428aaf95..a6feb1d413 100644
--- a/client-cpp/pom.xml
+++ b/client-cpp/pom.xml
@@ -50,6 +50,11 @@
             <artifactId>client-cpp-tools-thrift</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift-commons</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
     <profiles>
         <profile>
@@ -164,7 +169,7 @@
                                             <destinationFile>${project.build.directory}/thrift/rpc.thrift</destinationFile>
                                         </fileSet>
                                         <fileSet>
-                                            <sourceFile>../thrift/src/main/thrift/common.thrift</sourceFile>
+                                            <sourceFile>../thrift-commons/src/main/thrift/common.thrift</sourceFile>
                                             <destinationFile>${project.build.directory}/thrift/common.thrift</destinationFile>
                                         </fileSet>
                                     </fileSets>
diff --git a/client-py/pom.xml b/client-py/pom.xml
index fa7cce5e17..a5edfb6d7d 100644
--- a/client-py/pom.xml
+++ b/client-py/pom.xml
@@ -36,6 +36,11 @@
             <artifactId>iotdb-thrift</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift-commons</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
@@ -92,6 +97,9 @@
                                 <resource>
                                     <directory>${basedir}/../thrift/target/generated-sources-python/iotdb/thrift/</directory>
                                 </resource>
+                                <resource>
+                                    <directory>${basedir}/../thrift-commons/target/generated-sources-python/iotdb/thrift/</directory>
+                                </resource>
                             </resources>
                         </configuration>
                     </execution>
diff --git a/confignode/pom.xml b/confignode/pom.xml
index 651e5aa2c0..7f1ab3ff96 100644
--- a/confignode/pom.xml
+++ b/confignode/pom.xml
@@ -40,16 +40,6 @@
             <artifactId>service-rpc</artifactId>
             <version>${project.version}</version>
         </dependency>
-        <dependency>
-            <groupId>org.apache.iotdb</groupId>
-            <artifactId>iotdb-thrift-confignode</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.iotdb</groupId>
-            <artifactId>node-commons</artifactId>
-            <version>${project.version}</version>
-        </dependency>
         <dependency>
             <groupId>org.apache.iotdb</groupId>
             <artifactId>iotdb-consensus</artifactId>
diff --git a/confignode/src/assembly/resources/conf/iotdb-confignode.properties b/confignode/src/assembly/resources/conf/iotdb-confignode.properties
index a48fbdbbc8..dccb57f8ba 100644
--- a/confignode/src/assembly/resources/conf/iotdb-confignode.properties
+++ b/confignode/src/assembly/resources/conf/iotdb-confignode.properties
@@ -152,18 +152,18 @@ data_node_consensus_protocol_class=org.apache.iotdb.consensus.standalone.StandAl
 # Datatype: long
 # default_ttl=36000000
 
-####################
-### Region Configuration
-####################
+# Default number of SchemaRegion replicas
+# Datatype: int
+# schema_replication_factor=3
 
-# The number of replicas of each region
+# Default number of DataRegion replicas
 # Datatype: int
-region_replica_count=1
+# data_replication_factor=3
 
-# The number of SchemaRegions of each StorageGroup
+# The initial number of SchemaRegions of each StorageGroup
 # Datatype: int
-# schema_region_count=1
+# initial_schema_region_count=1
 
-# The number of DataRegions of each StorageGroup
+# The initial number of DataRegions of each StorageGroup
 # Datatype: int
-# data_region_count=1
\ No newline at end of file
+# initial_data_region_count=1
\ No newline at end of file
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/cli/TemporaryClient.java b/confignode/src/main/java/org/apache/iotdb/confignode/cli/TemporaryClient.java
index 437f75a536..de570eb36f 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/cli/TemporaryClient.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/cli/TemporaryClient.java
@@ -18,9 +18,9 @@
  */
 package org.apache.iotdb.confignode.cli;
 
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
 import org.apache.iotdb.confignode.persistence.DataNodeInfoPersistence;
 import org.apache.iotdb.mpp.rpc.thrift.InternalService;
 import org.apache.iotdb.mpp.rpc.thrift.TCreateDataRegionReq;
@@ -59,7 +59,7 @@ public class TemporaryClient {
     this.clients = new HashMap<>();
   }
 
-  public void buildClient(int dataNodeId, Endpoint endpoint) {
+  public void buildClient(int dataNodeId, TEndPoint endpoint) {
     for (int i = 0; i < retryNum; i++) {
       try {
         TTransport transport =
@@ -83,20 +83,22 @@ public class TemporaryClient {
   }
 
   private TCreateSchemaRegionReq genCreateSchemaRegionReq(
-      String storageGroup, RegionReplicaSet regionReplicaSet) {
+      String storageGroup, TRegionReplicaSet regionReplicaSet) {
     TCreateSchemaRegionReq req = new TCreateSchemaRegionReq();
     req.setStorageGroup(storageGroup);
-    req.setRegionReplicaSet(regionReplicaSet.convertToRPCTRegionReplicaSet());
+    req.setRegionReplicaSet(regionReplicaSet);
     return req;
   }
 
   public void createSchemaRegion(
-      int dataNodeId, String storageGroup, RegionReplicaSet regionReplicaSet) {
+      int dataNodeId, String storageGroup, TRegionReplicaSet regionReplicaSet) {
 
     if (clients.get(dataNodeId) == null) {
       buildClient(
           dataNodeId,
-          DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint());
+          DataNodeInfoPersistence.getInstance()
+              .getOnlineDataNode(dataNodeId)
+              .getInternalEndPoint());
     }
 
     TCreateSchemaRegionReq req = genCreateSchemaRegionReq(storageGroup, regionReplicaSet);
@@ -107,19 +109,25 @@ public class TemporaryClient {
           if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
             LOGGER.info(
                 "Create SchemaRegion on DataNode: {} success",
-                DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint());
+                DataNodeInfoPersistence.getInstance()
+                    .getOnlineDataNode(dataNodeId)
+                    .getInternalEndPoint());
             return;
           } else {
             LOGGER.error(
                 "Create SchemaRegion on DataNode: {} failed, {}. Retrying...",
-                DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint(),
+                DataNodeInfoPersistence.getInstance()
+                    .getOnlineDataNode(dataNodeId)
+                    .getInternalEndPoint(),
                 status);
           }
         } catch (TException e) {
           // TODO: Handler SocketTimeOutException
           LOGGER.error(
               "Create SchemaRegion on DataNode: {} failed, {}. Retrying...",
-              DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint(),
+              DataNodeInfoPersistence.getInstance()
+                  .getOnlineDataNode(dataNodeId)
+                  .getInternalEndPoint(),
               e.toString());
           try {
             TimeUnit.MILLISECONDS.sleep(retryWait);
@@ -131,25 +139,27 @@ public class TemporaryClient {
     }
     LOGGER.error(
         "Create SchemaRegion on DataNode: {} failed.",
-        DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint());
+        DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getInternalEndPoint());
   }
 
   private TCreateDataRegionReq genCreateDataRegionReq(
-      String storageGroup, RegionReplicaSet regionReplicaSet, long TTL) {
+      String storageGroup, TRegionReplicaSet regionReplicaSet, long TTL) {
     TCreateDataRegionReq req = new TCreateDataRegionReq();
     req.setStorageGroup(storageGroup);
-    req.setRegionReplicaSet(regionReplicaSet.convertToRPCTRegionReplicaSet());
+    req.setRegionReplicaSet(regionReplicaSet);
     req.setTtl(TTL);
     return req;
   }
 
   public void createDataRegion(
-      int dataNodeId, String storageGroup, RegionReplicaSet regionReplicaSet, long TTL) {
+      int dataNodeId, String storageGroup, TRegionReplicaSet regionReplicaSet, long TTL) {
 
     if (clients.get(dataNodeId) == null) {
       buildClient(
           dataNodeId,
-          DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint());
+          DataNodeInfoPersistence.getInstance()
+              .getOnlineDataNode(dataNodeId)
+              .getInternalEndPoint());
     }
 
     TCreateDataRegionReq req = genCreateDataRegionReq(storageGroup, regionReplicaSet, TTL);
@@ -160,19 +170,25 @@ public class TemporaryClient {
           if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
             LOGGER.info(
                 "Create DataRegion on DataNode: {} success",
-                DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint());
+                DataNodeInfoPersistence.getInstance()
+                    .getOnlineDataNode(dataNodeId)
+                    .getInternalEndPoint());
             return;
           } else {
             LOGGER.error(
                 "Create DataRegion on DataNode: {} failed, {}. Retrying...",
-                DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint(),
+                DataNodeInfoPersistence.getInstance()
+                    .getOnlineDataNode(dataNodeId)
+                    .getInternalEndPoint(),
                 status);
           }
         } catch (TException e) {
           // TODO: Handler SocketTimeOutException
           LOGGER.error(
               "Create DataRegion on DataNode: {} failed, {}. Retrying...",
-              DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint(),
+              DataNodeInfoPersistence.getInstance()
+                  .getOnlineDataNode(dataNodeId)
+                  .getInternalEndPoint(),
               e.toString());
           try {
             TimeUnit.MILLISECONDS.sleep(retryWait);
@@ -184,7 +200,7 @@ public class TemporaryClient {
     }
     LOGGER.error(
         "Create DataRegion on DataNode: {} failed.",
-        DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint());
+        DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getInternalEndPoint());
   }
 
   private static class TemporaryClientHolder {
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConf.java b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConf.java
index 27a60ea35b..80279295a5 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConf.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConf.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.confignode.conf;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.rpc.RpcUtils;
 
@@ -43,8 +43,8 @@ public class ConfigNodeConf {
   private String dataNodeConsensusProtocolClass = "org.apache.iotdb.consensus.ratis.RatisConsensus";
 
   /** Used for building the ConfigNode consensus group */
-  private Endpoint[] configNodeGroupAddressList =
-      Collections.singletonList(new Endpoint("0.0.0.0", 22278)).toArray(new Endpoint[0]);
+  private TEndPoint[] configNodeGroupAddressList =
+      Collections.singletonList(new TEndPoint("0.0.0.0", 22278)).toArray(new TEndPoint[0]);
 
   /** Number of SeriesPartitionSlots per StorageGroup */
   private int seriesPartitionSlotNum = 10000;
@@ -90,12 +90,17 @@ public class ConfigNodeConf {
   /** Default TTL for storage groups that are not set TTL by statements, in ms. */
   private long defaultTTL = Long.MAX_VALUE;
 
-  /** The number of replicas of each region */
-  private int regionReplicaCount = 3;
-  /** The number of SchemaRegions of each StorageGroup */
-  private int schemaRegionCount = 1;
-  /** The number of DataRegions of each StorageGroup */
-  private int dataRegionCount = 1;
+  /** Default number of SchemaRegion replicas */
+  private int schemaReplicationFactor = 3;
+
+  /** Default number of DataRegion replicas */
+  private int dataReplicationFactor = 3;
+
+  /** The initial number of SchemaRegions of each StorageGroup */
+  private int initialSchemaRegionCount = 1;
+
+  /** The initial number of DataRegions of each StorageGroup */
+  private int initialDataRegionCount = 1;
 
   public ConfigNodeConf() {
     // empty constructor
@@ -237,11 +242,11 @@ public class ConfigNodeConf {
     this.dataNodeConsensusProtocolClass = dataNodeConsensusProtocolClass;
   }
 
-  public Endpoint[] getConfigNodeGroupAddressList() {
+  public TEndPoint[] getConfigNodeGroupAddressList() {
     return configNodeGroupAddressList;
   }
 
-  public void setConfigNodeGroupAddressList(Endpoint[] configNodeGroupAddressList) {
+  public void setConfigNodeGroupAddressList(TEndPoint[] configNodeGroupAddressList) {
     this.configNodeGroupAddressList = configNodeGroupAddressList;
   }
 
@@ -277,27 +282,35 @@ public class ConfigNodeConf {
     this.defaultTTL = defaultTTL;
   }
 
-  public int getRegionReplicaCount() {
-    return regionReplicaCount;
+  public int getSchemaReplicationFactor() {
+    return schemaReplicationFactor;
+  }
+
+  public void setSchemaReplicationFactor(int schemaReplicationFactor) {
+    this.schemaReplicationFactor = schemaReplicationFactor;
+  }
+
+  public int getDataReplicationFactor() {
+    return dataReplicationFactor;
   }
 
-  public void setDataRegionCount(int dataRegionCount) {
-    this.dataRegionCount = dataRegionCount;
+  public void setDataReplicationFactor(int dataReplicationFactor) {
+    this.dataReplicationFactor = dataReplicationFactor;
   }
 
-  public int getSchemaRegionCount() {
-    return schemaRegionCount;
+  public int getInitialSchemaRegionCount() {
+    return initialSchemaRegionCount;
   }
 
-  public void setSchemaRegionCount(int schemaRegionCount) {
-    this.schemaRegionCount = schemaRegionCount;
+  public void setInitialSchemaRegionCount(int initialSchemaRegionCount) {
+    this.initialSchemaRegionCount = initialSchemaRegionCount;
   }
 
-  public int getDataRegionCount() {
-    return dataRegionCount;
+  public int getInitialDataRegionCount() {
+    return initialDataRegionCount;
   }
 
-  public void setRegionReplicaCount(int regionReplicaCount) {
-    this.regionReplicaCount = regionReplicaCount;
+  public void setInitialDataRegionCount(int initialDataRegionCount) {
+    this.initialDataRegionCount = initialDataRegionCount;
   }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
index f04d9d5f44..c42e81a70b 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.confignode.conf;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.commons.exception.BadNodeUrlException;
 import org.apache.iotdb.commons.utils.CommonUtils;
 
@@ -174,25 +174,31 @@ public class ConfigNodeDescriptor {
           Long.parseLong(
               properties.getProperty("default_ttl", String.valueOf(conf.getDefaultTTL()))));
 
-      conf.setRegionReplicaCount(
+      conf.setSchemaReplicationFactor(
           Integer.parseInt(
               properties.getProperty(
-                  "region_replica_count", String.valueOf(conf.getRegionReplicaCount()))));
+                  "schema_replication_factor", String.valueOf(conf.getSchemaReplicationFactor()))));
 
-      conf.setSchemaRegionCount(
+      conf.setDataReplicationFactor(
           Integer.parseInt(
               properties.getProperty(
-                  "schema_region_count", String.valueOf(conf.getSchemaRegionCount()))));
+                  "data_replication_factor", String.valueOf(conf.getDataReplicationFactor()))));
 
-      conf.setDataRegionCount(
+      conf.setInitialSchemaRegionCount(
           Integer.parseInt(
               properties.getProperty(
-                  "data_region_count", String.valueOf(conf.getDataRegionCount()))));
+                  "initial_schema_region_count",
+                  String.valueOf(conf.getInitialSchemaRegionCount()))));
+
+      conf.setInitialDataRegionCount(
+          Integer.parseInt(
+              properties.getProperty(
+                  "initial_data_region_count", String.valueOf(conf.getInitialDataRegionCount()))));
 
       String addresses = properties.getProperty("config_node_group_address_list", "0.0.0.0:22278");
 
       String[] addressList = addresses.split(",");
-      Endpoint[] endpointList = new Endpoint[addressList.length];
+      TEndPoint[] endpointList = new TEndPoint[addressList.length];
       for (int i = 0; i < addressList.length; i++) {
         endpointList[i] = CommonUtils.parseNodeUrl(addressList[i]);
       }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeConfigurationDataSet.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeConfigurationDataSet.java
index deb1a07ead..178210be01 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeConfigurationDataSet.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeConfigurationDataSet.java
@@ -43,10 +43,6 @@ public class DataNodeConfigurationDataSet implements DataSet {
     this.status = status;
   }
 
-  public Integer getDataNodeId() {
-    return dataNodeId;
-  }
-
   public void setDataNodeId(int dataNodeId) {
     this.dataNodeId = dataNodeId;
   }
@@ -59,7 +55,7 @@ public class DataNodeConfigurationDataSet implements DataSet {
     resp.setStatus(status);
     if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()
         || status.getCode() == TSStatusCode.DATANODE_ALREADY_REGISTERED.getStatusCode()) {
-      resp.setDataNodeID(dataNodeId);
+      resp.setDataNodeId(dataNodeId);
       resp.setGlobalConfig(globalConfig);
     }
   }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodesInfoDataSet.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeLocationsDataSet.java
similarity index 55%
rename from confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodesInfoDataSet.java
rename to confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeLocationsDataSet.java
index b4807cde92..db23bbc4b9 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodesInfoDataSet.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeLocationsDataSet.java
@@ -18,24 +18,20 @@
  */
 package org.apache.iotdb.confignode.consensus.response;
 
-import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessage;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessageResp;
+import org.apache.iotdb.confignode.rpc.thrift.TDataNodeLocationResp;
 import org.apache.iotdb.consensus.common.DataSet;
 import org.apache.iotdb.rpc.TSStatusCode;
 
-import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
-public class DataNodesInfoDataSet implements DataSet {
+public class DataNodeLocationsDataSet implements DataSet {
 
   private TSStatus status;
-  private List<DataNodeLocation> dataNodeList;
+  private Map<Integer, TDataNodeLocation> dataNodeLocationMap;
 
-  public DataNodesInfoDataSet() {
+  public DataNodeLocationsDataSet() {
     // empty constructor
   }
 
@@ -47,26 +43,14 @@ public class DataNodesInfoDataSet implements DataSet {
     return status;
   }
 
-  public void setDataNodeList(List<DataNodeLocation> dataNodeList) {
-    this.dataNodeList = dataNodeList;
+  public void setDataNodeLocations(Map<Integer, TDataNodeLocation> dataNodeLocationMap) {
+    this.dataNodeLocationMap = dataNodeLocationMap;
   }
 
-  public List<DataNodeLocation> getDataNodeList() {
-    return this.dataNodeList;
-  }
-
-  public void convertToRPCDataNodeMessageResp(TDataNodeMessageResp resp) {
+  public void convertToRpcDataNodeLocationResp(TDataNodeLocationResp resp) {
     resp.setStatus(status);
     if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
-      Map<Integer, TDataNodeMessage> msgMap = new HashMap<>();
-      for (DataNodeLocation info : dataNodeList) {
-        msgMap.put(
-            info.getDataNodeId(),
-            new TDataNodeMessage(
-                info.getDataNodeId(),
-                new TEndPoint(info.getEndPoint().getIp(), info.getEndPoint().getPort())));
-        resp.setDataNodeMessageMap(msgMap);
-      }
+      resp.setDataNodeLocationMap(dataNodeLocationMap);
     }
   }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataPartitionDataSet.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataPartitionDataSet.java
index 324badfe37..15da627333 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataPartitionDataSet.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataPartitionDataSet.java
@@ -19,20 +19,12 @@
 
 package org.apache.iotdb.confignode.consensus.response;
 
-import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
-import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.commons.partition.DataPartition;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionResp;
 import org.apache.iotdb.consensus.common.DataSet;
 import org.apache.iotdb.rpc.TSStatusCode;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 public class DataPartitionDataSet implements DataSet {
 
   private TSStatus status;
@@ -61,53 +53,10 @@ public class DataPartitionDataSet implements DataSet {
    * @param resp TDataPartitionResp
    */
   public void convertToRpcDataPartitionResp(TDataPartitionResp resp) {
-    Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
-        dataPartitionMap = new HashMap<>();
     resp.setStatus(status);
 
     if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
-      dataPartition
-          .getDataPartitionMap()
-          .forEach(
-              ((storageGroup, seriesPartitionSlotTimePartitionSlotRegionReplicaSetListMap) -> {
-                // Extract StorageGroupName
-                dataPartitionMap.putIfAbsent(storageGroup, new HashMap<>());
-
-                seriesPartitionSlotTimePartitionSlotRegionReplicaSetListMap.forEach(
-                    ((seriesPartitionSlot, timePartitionSlotReplicaSetListMap) -> {
-                      // Extract TSeriesPartitionSlot
-                      TSeriesPartitionSlot tSeriesPartitionSlot =
-                          new TSeriesPartitionSlot(seriesPartitionSlot.getSlotId());
-                      dataPartitionMap
-                          .get(storageGroup)
-                          .putIfAbsent(tSeriesPartitionSlot, new HashMap<>());
-
-                      timePartitionSlotReplicaSetListMap.forEach(
-                          ((timePartitionSlot, regionReplicaSets) -> {
-                            // Extract TTimePartitionSlot
-                            TTimePartitionSlot tTimePartitionSlot =
-                                new TTimePartitionSlot(timePartitionSlot.getStartTime());
-                            dataPartitionMap
-                                .get(storageGroup)
-                                .get(tSeriesPartitionSlot)
-                                .putIfAbsent(tTimePartitionSlot, new ArrayList<>());
-
-                            // Extract TRegionReplicaSets
-                            regionReplicaSets.forEach(
-                                regionReplicaSet -> {
-                                  TRegionReplicaSet tRegionReplicaSet =
-                                      regionReplicaSet.convertToRPCTRegionReplicaSet();
-                                  dataPartitionMap
-                                      .get(storageGroup)
-                                      .get(tSeriesPartitionSlot)
-                                      .get(tTimePartitionSlot)
-                                      .add(tRegionReplicaSet);
-                                });
-                          }));
-                    }));
-              }));
-
-      resp.setDataPartitionMap(dataPartitionMap);
+      resp.setDataPartitionMap(dataPartition.getDataPartitionMap());
     }
   }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/SchemaPartitionDataSet.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/SchemaPartitionDataSet.java
index 4955eebbf5..5ed93b7c05 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/SchemaPartitionDataSet.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/SchemaPartitionDataSet.java
@@ -19,17 +19,12 @@
 
 package org.apache.iotdb.confignode.consensus.response;
 
-import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
 import org.apache.iotdb.commons.partition.SchemaPartition;
 import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionResp;
 import org.apache.iotdb.consensus.common.DataSet;
 import org.apache.iotdb.rpc.TSStatusCode;
 
-import java.util.HashMap;
-import java.util.Map;
-
 public class SchemaPartitionDataSet implements DataSet {
 
   private TSStatus status;
@@ -53,34 +48,10 @@ public class SchemaPartitionDataSet implements DataSet {
   }
 
   public void convertToRpcSchemaPartitionResp(TSchemaPartitionResp resp) {
-    Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap = new HashMap<>();
     resp.setStatus(status);
 
     if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
-      schemaPartition
-          .getSchemaPartitionMap()
-          .forEach(
-              (storageGroup, seriesPartitionSlotRegionReplicaSetMap) -> {
-                // Extract StorageGroupName
-                schemaPartitionMap.putIfAbsent(storageGroup, new HashMap<>());
-
-                // Extract Map<SeriesPartitionSlot, RegionReplicaSet>
-                seriesPartitionSlotRegionReplicaSetMap.forEach(
-                    ((seriesPartitionSlot, regionReplicaSet) -> {
-                      // Extract TSeriesPartitionSlot
-                      TSeriesPartitionSlot tSeriesPartitionSlot =
-                          new TSeriesPartitionSlot(seriesPartitionSlot.getSlotId());
-
-                      // Extract TRegionReplicaSet
-                      TRegionReplicaSet tRegionReplicaSet =
-                          regionReplicaSet.convertToRPCTRegionReplicaSet();
-                      schemaPartitionMap
-                          .get(storageGroup)
-                          .put(tSeriesPartitionSlot, tRegionReplicaSet);
-                    }));
-              });
+      resp.setSchemaRegionMap(schemaPartition.getSchemaPartitionMap());
     }
-
-    resp.setSchemaRegionMap(schemaPartitionMap);
   }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/StorageGroupSchemaDataSet.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/StorageGroupSchemaDataSet.java
index 48e0c181ce..0874de13e3 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/StorageGroupSchemaDataSet.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/StorageGroupSchemaDataSet.java
@@ -19,7 +19,6 @@
 package org.apache.iotdb.confignode.consensus.response;
 
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.confignode.partition.StorageGroupSchema;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchemaResp;
 import org.apache.iotdb.consensus.common.DataSet;
@@ -33,7 +32,7 @@ public class StorageGroupSchemaDataSet implements DataSet {
 
   private TSStatus status;
 
-  private List<StorageGroupSchema> schemaList;
+  private List<TStorageGroupSchema> schemaList;
 
   public StorageGroupSchemaDataSet() {}
 
@@ -45,11 +44,11 @@ public class StorageGroupSchemaDataSet implements DataSet {
     this.status = status;
   }
 
-  public List<StorageGroupSchema> getSchemaList() {
+  public List<TStorageGroupSchema> getSchemaList() {
     return schemaList;
   }
 
-  public void setSchemaList(List<StorageGroupSchema> schemaList) {
+  public void setSchemaList(List<TStorageGroupSchema> schemaList) {
     this.schemaList = schemaList;
   }
 
@@ -57,8 +56,8 @@ public class StorageGroupSchemaDataSet implements DataSet {
     resp.setStatus(status);
     if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
       Map<String, TStorageGroupSchema> storageGroupMessageMap = new HashMap<>();
-      for (StorageGroupSchema schema : schemaList) {
-        storageGroupMessageMap.put(schema.getName(), new TStorageGroupSchema(schema.getName()));
+      for (TStorageGroupSchema schema : schemaList) {
+        storageGroupMessageMap.put(schema.getName(), schema);
       }
       resp.setStorageGroupSchemaMap(storageGroupMessageMap);
     }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
index 43db8e07eb..5ac486eee4 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
@@ -20,9 +20,9 @@
 package org.apache.iotdb.confignode.manager;
 
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
 import org.apache.iotdb.confignode.consensus.response.DataNodeConfigurationDataSet;
-import org.apache.iotdb.confignode.consensus.response.DataNodesInfoDataSet;
+import org.apache.iotdb.confignode.consensus.response.DataNodeLocationsDataSet;
 import org.apache.iotdb.confignode.consensus.response.DataPartitionDataSet;
 import org.apache.iotdb.confignode.consensus.response.PermissionInfoDataSet;
 import org.apache.iotdb.confignode.consensus.response.SchemaPartitionDataSet;
@@ -102,7 +102,7 @@ public class ConfigManager implements Manager {
     if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
       return dataNodeManager.getDataNodeInfo((QueryDataNodeInfoPlan) physicalPlan);
     } else {
-      DataNodesInfoDataSet dataSet = new DataNodesInfoDataSet();
+      DataNodeLocationsDataSet dataSet = new DataNodeLocationsDataSet();
       dataSet.setStatus(status);
       return dataSet;
     }
@@ -139,7 +139,7 @@ public class ConfigManager implements Manager {
 
       GetOrCreateSchemaPartitionPlan getSchemaPartitionPlan =
           new GetOrCreateSchemaPartitionPlan(PhysicalPlanType.GetSchemaPartition);
-      Map<String, List<SeriesPartitionSlot>> partitionSlotsMap = new HashMap<>();
+      Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap = new HashMap<>();
 
       boolean getAll = false;
       Set<String> getAllSet = new HashSet<>();
@@ -196,7 +196,7 @@ public class ConfigManager implements Manager {
 
       GetOrCreateSchemaPartitionPlan getOrCreateSchemaPartitionPlan =
           new GetOrCreateSchemaPartitionPlan(PhysicalPlanType.GetOrCreateSchemaPartition);
-      Map<String, List<SeriesPartitionSlot>> partitionSlotsMap = new HashMap<>();
+      Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap = new HashMap<>();
 
       for (String devicePath : devicePaths) {
         if (!devicePath.contains("*")) {
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java
index 09d1223df7..0058416429 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.confignode.manager;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.PartitionRegionId;
 import org.apache.iotdb.confignode.conf.ConfigNodeConf;
@@ -65,7 +65,7 @@ public class ConsensusManager {
     consensusImpl =
         ConsensusFactory.getConsensusImpl(
                 conf.getConfigNodeConsensusProtocolClass(),
-                new Endpoint(conf.getRpcAddress(), conf.getInternalPort()),
+                new TEndPoint(conf.getRpcAddress(), conf.getInternalPort()),
                 new File(conf.getConsensusDir()),
                 gid -> new PartitionRegionStateMachine())
             .orElseThrow(
@@ -81,7 +81,7 @@ public class ConsensusManager {
         "Set ConfigNode consensus group {}...",
         Arrays.toString(conf.getConfigNodeGroupAddressList()));
     List<Peer> peerList = new ArrayList<>();
-    for (Endpoint endpoint : conf.getConfigNodeGroupAddressList()) {
+    for (TEndPoint endpoint : conf.getConfigNodeGroupAddressList()) {
       peerList.add(new Peer(consensusGroupId, endpoint));
     }
     consensusImpl.addConsensusGroup(consensusGroupId, peerList);
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/DataNodeManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/DataNodeManager.java
index 21c2734ee7..6f28782cd1 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/DataNodeManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/DataNodeManager.java
@@ -18,11 +18,11 @@
  */
 package org.apache.iotdb.confignode.manager;
 
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
 import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
 import org.apache.iotdb.confignode.consensus.response.DataNodeConfigurationDataSet;
-import org.apache.iotdb.confignode.consensus.response.DataNodesInfoDataSet;
+import org.apache.iotdb.confignode.consensus.response.DataNodeLocationsDataSet;
 import org.apache.iotdb.confignode.persistence.DataNodeInfoPersistence;
 import org.apache.iotdb.confignode.physical.sys.QueryDataNodeInfoPlan;
 import org.apache.iotdb.confignode.physical.sys.RegisterDataNodePlan;
@@ -78,18 +78,19 @@ public class DataNodeManager {
   public DataSet registerDataNode(RegisterDataNodePlan plan) {
     DataNodeConfigurationDataSet dataSet = new DataNodeConfigurationDataSet();
 
-    if (DataNodeInfoPersistence.getInstance().containsValue(plan.getInfo())) {
+    if (DataNodeInfoPersistence.getInstance().containsValue(plan.getLocation())) {
       TSStatus status = new TSStatus(TSStatusCode.DATANODE_ALREADY_REGISTERED.getStatusCode());
       status.setMessage("DataNode already registered.");
       dataSet.setStatus(status);
     } else {
       // Persist DataNodeInfo
-      plan.getInfo().setDataNodeId(DataNodeInfoPersistence.getInstance().generateNextDataNodeId());
+      plan.getLocation()
+          .setDataNodeId(DataNodeInfoPersistence.getInstance().generateNextDataNodeId());
       ConsensusWriteResponse resp = getConsensusManager().write(plan);
       dataSet.setStatus(resp.getStatus());
     }
 
-    dataSet.setDataNodeId(plan.getInfo().getDataNodeId());
+    dataSet.setDataNodeId(plan.getLocation().getDataNodeId());
     setGlobalConfig(dataSet);
     return dataSet;
   }
@@ -101,15 +102,15 @@ public class DataNodeManager {
    * @return The specific DataNode's info or all DataNode info if dataNodeId in
    *     QueryDataNodeInfoPlan is -1
    */
-  public DataNodesInfoDataSet getDataNodeInfo(QueryDataNodeInfoPlan plan) {
-    return (DataNodesInfoDataSet) getConsensusManager().read(plan).getDataset();
+  public DataNodeLocationsDataSet getDataNodeInfo(QueryDataNodeInfoPlan plan) {
+    return (DataNodeLocationsDataSet) getConsensusManager().read(plan).getDataset();
   }
 
   public int getOnlineDataNodeCount() {
     return dataNodeInfoPersistence.getOnlineDataNodeCount();
   }
 
-  public List<DataNodeLocation> getOnlineDataNodes() {
+  public List<TDataNodeLocation> getOnlineDataNodes() {
     return dataNodeInfoPersistence.getOnlineDataNodes();
   }
 
@@ -138,12 +139,12 @@ public class DataNodeManager {
     }
 
     @Override
-    public void addDataNode(DataNodeLocation DataNodeInfo) {
+    public void addDataNode(TDataNodeLocation DataNodeInfo) {
       serverChanged();
     }
 
     @Override
-    public void removeDataNode(DataNodeLocation dataNodeInfo) {
+    public void removeDataNode(TDataNodeLocation dataNodeInfo) {
       serverChanged();
     }
 
@@ -169,13 +170,13 @@ public class DataNodeManager {
      *
      * @param dataNodeInfo datanode info
      */
-    void addDataNode(final DataNodeLocation dataNodeInfo);
+    void addDataNode(final TDataNodeLocation dataNodeInfo);
 
     /**
      * remove data node
      *
      * @param dataNodeInfo data node info
      */
-    void removeDataNode(final DataNodeLocation dataNodeInfo);
+    void removeDataNode(final TDataNodeLocation dataNodeInfo);
   }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/PartitionManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/PartitionManager.java
index 99f07bc31b..607e4bba0e 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/PartitionManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/PartitionManager.java
@@ -18,9 +18,9 @@
  */
 package org.apache.iotdb.confignode.manager;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
 import org.apache.iotdb.confignode.conf.ConfigNodeConf;
 import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
@@ -85,13 +85,13 @@ public class PartitionManager {
    * @return SchemaPartitionDataSet
    */
   public DataSet getOrCreateSchemaPartition(GetOrCreateSchemaPartitionPlan physicalPlan) {
-    Map<String, List<SeriesPartitionSlot>> noAssignedSchemaPartitionSlots =
+    Map<String, List<TSeriesPartitionSlot>> noAssignedSchemaPartitionSlots =
         partitionInfoPersistence.filterNoAssignedSchemaPartitionSlots(
             physicalPlan.getPartitionSlotsMap());
 
     if (noAssignedSchemaPartitionSlots.size() > 0) {
       // Allocate SchemaPartition
-      Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> assignedSchemaPartition =
+      Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> assignedSchemaPartition =
           allocateSchemaPartition(noAssignedSchemaPartitionSlots);
 
       // Persist SchemaPartition
@@ -109,18 +109,18 @@ public class PartitionManager {
    * @param noAssignedSchemaPartitionSlotsMap Map<StorageGroupName, List<SeriesPartitionSlot>>
    * @return assign result, Map<StorageGroupName, Map<SeriesPartitionSlot, RegionReplicaSet>>
    */
-  private Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> allocateSchemaPartition(
-      Map<String, List<SeriesPartitionSlot>> noAssignedSchemaPartitionSlotsMap) {
-    Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> result = new HashMap<>();
+  private Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> allocateSchemaPartition(
+      Map<String, List<TSeriesPartitionSlot>> noAssignedSchemaPartitionSlotsMap) {
+    Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> result = new HashMap<>();
 
     for (String storageGroup : noAssignedSchemaPartitionSlotsMap.keySet()) {
-      List<SeriesPartitionSlot> noAssignedPartitionSlots =
+      List<TSeriesPartitionSlot> noAssignedPartitionSlots =
           noAssignedSchemaPartitionSlotsMap.get(storageGroup);
-      List<RegionReplicaSet> schemaRegionEndPoints =
+      List<TRegionReplicaSet> schemaRegionEndPoints =
           RegionInfoPersistence.getInstance().getSchemaRegionEndPoint(storageGroup);
       Random random = new Random();
 
-      Map<SeriesPartitionSlot, RegionReplicaSet> allocateResult = new HashMap<>();
+      Map<TSeriesPartitionSlot, TRegionReplicaSet> allocateResult = new HashMap<>();
       noAssignedPartitionSlots.forEach(
           seriesPartitionSlot ->
               allocateResult.put(
@@ -155,13 +155,13 @@ public class PartitionManager {
    * @return DataPartitionDataSet
    */
   public DataSet getOrCreateDataPartition(GetOrCreateDataPartitionPlan physicalPlan) {
-    Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> noAssignedDataPartitionSlots =
+    Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> noAssignedDataPartitionSlots =
         partitionInfoPersistence.filterNoAssignedDataPartitionSlots(
             physicalPlan.getPartitionSlotsMap());
 
     if (noAssignedDataPartitionSlots.size() > 0) {
       // Allocate DataPartition
-      Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+      Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
           assignedDataPartition = allocateDataPartition(noAssignedDataPartitionSlots);
 
       // Persist DataPartition
@@ -181,27 +181,27 @@ public class PartitionManager {
    * @return assign result, Map<StorageGroupName, Map<SeriesPartitionSlot, Map<TimePartitionSlot,
    *     List<RegionReplicaSet>>>>
    */
-  private Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+  private Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
       allocateDataPartition(
-          Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>>
+          Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>>
               noAssignedDataPartitionSlotsMap) {
 
-    Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>> result =
-        new HashMap<>();
+    Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
+        result = new HashMap<>();
 
     for (String storageGroup : noAssignedDataPartitionSlotsMap.keySet()) {
-      Map<SeriesPartitionSlot, List<TimePartitionSlot>> noAssignedPartitionSlotsMap =
+      Map<TSeriesPartitionSlot, List<TTimePartitionSlot>> noAssignedPartitionSlotsMap =
           noAssignedDataPartitionSlotsMap.get(storageGroup);
-      List<RegionReplicaSet> dataRegionEndPoints =
+      List<TRegionReplicaSet> dataRegionEndPoints =
           RegionInfoPersistence.getInstance().getDataRegionEndPoint(storageGroup);
       Random random = new Random();
 
-      Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>> allocateResult =
+      Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>> allocateResult =
           new HashMap<>();
-      for (Map.Entry<SeriesPartitionSlot, List<TimePartitionSlot>> seriesPartitionEntry :
+      for (Map.Entry<TSeriesPartitionSlot, List<TTimePartitionSlot>> seriesPartitionEntry :
           noAssignedPartitionSlotsMap.entrySet()) {
         allocateResult.put(seriesPartitionEntry.getKey(), new HashMap<>());
-        for (TimePartitionSlot timePartitionSlot : seriesPartitionEntry.getValue()) {
+        for (TTimePartitionSlot timePartitionSlot : seriesPartitionEntry.getValue()) {
           allocateResult
               .get(seriesPartitionEntry.getKey())
               .computeIfAbsent(timePartitionSlot, key -> new ArrayList<>())
@@ -223,12 +223,12 @@ public class PartitionManager {
   }
 
   /**
-   * Get SeriesPartitionSlot
+   * Get TSeriesPartitionSlot
    *
    * @param devicePath Full path ending with device name
    * @return SeriesPartitionSlot
    */
-  public SeriesPartitionSlot getSeriesPartitionSlot(String devicePath) {
+  public TSeriesPartitionSlot getSeriesPartitionSlot(String devicePath) {
     return executor.getSeriesPartitionSlot(devicePath);
   }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/RegionManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/RegionManager.java
index fa98266fc0..dfbacf4b2e 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/RegionManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/RegionManager.java
@@ -19,13 +19,11 @@
 
 package org.apache.iotdb.confignode.manager;
 
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.consensus.ConsensusGroupId;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.GroupType;
-import org.apache.iotdb.commons.consensus.SchemaRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
 import org.apache.iotdb.confignode.cli.TemporaryClient;
 import org.apache.iotdb.confignode.conf.ConfigNodeConf;
 import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
@@ -44,9 +42,10 @@ import java.util.List;
 public class RegionManager {
 
   private static final ConfigNodeConf conf = ConfigNodeDescriptor.getInstance().getConf();
-  private static final int regionReplicaCount = conf.getRegionReplicaCount();
-  private static final int schemaRegionCount = conf.getSchemaRegionCount();
-  private static final int dataRegionCount = conf.getDataRegionCount();
+  private static final int schemaReplicationFactor = conf.getSchemaReplicationFactor();
+  private static final int dataReplicationFactor = conf.getDataReplicationFactor();
+  private static final int initialSchemaRegionCount = conf.getInitialSchemaRegionCount();
+  private static final int initialDataRegionCount = conf.getInitialDataRegionCount();
 
   private static final RegionInfoPersistence regionInfoPersistence =
       RegionInfoPersistence.getInstance();
@@ -71,7 +70,8 @@ public class RegionManager {
    */
   public TSStatus setStorageGroup(SetStorageGroupPlan plan) {
     TSStatus result;
-    if (configNodeManager.getDataNodeManager().getOnlineDataNodeCount() < regionReplicaCount) {
+    if (configNodeManager.getDataNodeManager().getOnlineDataNodeCount()
+        < Math.max(initialSchemaRegionCount, initialDataRegionCount)) {
       result = new TSStatus(TSStatusCode.NOT_ENOUGH_DATA_NODE.getStatusCode());
       result.setMessage("DataNode is not enough, please register more.");
     } else {
@@ -84,29 +84,31 @@ public class RegionManager {
         createPlan.setStorageGroup(plan.getSchema().getName());
 
         // Allocate default Regions
-        allocateRegions(GroupType.SchemaRegion, createPlan);
-        allocateRegions(GroupType.DataRegion, createPlan);
+        allocateRegions(TConsensusGroupType.SchemaRegion, createPlan);
+        allocateRegions(TConsensusGroupType.DataRegion, createPlan);
 
         // Persist StorageGroup and Regions
         getConsensusManager().write(plan);
         result = getConsensusManager().write(createPlan).getStatus();
 
         // Create Regions in DataNode
-        for (RegionReplicaSet regionReplicaSet : createPlan.getRegionReplicaSets()) {
-          for (DataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeList()) {
-            if (regionReplicaSet.getConsensusGroupId() instanceof SchemaRegionId) {
-              TemporaryClient.getInstance()
-                  .createSchemaRegion(
-                      dataNodeLocation.getDataNodeId(),
-                      createPlan.getStorageGroup(),
-                      regionReplicaSet);
-            } else if (regionReplicaSet.getConsensusGroupId() instanceof DataRegionId) {
-              TemporaryClient.getInstance()
-                  .createDataRegion(
-                      dataNodeLocation.getDataNodeId(),
-                      createPlan.getStorageGroup(),
-                      regionReplicaSet,
-                      plan.getSchema().getTTL());
+        for (TRegionReplicaSet regionReplicaSet : createPlan.getRegionReplicaSets()) {
+          for (TDataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeLocations()) {
+            switch (regionReplicaSet.getRegionId().getType()) {
+              case SchemaRegion:
+                TemporaryClient.getInstance()
+                    .createSchemaRegion(
+                        dataNodeLocation.getDataNodeId(),
+                        createPlan.getStorageGroup(),
+                        regionReplicaSet);
+                break;
+              case DataRegion:
+                TemporaryClient.getInstance()
+                    .createDataRegion(
+                        dataNodeLocation.getDataNodeId(),
+                        createPlan.getStorageGroup(),
+                        regionReplicaSet,
+                        plan.getSchema().getTTL());
             }
           }
         }
@@ -119,26 +121,27 @@ public class RegionManager {
     return configNodeManager.getDataNodeManager();
   }
 
-  private void allocateRegions(GroupType type, CreateRegionsPlan plan) {
+  private void allocateRegions(TConsensusGroupType type, CreateRegionsPlan plan) {
 
     // TODO: Use CopySet algorithm to optimize region allocation policy
 
-    int regionCount = type.equals(GroupType.SchemaRegion) ? schemaRegionCount : dataRegionCount;
-    List<DataNodeLocation> onlineDataNodes = getDataNodeInfoManager().getOnlineDataNodes();
+    int replicaCount =
+        type.equals(TConsensusGroupType.SchemaRegion)
+            ? schemaReplicationFactor
+            : dataReplicationFactor;
+    int regionCount =
+        type.equals(TConsensusGroupType.SchemaRegion)
+            ? initialSchemaRegionCount
+            : initialDataRegionCount;
+    List<TDataNodeLocation> onlineDataNodes = getDataNodeInfoManager().getOnlineDataNodes();
     for (int i = 0; i < regionCount; i++) {
       Collections.shuffle(onlineDataNodes);
 
-      RegionReplicaSet regionReplicaSet = new RegionReplicaSet();
-      ConsensusGroupId consensusGroupId = null;
-      switch (type) {
-        case SchemaRegion:
-          consensusGroupId = new SchemaRegionId(regionInfoPersistence.generateNextRegionGroupId());
-          break;
-        case DataRegion:
-          consensusGroupId = new DataRegionId(regionInfoPersistence.generateNextRegionGroupId());
-      }
-      regionReplicaSet.setConsensusGroupId(consensusGroupId);
-      regionReplicaSet.setDataNodeList(onlineDataNodes.subList(0, regionReplicaCount));
+      TRegionReplicaSet regionReplicaSet = new TRegionReplicaSet();
+      TConsensusGroupId consensusGroupId =
+          new TConsensusGroupId(type, regionInfoPersistence.generateNextRegionGroupId());
+      regionReplicaSet.setRegionId(consensusGroupId);
+      regionReplicaSet.setDataNodeLocations(onlineDataNodes.subList(0, replicaCount));
       plan.addRegion(regionReplicaSet);
     }
   }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/partition/StorageGroupSchema.java b/confignode/src/main/java/org/apache/iotdb/confignode/partition/StorageGroupSchema.java
deleted file mode 100644
index e128a2092c..0000000000
--- a/confignode/src/main/java/org/apache/iotdb/confignode/partition/StorageGroupSchema.java
+++ /dev/null
@@ -1,121 +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.confignode.partition;
-
-import org.apache.iotdb.commons.consensus.ConsensusGroupId;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-
-public class StorageGroupSchema {
-
-  private String name;
-  private long TTL;
-
-  private final List<ConsensusGroupId> schemaRegionGroupIds;
-  private final List<ConsensusGroupId> dataRegionGroupIds;
-
-  public StorageGroupSchema() {
-    schemaRegionGroupIds = new ArrayList<>();
-    dataRegionGroupIds = new ArrayList<>();
-  }
-
-  public StorageGroupSchema(String name) {
-    this();
-    this.name = name;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public long getTTL() {
-    return TTL;
-  }
-
-  public void setTTL(long TTL) {
-    this.TTL = TTL;
-  }
-
-  public List<ConsensusGroupId> getSchemaRegionGroupIds() {
-    return schemaRegionGroupIds;
-  }
-
-  public void addSchemaRegionGroup(ConsensusGroupId id) {
-    schemaRegionGroupIds.add(id);
-  }
-
-  public List<ConsensusGroupId> getDataRegionGroupIds() {
-    return dataRegionGroupIds;
-  }
-
-  public void addDataRegionGroup(ConsensusGroupId id) {
-    dataRegionGroupIds.add(id);
-  }
-
-  public void serialize(ByteBuffer buffer) {
-    buffer.putInt(name.length());
-    buffer.put(name.getBytes());
-
-    buffer.putInt(schemaRegionGroupIds.size());
-    for (ConsensusGroupId schemaRegionGroupId : schemaRegionGroupIds) {
-      schemaRegionGroupId.serializeImpl(buffer);
-    }
-
-    buffer.putInt(dataRegionGroupIds.size());
-    for (ConsensusGroupId dataRegionGroupId : dataRegionGroupIds) {
-      dataRegionGroupId.serializeImpl(buffer);
-    }
-  }
-
-  public void deserialize(ByteBuffer buffer) throws IOException {
-    name = SerializeDeserializeUtil.readString(buffer);
-
-    int length = buffer.getInt();
-    for (int i = 0; i < length; i++) {
-      ConsensusGroupId schemaRegionId = ConsensusGroupId.Factory.create(buffer);
-      schemaRegionGroupIds.add(schemaRegionId);
-    }
-
-    length = buffer.getInt();
-    for (int i = 0; i < length; i++) {
-      ConsensusGroupId dataRegionId = ConsensusGroupId.Factory.create(buffer);
-      dataRegionGroupIds.add(dataRegionId);
-    }
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-    StorageGroupSchema that = (StorageGroupSchema) o;
-    return name.equals(that.name)
-        && schemaRegionGroupIds.equals(that.schemaRegionGroupIds)
-        && dataRegionGroupIds.equals(that.dataRegionGroupIds);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(name, schemaRegionGroupIds, dataRegionGroupIds);
-  }
-}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/DataNodeInfoPersistence.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/DataNodeInfoPersistence.java
index c1a64bb2f8..d09b3dc740 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/DataNodeInfoPersistence.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/DataNodeInfoPersistence.java
@@ -18,16 +18,17 @@
  */
 package org.apache.iotdb.confignode.persistence;
 
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
 import org.apache.iotdb.commons.utils.TestOnly;
-import org.apache.iotdb.confignode.consensus.response.DataNodesInfoDataSet;
+import org.apache.iotdb.confignode.consensus.response.DataNodeLocationsDataSet;
 import org.apache.iotdb.confignode.physical.sys.QueryDataNodeInfoPlan;
 import org.apache.iotdb.confignode.physical.sys.RegisterDataNodePlan;
 import org.apache.iotdb.rpc.TSStatusCode;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -45,25 +46,25 @@ public class DataNodeInfoPersistence {
 
   /** online data nodes */
   // TODO: serialize and deserialize
-  private final ConcurrentNavigableMap<Integer, DataNodeLocation> onlineDataNodes =
+  private final ConcurrentNavigableMap<Integer, TDataNodeLocation> onlineDataNodes =
       new ConcurrentSkipListMap();
 
   /** For remove node or draining node */
-  private final Set<DataNodeLocation> drainingDataNodes = new HashSet<>();
+  private final Set<TDataNodeLocation> drainingDataNodes = new HashSet<>();
 
   private DataNodeInfoPersistence() {
     this.dataNodeInfoReadWriteLock = new ReentrantReadWriteLock();
   }
 
-  public boolean containsValue(DataNodeLocation info) {
+  public boolean containsValue(TDataNodeLocation info) {
     boolean result = false;
     dataNodeInfoReadWriteLock.readLock().lock();
 
     try {
-      for (Map.Entry<Integer, DataNodeLocation> entry : onlineDataNodes.entrySet()) {
-        if (entry.getValue().getEndPoint().equals(info.getEndPoint())) {
+      for (Map.Entry<Integer, TDataNodeLocation> entry : onlineDataNodes.entrySet()) {
+        info.setDataNodeId(entry.getKey());
+        if (entry.getValue().equals(info)) {
           result = true;
-          info.setDataNodeId(entry.getKey());
           break;
         }
       }
@@ -74,7 +75,7 @@ public class DataNodeInfoPersistence {
     return result;
   }
 
-  public void put(int dataNodeID, DataNodeLocation info) {
+  public void put(int dataNodeID, TDataNodeLocation info) {
     onlineDataNodes.put(dataNodeID, info);
   }
 
@@ -86,7 +87,7 @@ public class DataNodeInfoPersistence {
    */
   public TSStatus registerDataNode(RegisterDataNodePlan plan) {
     TSStatus result;
-    DataNodeLocation info = plan.getInfo();
+    TDataNodeLocation info = plan.getLocation();
     dataNodeInfoReadWriteLock.writeLock().lock();
     try {
       nextDataNodeId = Math.max(nextDataNodeId, info.getDataNodeId());
@@ -105,17 +106,19 @@ public class DataNodeInfoPersistence {
    * @return The specific DataNode's info or all DataNode info if dataNodeId in
    *     QueryDataNodeInfoPlan is -1
    */
-  public DataNodesInfoDataSet getDataNodeInfo(QueryDataNodeInfoPlan plan) {
-    DataNodesInfoDataSet result = new DataNodesInfoDataSet();
+  public DataNodeLocationsDataSet getDataNodeInfo(QueryDataNodeInfoPlan plan) {
+    DataNodeLocationsDataSet result = new DataNodeLocationsDataSet();
     result.setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()));
 
     int dataNodeId = plan.getDataNodeID();
     dataNodeInfoReadWriteLock.readLock().lock();
     try {
       if (dataNodeId == -1) {
-        result.setDataNodeList(new ArrayList<>(onlineDataNodes.values()));
+        result.setDataNodeLocations(new HashMap<>(onlineDataNodes));
       } else {
-        result.setDataNodeList(Collections.singletonList(onlineDataNodes.get(dataNodeId)));
+
+        result.setDataNodeLocations(
+            Collections.singletonMap(dataNodeId, onlineDataNodes.get(dataNodeId)));
       }
     } finally {
       dataNodeInfoReadWriteLock.readLock().unlock();
@@ -135,8 +138,8 @@ public class DataNodeInfoPersistence {
     return result;
   }
 
-  public List<DataNodeLocation> getOnlineDataNodes() {
-    List<DataNodeLocation> result;
+  public List<TDataNodeLocation> getOnlineDataNodes() {
+    List<TDataNodeLocation> result;
     dataNodeInfoReadWriteLock.readLock().lock();
     try {
       result = new ArrayList<>(onlineDataNodes.values());
@@ -146,8 +149,8 @@ public class DataNodeInfoPersistence {
     return result;
   }
 
-  public DataNodeLocation getOnlineDataNode(int dataNodeId) {
-    DataNodeLocation result;
+  public TDataNodeLocation getOnlineDataNode(int dataNodeId) {
+    TDataNodeLocation result;
     dataNodeInfoReadWriteLock.readLock().lock();
     try {
       result = onlineDataNodes.get(dataNodeId);
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/PartitionInfoPersistence.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/PartitionInfoPersistence.java
index 762cad1047..348c21efa8 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/PartitionInfoPersistence.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/PartitionInfoPersistence.java
@@ -19,12 +19,12 @@
 
 package org.apache.iotdb.confignode.persistence;
 
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.commons.partition.DataPartition;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
 import org.apache.iotdb.commons.partition.SchemaPartition;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
 import org.apache.iotdb.confignode.consensus.response.DataPartitionDataSet;
@@ -103,7 +103,7 @@ public class PartitionInfoPersistence {
 
     try {
       // Allocate SchemaPartition by CreateSchemaPartitionPlan
-      Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> assignedResult =
+      Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> assignedResult =
           physicalPlan.getAssignedSchemaPartition();
       assignedResult.forEach(
           (storageGroup, partitionSlots) ->
@@ -118,9 +118,9 @@ public class PartitionInfoPersistence {
     return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
   }
 
-  public Map<String, List<SeriesPartitionSlot>> filterNoAssignedSchemaPartitionSlots(
-      Map<String, List<SeriesPartitionSlot>> partitionSlotsMap) {
-    Map<String, List<SeriesPartitionSlot>> result;
+  public Map<String, List<TSeriesPartitionSlot>> filterNoAssignedSchemaPartitionSlots(
+      Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap) {
+    Map<String, List<TSeriesPartitionSlot>> result;
     schemaPartitionReadWriteLock.readLock().lock();
     try {
       result = schemaPartition.filterNoAssignedSchemaPartitionSlot(partitionSlotsMap);
@@ -165,7 +165,7 @@ public class PartitionInfoPersistence {
 
     try {
       // Allocate DataPartition by CreateDataPartitionPlan
-      Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+      Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
           assignedResult = physicalPlan.getAssignedDataPartition();
       assignedResult.forEach(
           (storageGroup, seriesPartitionTimePartitionSlots) ->
@@ -187,10 +187,10 @@ public class PartitionInfoPersistence {
     return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
   }
 
-  public Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>>
+  public Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>>
       filterNoAssignedDataPartitionSlots(
-          Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> partitionSlotsMap) {
-    Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> result;
+          Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap) {
+    Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> result;
     dataPartitionReadWriteLock.readLock().lock();
     try {
       result = dataPartition.filterNoAssignedDataPartitionSlots(partitionSlotsMap);
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/RegionInfoPersistence.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/RegionInfoPersistence.java
index 97249e85d9..0e426d44a8 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/RegionInfoPersistence.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/RegionInfoPersistence.java
@@ -19,16 +19,14 @@
 
 package org.apache.iotdb.confignode.persistence;
 
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.consensus.ConsensusGroupId;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.SchemaRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.confignode.consensus.response.StorageGroupSchemaDataSet;
-import org.apache.iotdb.confignode.partition.StorageGroupSchema;
 import org.apache.iotdb.confignode.physical.crud.CreateRegionsPlan;
 import org.apache.iotdb.confignode.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.rpc.TSStatusCode;
 
 import java.util.ArrayList;
@@ -42,7 +40,7 @@ public class RegionInfoPersistence {
 
   // TODO: Serialize and Deserialize
   // Map<StorageGroupName, StorageGroupSchema>
-  private final Map<String, StorageGroupSchema> storageGroupsMap;
+  private final Map<String, TStorageGroupSchema> storageGroupsMap;
 
   // Region allocate lock
   private final ReentrantReadWriteLock regionAllocateLock;
@@ -51,8 +49,7 @@ public class RegionInfoPersistence {
 
   // Region read write lock
   private final ReentrantReadWriteLock regionReadWriteLock;
-  // Map<ConsensusGroupId, RegionReplicaSet>
-  private final Map<ConsensusGroupId, RegionReplicaSet> regionMap;
+  private final Map<TConsensusGroupId, TRegionReplicaSet> regionMap;
 
   public RegionInfoPersistence() {
     this.regionAllocateLock = new ReentrantReadWriteLock();
@@ -71,8 +68,8 @@ public class RegionInfoPersistence {
     TSStatus result;
     regionReadWriteLock.writeLock().lock();
     try {
-      StorageGroupSchema schema = plan.getSchema();
-      storageGroupsMap.put(schema.getName(), schema);
+      TStorageGroupSchema storageGroupSchema = plan.getSchema();
+      storageGroupsMap.put(storageGroupSchema.getName(), storageGroupSchema);
       result = new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
     } finally {
       regionReadWriteLock.writeLock().unlock();
@@ -103,16 +100,17 @@ public class RegionInfoPersistence {
     regionReadWriteLock.writeLock().lock();
     regionAllocateLock.writeLock().lock();
     try {
-      StorageGroupSchema schema = storageGroupsMap.get(plan.getStorageGroup());
-
-      for (RegionReplicaSet regionReplicaSet : plan.getRegionReplicaSets()) {
-        nextRegionGroupId =
-            Math.max(nextRegionGroupId, regionReplicaSet.getConsensusGroupId().getId());
-        regionMap.put(regionReplicaSet.getConsensusGroupId(), regionReplicaSet);
-        if (regionReplicaSet.getConsensusGroupId() instanceof DataRegionId) {
-          schema.addDataRegionGroup(regionReplicaSet.getConsensusGroupId());
-        } else if (regionReplicaSet.getConsensusGroupId() instanceof SchemaRegionId) {
-          schema.addSchemaRegionGroup(regionReplicaSet.getConsensusGroupId());
+      TStorageGroupSchema storageGroupSchema = storageGroupsMap.get(plan.getStorageGroup());
+
+      for (TRegionReplicaSet regionReplicaSet : plan.getRegionReplicaSets()) {
+        nextRegionGroupId = Math.max(nextRegionGroupId, regionReplicaSet.getRegionId().getId());
+        regionMap.put(regionReplicaSet.getRegionId(), regionReplicaSet);
+        switch (regionReplicaSet.getRegionId().getType()) {
+          case SchemaRegion:
+            storageGroupSchema.getSchemaRegionGroupIds().add(regionReplicaSet.getRegionId());
+            break;
+          case DataRegion:
+            storageGroupSchema.getDataRegionGroupIds().add(regionReplicaSet.getRegionId());
         }
       }
 
@@ -125,14 +123,14 @@ public class RegionInfoPersistence {
   }
 
   /** @return The SchemaRegion ReplicaSets in the specific StorageGroup */
-  public List<RegionReplicaSet> getSchemaRegionEndPoint(String storageGroup) {
-    List<RegionReplicaSet> schemaRegionEndPoints = new ArrayList<>();
+  public List<TRegionReplicaSet> getSchemaRegionEndPoint(String storageGroup) {
+    List<TRegionReplicaSet> schemaRegionEndPoints = new ArrayList<>();
     regionReadWriteLock.readLock().lock();
     try {
       if (storageGroupsMap.containsKey(storageGroup)) {
-        List<ConsensusGroupId> schemaRegionIds =
+        List<TConsensusGroupId> schemaRegionIds =
             storageGroupsMap.get(storageGroup).getSchemaRegionGroupIds();
-        for (ConsensusGroupId consensusGroupId : schemaRegionIds) {
+        for (TConsensusGroupId consensusGroupId : schemaRegionIds) {
           schemaRegionEndPoints.add(regionMap.get(consensusGroupId));
         }
       }
@@ -144,14 +142,14 @@ public class RegionInfoPersistence {
   }
 
   /** @return The DataRegion ReplicaSets in the specific StorageGroup */
-  public List<RegionReplicaSet> getDataRegionEndPoint(String storageGroup) {
-    List<RegionReplicaSet> dataRegionEndPoints = new ArrayList<>();
+  public List<TRegionReplicaSet> getDataRegionEndPoint(String storageGroup) {
+    List<TRegionReplicaSet> dataRegionEndPoints = new ArrayList<>();
     regionReadWriteLock.readLock().lock();
     try {
       if (storageGroupsMap.containsKey(storageGroup)) {
-        List<ConsensusGroupId> dataRegionIds =
+        List<TConsensusGroupId> dataRegionIds =
             storageGroupsMap.get(storageGroup).getDataRegionGroupIds();
-        for (ConsensusGroupId consensusGroupId : dataRegionIds) {
+        for (TConsensusGroupId consensusGroupId : dataRegionIds) {
           dataRegionEndPoints.add(regionMap.get(consensusGroupId));
         }
       }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateDataPartitionPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateDataPartitionPlan.java
index 9daf5ef921..0b1bc95bba 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateDataPartitionPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateDataPartitionPlan.java
@@ -18,12 +18,13 @@
  */
 package org.apache.iotdb.confignode.physical.crud;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
 import org.apache.iotdb.confignode.physical.PhysicalPlan;
 import org.apache.iotdb.confignode.physical.PhysicalPlanType;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -36,20 +37,20 @@ import java.util.Objects;
 /** Create DataPartition by assignedDataPartition */
 public class CreateDataPartitionPlan extends PhysicalPlan {
 
-  private Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+  private Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
       assignedDataPartition;
 
   public CreateDataPartitionPlan() {
     super(PhysicalPlanType.CreateDataPartition);
   }
 
-  public Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+  public Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
       getAssignedDataPartition() {
     return assignedDataPartition;
   }
 
   public void setAssignedDataPartition(
-      Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+      Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
           assignedDataPartition) {
     this.assignedDataPartition = assignedDataPartition;
   }
@@ -59,20 +60,21 @@ public class CreateDataPartitionPlan extends PhysicalPlan {
     buffer.putInt(PhysicalPlanType.CreateDataPartition.ordinal());
 
     buffer.putInt(assignedDataPartition.size());
-    for (Map.Entry<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+    for (Map.Entry<
+            String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
         seriesPartitionTimePartitionEntry : assignedDataPartition.entrySet()) {
-      SerializeDeserializeUtil.write(seriesPartitionTimePartitionEntry.getKey(), buffer);
+      BasicStructureSerDeUtil.write(seriesPartitionTimePartitionEntry.getKey(), buffer);
       buffer.putInt(seriesPartitionTimePartitionEntry.getValue().size());
-      for (Map.Entry<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>
+      for (Map.Entry<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>
           timePartitionEntry : seriesPartitionTimePartitionEntry.getValue().entrySet()) {
-        timePartitionEntry.getKey().serializeImpl(buffer);
+        ThriftCommonsSerDeUtils.writeTSeriesPartitionSlot(timePartitionEntry.getKey(), buffer);
         buffer.putInt(timePartitionEntry.getValue().size());
-        for (Map.Entry<TimePartitionSlot, List<RegionReplicaSet>> regionReplicaSetEntry :
+        for (Map.Entry<TTimePartitionSlot, List<TRegionReplicaSet>> regionReplicaSetEntry :
             timePartitionEntry.getValue().entrySet()) {
-          regionReplicaSetEntry.getKey().serializeImpl(buffer);
+          ThriftCommonsSerDeUtils.writeTTimePartitionSlot(regionReplicaSetEntry.getKey(), buffer);
           buffer.putInt(regionReplicaSetEntry.getValue().size());
-          for (RegionReplicaSet regionReplicaSet : regionReplicaSetEntry.getValue()) {
-            regionReplicaSet.serializeImpl(buffer);
+          for (TRegionReplicaSet regionReplicaSet : regionReplicaSetEntry.getValue()) {
+            ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet, buffer);
           }
         }
       }
@@ -84,17 +86,17 @@ public class CreateDataPartitionPlan extends PhysicalPlan {
     assignedDataPartition = new HashMap<>();
     int storageGroupNum = buffer.getInt();
     for (int i = 0; i < storageGroupNum; i++) {
-      String storageGroupName = SerializeDeserializeUtil.readString(buffer);
+      String storageGroupName = BasicStructureSerDeUtil.readString(buffer);
       assignedDataPartition.put(storageGroupName, new HashMap<>());
       int seriesPartitionSlotNum = buffer.getInt();
       for (int j = 0; j < seriesPartitionSlotNum; j++) {
-        SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot();
-        seriesPartitionSlot.deserializeImpl(buffer);
+        TSeriesPartitionSlot seriesPartitionSlot =
+            ThriftCommonsSerDeUtils.readTSeriesPartitionSlot(buffer);
         assignedDataPartition.get(storageGroupName).put(seriesPartitionSlot, new HashMap<>());
         int timePartitionSlotNum = buffer.getInt();
         for (int k = 0; k < timePartitionSlotNum; k++) {
-          TimePartitionSlot timePartitionSlot = new TimePartitionSlot();
-          timePartitionSlot.deserializeImpl(buffer);
+          TTimePartitionSlot timePartitionSlot =
+              ThriftCommonsSerDeUtils.readTTimePartitionSlot(buffer);
           assignedDataPartition
               .get(storageGroupName)
               .get(seriesPartitionSlot)
@@ -105,7 +107,7 @@ public class CreateDataPartitionPlan extends PhysicalPlan {
                 .get(storageGroupName)
                 .get(seriesPartitionSlot)
                 .get(timePartitionSlot)
-                .add(RegionReplicaSet.deserializeImpl(buffer));
+                .add(ThriftCommonsSerDeUtils.readTRegionReplicaSet(buffer));
           }
         }
       }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateRegionsPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateRegionsPlan.java
index 56176a70dc..e5ceca415b 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateRegionsPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateRegionsPlan.java
@@ -18,10 +18,11 @@
  */
 package org.apache.iotdb.confignode.physical.crud;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
 import org.apache.iotdb.confignode.physical.PhysicalPlan;
 import org.apache.iotdb.confignode.physical.PhysicalPlanType;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -34,7 +35,7 @@ public class CreateRegionsPlan extends PhysicalPlan {
 
   private String storageGroup;
 
-  private final List<RegionReplicaSet> regionReplicaSets;
+  private final List<TRegionReplicaSet> regionReplicaSets;
 
   public CreateRegionsPlan() {
     super(PhysicalPlanType.CreateRegions);
@@ -49,11 +50,11 @@ public class CreateRegionsPlan extends PhysicalPlan {
     this.storageGroup = storageGroup;
   }
 
-  public void addRegion(RegionReplicaSet regionReplicaSet) {
+  public void addRegion(TRegionReplicaSet regionReplicaSet) {
     this.regionReplicaSets.add(regionReplicaSet);
   }
 
-  public List<RegionReplicaSet> getRegionReplicaSets() {
+  public List<TRegionReplicaSet> getRegionReplicaSets() {
     return regionReplicaSets;
   }
 
@@ -61,21 +62,21 @@ public class CreateRegionsPlan extends PhysicalPlan {
   protected void serializeImpl(ByteBuffer buffer) {
     buffer.putInt(PhysicalPlanType.CreateRegions.ordinal());
 
-    SerializeDeserializeUtil.write(storageGroup, buffer);
+    BasicStructureSerDeUtil.write(storageGroup, buffer);
 
     buffer.putInt(regionReplicaSets.size());
-    for (RegionReplicaSet regionReplicaSet : regionReplicaSets) {
-      regionReplicaSet.serializeImpl(buffer);
+    for (TRegionReplicaSet regionReplicaSet : regionReplicaSets) {
+      ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet, buffer);
     }
   }
 
   @Override
   protected void deserializeImpl(ByteBuffer buffer) throws IOException {
-    storageGroup = SerializeDeserializeUtil.readString(buffer);
+    storageGroup = BasicStructureSerDeUtil.readString(buffer);
 
     int length = buffer.getInt();
     for (int i = 0; i < length; i++) {
-      regionReplicaSets.add(RegionReplicaSet.deserializeImpl(buffer));
+      regionReplicaSets.add(ThriftCommonsSerDeUtils.readTRegionReplicaSet(buffer));
     }
   }
 
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateSchemaPartitionPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateSchemaPartitionPlan.java
index 6a93dfdb6b..417227b920 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateSchemaPartitionPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateSchemaPartitionPlan.java
@@ -18,11 +18,12 @@
  */
 package org.apache.iotdb.confignode.physical.crud;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
 import org.apache.iotdb.confignode.physical.PhysicalPlan;
 import org.apache.iotdb.confignode.physical.PhysicalPlanType;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -33,18 +34,18 @@ import java.util.Objects;
 /** Create SchemaPartition by assignedSchemaPartition */
 public class CreateSchemaPartitionPlan extends PhysicalPlan {
 
-  private Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> assignedSchemaPartition;
+  private Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> assignedSchemaPartition;
 
   public CreateSchemaPartitionPlan() {
     super(PhysicalPlanType.CreateSchemaPartition);
   }
 
-  public Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> getAssignedSchemaPartition() {
+  public Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> getAssignedSchemaPartition() {
     return assignedSchemaPartition;
   }
 
   public void setAssignedSchemaPartition(
-      Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> assignedSchemaPartition) {
+      Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> assignedSchemaPartition) {
     this.assignedSchemaPartition = assignedSchemaPartition;
   }
 
@@ -55,12 +56,12 @@ public class CreateSchemaPartitionPlan extends PhysicalPlan {
     buffer.putInt(assignedSchemaPartition.size());
     assignedSchemaPartition.forEach(
         (storageGroup, partitionSlots) -> {
-          SerializeDeserializeUtil.write(storageGroup, buffer);
+          BasicStructureSerDeUtil.write(storageGroup, buffer);
           buffer.putInt(partitionSlots.size());
           partitionSlots.forEach(
               (seriesPartitionSlot, regionReplicaSet) -> {
-                seriesPartitionSlot.serializeImpl(buffer);
-                regionReplicaSet.serializeImpl(buffer);
+                ThriftCommonsSerDeUtils.writeTSeriesPartitionSlot(seriesPartitionSlot, buffer);
+                ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet, buffer);
               });
         });
   }
@@ -71,15 +72,15 @@ public class CreateSchemaPartitionPlan extends PhysicalPlan {
 
     int storageGroupNum = buffer.getInt();
     for (int i = 0; i < storageGroupNum; i++) {
-      String storageGroup = SerializeDeserializeUtil.readString(buffer);
+      String storageGroup = BasicStructureSerDeUtil.readString(buffer);
       assignedSchemaPartition.put(storageGroup, new HashMap<>());
       int seriesPartitionSlotNum = buffer.getInt();
       for (int j = 0; j < seriesPartitionSlotNum; j++) {
-        SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot();
-        seriesPartitionSlot.deserializeImpl(buffer);
+        TSeriesPartitionSlot seriesPartitionSlot =
+            ThriftCommonsSerDeUtils.readTSeriesPartitionSlot(buffer);
         assignedSchemaPartition
             .get(storageGroup)
-            .put(seriesPartitionSlot, RegionReplicaSet.deserializeImpl(buffer));
+            .put(seriesPartitionSlot, ThriftCommonsSerDeUtils.readTRegionReplicaSet(buffer));
       }
     }
   }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateDataPartitionPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateDataPartitionPlan.java
index 7715e7e6da..8547b7118f 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateDataPartitionPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateDataPartitionPlan.java
@@ -18,13 +18,14 @@
  */
 package org.apache.iotdb.confignode.physical.crud;
 
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
 import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
 import org.apache.iotdb.confignode.physical.PhysicalPlan;
 import org.apache.iotdb.confignode.physical.PhysicalPlanType;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -36,24 +37,24 @@ import java.util.Objects;
 /** Get or create DataPartition by the specific partitionSlotsMap. */
 public class GetOrCreateDataPartitionPlan extends PhysicalPlan {
 
-  private Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> partitionSlotsMap;
+  private Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap;
 
   public GetOrCreateDataPartitionPlan(PhysicalPlanType physicalPlanType) {
     super(physicalPlanType);
   }
 
-  public Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> getPartitionSlotsMap() {
+  public Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> getPartitionSlotsMap() {
     return partitionSlotsMap;
   }
 
   @TestOnly
   public void setPartitionSlotsMap(
-      Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> partitionSlotsMap) {
+      Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap) {
     this.partitionSlotsMap = partitionSlotsMap;
   }
 
   /**
-   * Convert TDataPartitionReq to DataPartitionPlan
+   * Convert TDataPartitionReq to GetOrCreateDataPartitionPlan
    *
    * @param req TDataPartitionReq
    */
@@ -68,19 +69,17 @@ public class GetOrCreateDataPartitionPlan extends PhysicalPlan {
               tSeriesPartitionTimePartitionSlots.forEach(
                   ((tSeriesPartitionSlot, tTimePartitionSlots) -> {
                     // Extract SeriesPartitionSlot
-                    SeriesPartitionSlot seriesPartitionSlot =
-                        new SeriesPartitionSlot(tSeriesPartitionSlot.getSlotId());
                     partitionSlotsMap
                         .get(storageGroup)
-                        .putIfAbsent(seriesPartitionSlot, new ArrayList<>());
+                        .putIfAbsent(tSeriesPartitionSlot, new ArrayList<>());
 
                     // Extract TimePartitionSlots
                     tTimePartitionSlots.forEach(
                         tTimePartitionSlot ->
                             partitionSlotsMap
                                 .get(storageGroup)
-                                .get(seriesPartitionSlot)
-                                .add(new TimePartitionSlot(tTimePartitionSlot.getStartTime())));
+                                .get(tSeriesPartitionSlot)
+                                .add(tTimePartitionSlot));
                   }));
             }));
   }
@@ -92,14 +91,15 @@ public class GetOrCreateDataPartitionPlan extends PhysicalPlan {
     buffer.putInt(partitionSlotsMap.size());
     partitionSlotsMap.forEach(
         ((storageGroup, seriesPartitionTimePartitionSlots) -> {
-          SerializeDeserializeUtil.write(storageGroup, buffer);
+          BasicStructureSerDeUtil.write(storageGroup, buffer);
           buffer.putInt(seriesPartitionTimePartitionSlots.size());
           seriesPartitionTimePartitionSlots.forEach(
               ((seriesPartitionSlot, timePartitionSlots) -> {
-                seriesPartitionSlot.serializeImpl(buffer);
+                ThriftCommonsSerDeUtils.writeTSeriesPartitionSlot(seriesPartitionSlot, buffer);
                 buffer.putInt(timePartitionSlots.size());
                 timePartitionSlots.forEach(
-                    timePartitionSlot -> timePartitionSlot.serializeImpl(buffer));
+                    timePartitionSlot ->
+                        ThriftCommonsSerDeUtils.writeTTimePartitionSlot(timePartitionSlot, buffer));
               }));
         }));
   }
@@ -109,17 +109,17 @@ public class GetOrCreateDataPartitionPlan extends PhysicalPlan {
     partitionSlotsMap = new HashMap<>();
     int storageGroupNum = buffer.getInt();
     for (int i = 0; i < storageGroupNum; i++) {
-      String storageGroup = SerializeDeserializeUtil.readString(buffer);
+      String storageGroup = BasicStructureSerDeUtil.readString(buffer);
       partitionSlotsMap.put(storageGroup, new HashMap<>());
       int seriesPartitionSlotNum = buffer.getInt();
       for (int j = 0; j < seriesPartitionSlotNum; j++) {
-        SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot();
-        seriesPartitionSlot.deserializeImpl(buffer);
+        TSeriesPartitionSlot seriesPartitionSlot =
+            ThriftCommonsSerDeUtils.readTSeriesPartitionSlot(buffer);
         partitionSlotsMap.get(storageGroup).put(seriesPartitionSlot, new ArrayList<>());
         int timePartitionSlotNum = buffer.getInt();
         for (int k = 0; k < timePartitionSlotNum; k++) {
-          TimePartitionSlot timePartitionSlot = new TimePartitionSlot();
-          timePartitionSlot.deserializeImpl(buffer);
+          TTimePartitionSlot timePartitionSlot =
+              ThriftCommonsSerDeUtils.readTTimePartitionSlot(buffer);
           partitionSlotsMap.get(storageGroup).get(seriesPartitionSlot).add(timePartitionSlot);
         }
       }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateSchemaPartitionPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateSchemaPartitionPlan.java
index 5b6e3d5149..2f3df69725 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateSchemaPartitionPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateSchemaPartitionPlan.java
@@ -18,10 +18,11 @@
  */
 package org.apache.iotdb.confignode.physical.crud;
 
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
 import org.apache.iotdb.confignode.physical.PhysicalPlan;
 import org.apache.iotdb.confignode.physical.PhysicalPlanType;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -37,17 +38,17 @@ public class GetOrCreateSchemaPartitionPlan extends PhysicalPlan {
   // Map<StorageGroup, List<SeriesPartitionSlot>>
   // Get all SchemaPartitions when the partitionSlotsMap is empty
   // Get all exists SchemaPartitions in one StorageGroup when the SeriesPartitionSlot is empty
-  private Map<String, List<SeriesPartitionSlot>> partitionSlotsMap;
+  private Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap;
 
   public GetOrCreateSchemaPartitionPlan(PhysicalPlanType physicalPlanType) {
     super(physicalPlanType);
   }
 
-  public void setPartitionSlotsMap(Map<String, List<SeriesPartitionSlot>> partitionSlotsMap) {
+  public void setPartitionSlotsMap(Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap) {
     this.partitionSlotsMap = partitionSlotsMap;
   }
 
-  public Map<String, List<SeriesPartitionSlot>> getPartitionSlotsMap() {
+  public Map<String, List<TSeriesPartitionSlot>> getPartitionSlotsMap() {
     return partitionSlotsMap;
   }
 
@@ -58,10 +59,11 @@ public class GetOrCreateSchemaPartitionPlan extends PhysicalPlan {
     buffer.putInt(partitionSlotsMap.size());
     partitionSlotsMap.forEach(
         (storageGroup, seriesPartitionSlots) -> {
-          SerializeDeserializeUtil.write(storageGroup, buffer);
+          BasicStructureSerDeUtil.write(storageGroup, buffer);
           buffer.putInt(seriesPartitionSlots.size());
           seriesPartitionSlots.forEach(
-              seriesPartitionSlot -> seriesPartitionSlot.serializeImpl(buffer));
+              seriesPartitionSlot ->
+                  ThriftCommonsSerDeUtils.writeTSeriesPartitionSlot(seriesPartitionSlot, buffer));
         });
   }
 
@@ -70,12 +72,12 @@ public class GetOrCreateSchemaPartitionPlan extends PhysicalPlan {
     partitionSlotsMap = new HashMap<>();
     int storageGroupNum = buffer.getInt();
     for (int i = 0; i < storageGroupNum; i++) {
-      String storageGroup = SerializeDeserializeUtil.readString(buffer);
+      String storageGroup = BasicStructureSerDeUtil.readString(buffer);
       partitionSlotsMap.put(storageGroup, new ArrayList<>());
       int seriesPartitionSlotNum = buffer.getInt();
       for (int j = 0; j < seriesPartitionSlotNum; j++) {
-        SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot();
-        seriesPartitionSlot.deserializeImpl(buffer);
+        TSeriesPartitionSlot seriesPartitionSlot =
+            ThriftCommonsSerDeUtils.readTSeriesPartitionSlot(buffer);
         partitionSlotsMap.get(storageGroup).add(seriesPartitionSlot);
       }
     }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/AuthorPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/AuthorPlan.java
index 32f6649736..bc90b6edfb 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/AuthorPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/AuthorPlan.java
@@ -18,9 +18,9 @@
  */
 package org.apache.iotdb.confignode.physical.sys;
 
+import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
 import org.apache.iotdb.confignode.physical.PhysicalPlan;
 import org.apache.iotdb.confignode.physical.PhysicalPlanType;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
 import org.apache.iotdb.db.auth.AuthException;
 
 import java.nio.ByteBuffer;
@@ -132,11 +132,11 @@ public class AuthorPlan extends PhysicalPlan {
 
   @Override
   protected void serializeImpl(ByteBuffer buffer) {
-    SerializeDeserializeUtil.write(getPlanTypeOrdinal(authorType), buffer);
-    SerializeDeserializeUtil.write(userName, buffer);
-    SerializeDeserializeUtil.write(roleName, buffer);
-    SerializeDeserializeUtil.write(password, buffer);
-    SerializeDeserializeUtil.write(newPassword, buffer);
+    BasicStructureSerDeUtil.write(getPlanTypeOrdinal(authorType), buffer);
+    BasicStructureSerDeUtil.write(userName, buffer);
+    BasicStructureSerDeUtil.write(roleName, buffer);
+    BasicStructureSerDeUtil.write(password, buffer);
+    BasicStructureSerDeUtil.write(newPassword, buffer);
     if (permissions == null) {
       buffer.put((byte) 0);
     } else {
@@ -146,15 +146,15 @@ public class AuthorPlan extends PhysicalPlan {
         buffer.putInt(permission);
       }
     }
-    SerializeDeserializeUtil.write(nodeName, buffer);
+    BasicStructureSerDeUtil.write(nodeName, buffer);
   }
 
   @Override
   protected void deserializeImpl(ByteBuffer buffer) {
-    userName = SerializeDeserializeUtil.readString(buffer);
-    roleName = SerializeDeserializeUtil.readString(buffer);
-    password = SerializeDeserializeUtil.readString(buffer);
-    newPassword = SerializeDeserializeUtil.readString(buffer);
+    userName = BasicStructureSerDeUtil.readString(buffer);
+    roleName = BasicStructureSerDeUtil.readString(buffer);
+    password = BasicStructureSerDeUtil.readString(buffer);
+    newPassword = BasicStructureSerDeUtil.readString(buffer);
     byte hasPermissions = buffer.get();
     if (hasPermissions == (byte) 0) {
       this.permissions = null;
@@ -165,7 +165,7 @@ public class AuthorPlan extends PhysicalPlan {
         permissions.add(buffer.getInt());
       }
     }
-    nodeName = SerializeDeserializeUtil.readString(buffer);
+    nodeName = BasicStructureSerDeUtil.readString(buffer);
   }
 
   private int getPlanTypeOrdinal(PhysicalPlanType physicalPlanType) {
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/RegisterDataNodePlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/RegisterDataNodePlan.java
index c4c3fd5d52..01a8199ee6 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/RegisterDataNodePlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/RegisterDataNodePlan.java
@@ -18,8 +18,8 @@
  */
 package org.apache.iotdb.confignode.physical.sys;
 
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
 import org.apache.iotdb.confignode.physical.PhysicalPlan;
 import org.apache.iotdb.confignode.physical.PhysicalPlanType;
 
@@ -28,40 +28,30 @@ import java.util.Objects;
 
 public class RegisterDataNodePlan extends PhysicalPlan {
 
-  private DataNodeLocation info;
+  private TDataNodeLocation location;
 
   public RegisterDataNodePlan() {
     super(PhysicalPlanType.RegisterDataNode);
   }
 
-  public RegisterDataNodePlan(DataNodeLocation info) {
+  public RegisterDataNodePlan(TDataNodeLocation location) {
     this();
-    this.info = info;
+    this.location = location;
   }
 
-  public DataNodeLocation getInfo() {
-    return info;
+  public TDataNodeLocation getLocation() {
+    return location;
   }
 
   @Override
   protected void serializeImpl(ByteBuffer buffer) {
     buffer.putInt(PhysicalPlanType.RegisterDataNode.ordinal());
-    buffer.putInt(info.getDataNodeId());
-    buffer.putInt(info.getEndPoint().getIp().length());
-    buffer.put(info.getEndPoint().getIp().getBytes());
-    buffer.putInt(info.getEndPoint().getPort());
+    ThriftCommonsSerDeUtils.writeTDataNodeLocation(location, buffer);
   }
 
   @Override
   protected void deserializeImpl(ByteBuffer buffer) {
-    int dataNodeID = buffer.getInt();
-    int ipLength = buffer.getInt();
-    byte[] byteIp = new byte[ipLength];
-    buffer.get(byteIp, 0, ipLength);
-    String ip = new String(byteIp, 0, ipLength);
-    int port = buffer.getInt();
-
-    this.info = new DataNodeLocation(dataNodeID, new Endpoint(ip, port));
+    location = ThriftCommonsSerDeUtils.readTDataNodeLocation(buffer);
   }
 
   @Override
@@ -69,11 +59,11 @@ public class RegisterDataNodePlan extends PhysicalPlan {
     if (this == o) return true;
     if (o == null || getClass() != o.getClass()) return false;
     RegisterDataNodePlan plan = (RegisterDataNodePlan) o;
-    return info.equals(plan.info);
+    return location.equals(plan.location);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(info);
+    return Objects.hash(location);
   }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/SetStorageGroupPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/SetStorageGroupPlan.java
index 865829592d..80078672e2 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/SetStorageGroupPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/SetStorageGroupPlan.java
@@ -18,9 +18,10 @@
  */
 package org.apache.iotdb.confignode.physical.sys;
 
-import org.apache.iotdb.confignode.partition.StorageGroupSchema;
+import org.apache.iotdb.commons.utils.ThriftConfigNodeSerDeUtils;
 import org.apache.iotdb.confignode.physical.PhysicalPlan;
 import org.apache.iotdb.confignode.physical.PhysicalPlanType;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -28,35 +29,35 @@ import java.util.Objects;
 
 public class SetStorageGroupPlan extends PhysicalPlan {
 
-  private StorageGroupSchema schema;
+  private TStorageGroupSchema schema;
 
   public SetStorageGroupPlan() {
     super(PhysicalPlanType.SetStorageGroup);
-    this.schema = new StorageGroupSchema();
+    this.schema = new TStorageGroupSchema();
   }
 
-  public SetStorageGroupPlan(StorageGroupSchema schema) {
+  public SetStorageGroupPlan(TStorageGroupSchema schema) {
     this();
     this.schema = schema;
   }
 
-  public StorageGroupSchema getSchema() {
+  public TStorageGroupSchema getSchema() {
     return schema;
   }
 
-  public void setSchema(StorageGroupSchema schema) {
+  public void setSchema(TStorageGroupSchema schema) {
     this.schema = schema;
   }
 
   @Override
   protected void serializeImpl(ByteBuffer buffer) {
     buffer.putInt(PhysicalPlanType.SetStorageGroup.ordinal());
-    schema.serialize(buffer);
+    ThriftConfigNodeSerDeUtils.writeTStorageGroupSchema(schema, buffer);
   }
 
   @Override
   protected void deserializeImpl(ByteBuffer buffer) throws IOException {
-    schema.deserialize(buffer);
+    schema = ThriftConfigNodeSerDeUtils.readTStorageGroupSchema(buffer);
   }
 
   @Override
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessor.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessor.java
index af6647c3b7..99c1de0334 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessor.java
@@ -19,17 +19,14 @@
 package org.apache.iotdb.confignode.service.thrift.server;
 
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
 import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
 import org.apache.iotdb.confignode.consensus.response.DataNodeConfigurationDataSet;
-import org.apache.iotdb.confignode.consensus.response.DataNodesInfoDataSet;
+import org.apache.iotdb.confignode.consensus.response.DataNodeLocationsDataSet;
 import org.apache.iotdb.confignode.consensus.response.DataPartitionDataSet;
 import org.apache.iotdb.confignode.consensus.response.PermissionInfoDataSet;
 import org.apache.iotdb.confignode.consensus.response.SchemaPartitionDataSet;
 import org.apache.iotdb.confignode.consensus.response.StorageGroupSchemaDataSet;
 import org.apache.iotdb.confignode.manager.ConfigManager;
-import org.apache.iotdb.confignode.partition.StorageGroupSchema;
 import org.apache.iotdb.confignode.physical.PhysicalPlanType;
 import org.apache.iotdb.confignode.physical.crud.GetOrCreateDataPartitionPlan;
 import org.apache.iotdb.confignode.physical.sys.AuthorPlan;
@@ -39,7 +36,7 @@ import org.apache.iotdb.confignode.physical.sys.SetStorageGroupPlan;
 import org.apache.iotdb.confignode.rpc.thrift.ConfigIService;
 import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerReq;
 import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerResp;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessageResp;
+import org.apache.iotdb.confignode.rpc.thrift.TDataNodeLocationResp;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterResp;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
@@ -49,6 +46,8 @@ import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionReq;
 import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionResp;
 import org.apache.iotdb.confignode.rpc.thrift.TSetStorageGroupReq;
 import org.apache.iotdb.confignode.rpc.thrift.TSetTTLReq;
+import org.apache.iotdb.confignode.rpc.thrift.TSetTimePartitionIntervalReq;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchemaResp;
 import org.apache.iotdb.db.auth.AuthException;
 import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
@@ -60,6 +59,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 
 /** ConfigNodeRPCServer exposes the interface that interacts with the DataNode */
 public class ConfigNodeRPCServerProcessor implements ConfigIService.Iface {
@@ -78,10 +78,7 @@ public class ConfigNodeRPCServerProcessor implements ConfigIService.Iface {
 
   @Override
   public TDataNodeRegisterResp registerDataNode(TDataNodeRegisterReq req) throws TException {
-    RegisterDataNodePlan plan =
-        new RegisterDataNodePlan(
-            new DataNodeLocation(
-                -1, new Endpoint(req.getEndPoint().getIp(), req.getEndPoint().getPort())));
+    RegisterDataNodePlan plan = new RegisterDataNodePlan(req.getDataNodeLocation());
     DataNodeConfigurationDataSet dataSet =
         (DataNodeConfigurationDataSet) configManager.registerDataNode(plan);
 
@@ -92,23 +89,32 @@ public class ConfigNodeRPCServerProcessor implements ConfigIService.Iface {
   }
 
   @Override
-  public TDataNodeMessageResp getDataNodesMessage(int dataNodeID) throws TException {
+  public TDataNodeLocationResp getDataNodeLocations(int dataNodeID) throws TException {
     QueryDataNodeInfoPlan plan = new QueryDataNodeInfoPlan(dataNodeID);
-    DataNodesInfoDataSet dataSet = (DataNodesInfoDataSet) configManager.getDataNodeInfo(plan);
+    DataNodeLocationsDataSet dataSet =
+        (DataNodeLocationsDataSet) configManager.getDataNodeInfo(plan);
 
-    TDataNodeMessageResp resp = new TDataNodeMessageResp();
-    dataSet.convertToRPCDataNodeMessageResp(resp);
+    TDataNodeLocationResp resp = new TDataNodeLocationResp();
+    dataSet.convertToRpcDataNodeLocationResp(resp);
     return resp;
   }
 
   @Override
   public TSStatus setStorageGroup(TSetStorageGroupReq req) throws TException {
-    SetStorageGroupPlan plan =
-        new SetStorageGroupPlan(new StorageGroupSchema(req.getStorageGroup()));
-
-    // TODO: Set TTL by optional field TSetStorageGroupReq.TTL
-    plan.getSchema().setTTL(ConfigNodeDescriptor.getInstance().getConf().getDefaultTTL());
-
+    TStorageGroupSchema storageGroupSchema = req.getStorageGroup();
+    // TODO: Set this filed by optional fields in TSetStorageGroupReq
+    storageGroupSchema.setTTL(ConfigNodeDescriptor.getInstance().getConf().getDefaultTTL());
+    storageGroupSchema.setSchemaReplicationFactor(
+        ConfigNodeDescriptor.getInstance().getConf().getSchemaReplicationFactor());
+    storageGroupSchema.setDataReplicationFactor(
+        ConfigNodeDescriptor.getInstance().getConf().getDataReplicationFactor());
+    storageGroupSchema.setTimePartitionInterval(
+        ConfigNodeDescriptor.getInstance().getConf().getTimePartitionInterval());
+
+    storageGroupSchema.setSchemaRegionGroupIds(new ArrayList<>());
+    storageGroupSchema.setDataRegionGroupIds(new ArrayList<>());
+
+    SetStorageGroupPlan plan = new SetStorageGroupPlan(storageGroupSchema);
     return configManager.setStorageGroup(plan);
   }
 
@@ -124,6 +130,12 @@ public class ConfigNodeRPCServerProcessor implements ConfigIService.Iface {
     return null;
   }
 
+  @Override
+  public TSStatus setTimePartitionInterval(TSetTimePartitionIntervalReq req) throws TException {
+    // TODO: Set TimePartitionInterval
+    return null;
+  }
+
   @Override
   public TStorageGroupSchemaResp getStorageGroupsSchema() throws TException {
     StorageGroupSchemaDataSet dataSet =
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/cli/TemporaryClientDemo.java b/confignode/src/test/java/org/apache/iotdb/confignode/cli/TemporaryClientDemo.java
index 245c0b0acc..2071b75277 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/cli/TemporaryClientDemo.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/cli/TemporaryClientDemo.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.confignode.cli;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.confignode.rpc.thrift.ConfigIService;
 import org.apache.iotdb.confignode.rpc.thrift.TSetStorageGroupReq;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.rpc.RpcTransportFactory;
 import org.apache.iotdb.rpc.TSStatusCode;
 
@@ -46,7 +47,8 @@ public class TemporaryClientDemo {
     defaultClient = clients.get(22277);
 
     for (int i = 0; i < 5; i++) {
-      TSetStorageGroupReq setReq = new TSetStorageGroupReq("root.sg" + i);
+      TSetStorageGroupReq setReq =
+          new TSetStorageGroupReq(new TStorageGroupSchema().setName("root.sg" + i));
       while (true) {
         TSStatus status = defaultClient.setStorageGroup(setReq);
         System.out.println(status.toString());
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/consensus/RatisConsensusDemo.java b/confignode/src/test/java/org/apache/iotdb/confignode/consensus/RatisConsensusDemo.java
deleted file mode 100644
index 6956d486ed..0000000000
--- a/confignode/src/test/java/org/apache/iotdb/confignode/consensus/RatisConsensusDemo.java
+++ /dev/null
@@ -1,151 +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.confignode.consensus;
-
-import org.apache.iotdb.common.rpc.thrift.TEndPoint;
-import org.apache.iotdb.confignode.rpc.thrift.ConfigIService;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessageResp;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterResp;
-import org.apache.iotdb.confignode.rpc.thrift.TSetStorageGroupReq;
-import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
-import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchemaResp;
-import org.apache.iotdb.rpc.RpcTransportFactory;
-import org.apache.iotdb.rpc.TSStatusCode;
-
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.junit.Assert;
-
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-/** Demo for ConfigNode's integration with the ratis-consensus protocol. */
-public class RatisConsensusDemo {
-
-  private static final String localhost = "0.0.0.0";
-  private static final int timeOutInMS = 2000;
-
-  private ConfigIService.Client[] clients;
-
-  /**
-   * To run this code, follow these steps: 1. Compile IoTDB 2. Copy at least three
-   * iotdb-confignode-0.14.0-SNAPSHOT 3. Make sure these parameters: config_node_rpc_address(all
-   * 0.0.0.0), config_node_rpc_port(22277, 22279, 22281), config_node_internal_port(22278, 22280,
-   * 22282), consensus_type(all ratis) and config_node_group_address_list(all 0.0.0.0:22278,
-   * 0.0.0.0:22280, 0.0.0.0:22282) in each iotdb-confignode.properties file are set 4. Start these
-   * ConfigNode by yourself 5. Add @Test 6. run ratisConsensusRegisterDemo 7. run
-   * ratisConsensusQueryDemo
-   */
-  public void ratisConsensusSetStorageGroupsDemo() throws TException, InterruptedException {
-    createClients();
-    setStorageGroups();
-  }
-
-  public void ratisConsensusQueryDataNodesDemo() throws TException, InterruptedException {
-    createClients();
-    queryDataNodes();
-  }
-
-  public void ratisConsensusQueryStorageGroupsDemo() throws TException, InterruptedException {
-    createClients();
-    queryStorageGroups();
-  }
-
-  public void ratisConsensusLeaderRedirectDemo() throws TException {
-    createClients();
-    registerDataNodeOnLeader();
-  }
-
-  private void createClients() throws TTransportException {
-    // Create clients for these three ConfigNodes
-    // to simulate DataNodes to send RPC requests
-    clients = new ConfigIService.Client[3];
-    for (int i = 0; i < 3; i++) {
-      TTransport transport =
-          RpcTransportFactory.INSTANCE.getTransport(localhost, 22277 + i * 2, timeOutInMS);
-      transport.open();
-      clients[i] = new ConfigIService.Client(new TBinaryProtocol(transport));
-    }
-  }
-
-  private void registerDataNodes() throws TException, InterruptedException {
-    // DataNodes can connect to any ConfigNode and send write requests
-    for (int i = 0; i < 10; i++) {
-      TEndPoint endPoint = new TEndPoint("0.0.0.0", 6667 + i);
-      TDataNodeRegisterReq req = new TDataNodeRegisterReq(endPoint);
-      TDataNodeRegisterResp resp = clients[0].registerDataNode(req);
-      Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
-      Assert.assertEquals(i, resp.getDataNodeID());
-      System.out.printf(
-          "\nRegister DataNode successful. DataNodeID: %d, %s\n", resp.getDataNodeID(), endPoint);
-
-      TimeUnit.SECONDS.sleep(1);
-    }
-  }
-
-  private void queryDataNodes() throws InterruptedException, TException {
-    // sleep 1s to make sure all ConfigNode in ConfigNodeGroup hold the same PartitionTable
-    TimeUnit.SECONDS.sleep(1);
-
-    // DataNodes can connect to any ConfigNode and send read requests
-    for (int i = 0; i < 3; i++) {
-      TDataNodeMessageResp msgMap = clients[i].getDataNodesMessage(-1);
-      System.out.printf(
-          "\nQuery DataNode message from ConfigNode 0.0.0.0:%d. Result: %s\n",
-          22277 + i * 2, msgMap);
-    }
-  }
-
-  private void setStorageGroups() throws TException, InterruptedException {
-    for (int i = 0; i < 10; i++) {
-      TSetStorageGroupReq req = new TSetStorageGroupReq("root.sg" + i);
-      clients[0].setStorageGroup(req);
-      System.out.printf("\nSet StorageGroup successful. StorageGroup: %s\n", "root.sg" + i);
-      TimeUnit.SECONDS.sleep(1);
-    }
-  }
-
-  private void queryStorageGroups() throws InterruptedException, TException {
-    // sleep 1s to make sure all ConfigNode in ConfigNodeGroup hold the same PartitionTable
-    TimeUnit.SECONDS.sleep(1);
-
-    for (int i = 0; i < 3; i++) {
-      TStorageGroupSchemaResp msgMap = clients[i].getStorageGroupsSchema();
-      System.out.printf(
-          "\nQuery StorageGroup message from ConfigNode 0.0.0.0:%d. Result: {\n", 22277 + i * 2);
-      for (Map.Entry<String, TStorageGroupSchema> entry :
-          msgMap.getStorageGroupSchemaMap().entrySet()) {
-        System.out.printf("  Key(%s)=%s\n", entry.getKey(), entry.getValue().toString());
-      }
-      System.out.println("}");
-    }
-  }
-
-  private void registerDataNodeOnLeader() throws TException {
-    for (int i = 0; i < 3; i++) {
-      TEndPoint endPoint = new TEndPoint("0.0.0.0", 6667);
-      TDataNodeRegisterReq req = new TDataNodeRegisterReq(endPoint);
-      TDataNodeRegisterResp resp = clients[i].registerDataNode(req);
-      System.out.println(resp);
-    }
-  }
-}
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/manager/ConfigManagerManualTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/manager/ConfigManagerManualTest.java
deleted file mode 100644
index d7d54b355f..0000000000
--- a/confignode/src/test/java/org/apache/iotdb/confignode/manager/ConfigManagerManualTest.java
+++ /dev/null
@@ -1,133 +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.confignode.manager;
-
-import org.apache.iotdb.common.rpc.thrift.TEndPoint;
-import org.apache.iotdb.confignode.rpc.thrift.ConfigIService;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessage;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterResp;
-import org.apache.iotdb.rpc.RpcTransportFactory;
-import org.apache.iotdb.rpc.TSStatusCode;
-
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.junit.Assert;
-
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-public class ConfigManagerManualTest {
-
-  // TODO: Optimize this manual test to automatic test after the test environment is set up.
-  // @YongzaoDan
-
-  private static final String localhost = "0.0.0.0";
-  private static final int timeOutInMS = 2000;
-
-  private ConfigIService.Client[] clients;
-
-  /**
-   * This is a temporary test of ConfigNode's integration with the ratis-consensus protocol. To run
-   * this code, follow these steps: 1. Compile IoTDB 2. Copy at least three
-   * iotdb-confignode-0.14.0-SNAPSHOT 3. Make sure these parameters: config_node_rpc_address(all
-   * 0.0.0.0), config_node_rpc_port(22277, 22279, 22281), config_node_internal_port(22278, 22280,
-   * 22282), consensus_type(all ratis) and config_node_group_address_list(all 0.0.0.0:22278,
-   * 0.0.0.0:22280, 0.0.0.0:22282) in each iotdb-confignode.properties file are set 4. Start these
-   * ConfigNode by yourself 5. Add @Test and run
-   */
-  public void ratisConsensusTest() throws TException, InterruptedException {
-    createClients();
-
-    registerDataNodes();
-
-    queryDataNodes();
-  }
-
-  private void createClients() throws TTransportException {
-    clients = new ConfigIService.Client[3];
-    for (int i = 0; i < 3; i++) {
-      TTransport transport =
-          RpcTransportFactory.INSTANCE.getTransport(localhost, 22277 + i * 2, timeOutInMS);
-      transport.open();
-      clients[i] = new ConfigIService.Client(new TBinaryProtocol(transport));
-    }
-  }
-
-  private void registerDataNodes() throws TException {
-    for (int i = 0; i < 3; i++) {
-      TDataNodeRegisterReq req = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6667 + i));
-      TDataNodeRegisterResp resp = clients[0].registerDataNode(req);
-      Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
-      Assert.assertEquals(i, resp.getDataNodeID());
-    }
-  }
-
-  private void queryDataNodes() throws InterruptedException, TException {
-    // sleep 1s to make sure all ConfigNode in ConfigNodeGroup hold the same PartitionTable
-    TimeUnit.SECONDS.sleep(1);
-
-    for (int i = 0; i < 3; i++) {
-      Map<Integer, TDataNodeMessage> msgMap =
-          clients[i].getDataNodesMessage(-1).getDataNodeMessageMap();
-      Assert.assertEquals(3, msgMap.size());
-      for (int j = 0; j < 3; j++) {
-        Assert.assertNotNull(msgMap.get(j));
-        Assert.assertEquals(j, msgMap.get(j).getDataNodeId());
-        Assert.assertEquals(localhost, msgMap.get(j).getEndPoint().getIp());
-        Assert.assertEquals(6667 + j, msgMap.get(j).getEndPoint().getPort());
-      }
-    }
-  }
-
-  /**
-   * This is a temporary test of ConfigNode's integration with the ratis-consensus protocol. This
-   * code tests the high availability of the ratis-consensus protocol. Make sure that you have run
-   * according to the comments of ratisConsensusTest before executing this code. Next, close
-   * ConfigNode that occupies ports 22281 and 22282 on the local machine. Finally, run this test.
-   */
-  public void killTest() throws TException {
-    clients = new ConfigIService.Client[2];
-    for (int i = 0; i < 2; i++) {
-      TTransport transport =
-          RpcTransportFactory.INSTANCE.getTransport(localhost, 22277 + i * 2, timeOutInMS);
-      transport.open();
-      clients[i] = new ConfigIService.Client(new TBinaryProtocol(transport));
-    }
-
-    TDataNodeRegisterResp resp =
-        clients[1].registerDataNode(new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6670)));
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
-    Assert.assertEquals(3, resp.getDataNodeID());
-
-    for (int i = 0; i < 2; i++) {
-      Map<Integer, TDataNodeMessage> msgMap =
-          clients[i].getDataNodesMessage(-1).getDataNodeMessageMap();
-      Assert.assertEquals(4, msgMap.size());
-      for (int j = 0; j < 4; j++) {
-        Assert.assertNotNull(msgMap.get(j));
-        Assert.assertEquals(j, msgMap.get(j).getDataNodeId());
-        Assert.assertEquals(localhost, msgMap.get(j).getEndPoint().getIp());
-        Assert.assertEquals(6667 + j, msgMap.get(j).getEndPoint().getPort());
-      }
-    }
-  }
-}
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/physical/SerializeDeserializeUT.java b/confignode/src/test/java/org/apache/iotdb/confignode/physical/PhysicalPlanSerDeTest.java
similarity index 73%
rename from confignode/src/test/java/org/apache/iotdb/confignode/physical/SerializeDeserializeUT.java
rename to confignode/src/test/java/org/apache/iotdb/confignode/physical/PhysicalPlanSerDeTest.java
index 3dc71321d7..78386bee94 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/physical/SerializeDeserializeUT.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/physical/PhysicalPlanSerDeTest.java
@@ -18,14 +18,13 @@
  */
 package org.apache.iotdb.confignode.physical;
 
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.SchemaRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
-import org.apache.iotdb.confignode.partition.StorageGroupSchema;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.confignode.physical.crud.CreateDataPartitionPlan;
 import org.apache.iotdb.confignode.physical.crud.CreateRegionsPlan;
 import org.apache.iotdb.confignode.physical.crud.CreateSchemaPartitionPlan;
@@ -35,6 +34,7 @@ import org.apache.iotdb.confignode.physical.sys.AuthorPlan;
 import org.apache.iotdb.confignode.physical.sys.QueryDataNodeInfoPlan;
 import org.apache.iotdb.confignode.physical.sys.RegisterDataNodePlan;
 import org.apache.iotdb.confignode.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.db.auth.AuthException;
 import org.apache.iotdb.db.auth.entity.PrivilegeType;
 
@@ -52,7 +52,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-public class SerializeDeserializeUT {
+public class PhysicalPlanSerDeTest {
 
   private final ByteBuffer buffer = ByteBuffer.allocate(10240);
 
@@ -63,8 +63,13 @@ public class SerializeDeserializeUT {
 
   @Test
   public void RegisterDataNodePlanTest() throws IOException {
-    RegisterDataNodePlan plan0 =
-        new RegisterDataNodePlan(new DataNodeLocation(1, new Endpoint("0.0.0.0", 6667)));
+    TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+    dataNodeLocation.setDataNodeId(1);
+    dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+    dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+    dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+    dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 7777));
+    RegisterDataNodePlan plan0 = new RegisterDataNodePlan(dataNodeLocation);
     plan0.serialize(buffer);
     buffer.flip();
     RegisterDataNodePlan plan1 = (RegisterDataNodePlan) PhysicalPlan.Factory.create(buffer);
@@ -82,7 +87,16 @@ public class SerializeDeserializeUT {
 
   @Test
   public void SetStorageGroupPlanTest() throws IOException {
-    SetStorageGroupPlan plan0 = new SetStorageGroupPlan(new StorageGroupSchema("sg"));
+    SetStorageGroupPlan plan0 =
+        new SetStorageGroupPlan(
+            new TStorageGroupSchema()
+                .setName("sg")
+                .setTTL(Long.MAX_VALUE)
+                .setSchemaReplicationFactor(3)
+                .setDataReplicationFactor(3)
+                .setTimePartitionInterval(604800)
+                .setSchemaRegionGroupIds(new ArrayList<>())
+                .setDataRegionGroupIds(new ArrayList<>()));
     plan0.serialize(buffer);
     buffer.flip();
     SetStorageGroupPlan plan1 = (SetStorageGroupPlan) PhysicalPlan.Factory.create(buffer);
@@ -96,17 +110,23 @@ public class SerializeDeserializeUT {
 
   @Test
   public void CreateRegionsPlanTest() throws IOException {
+    TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+    dataNodeLocation.setDataNodeId(0);
+    dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+    dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+    dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+    dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010));
+
     CreateRegionsPlan plan0 = new CreateRegionsPlan();
     plan0.setStorageGroup("sg");
-    RegionReplicaSet dataRegionSet = new RegionReplicaSet();
-    dataRegionSet.setConsensusGroupId(new DataRegionId(0));
-    dataRegionSet.setDataNodeList(
-        Collections.singletonList(new DataNodeLocation(0, new Endpoint("0.0.0.0", 6667))));
+    TRegionReplicaSet dataRegionSet = new TRegionReplicaSet();
+    dataRegionSet.setRegionId(new TConsensusGroupId(TConsensusGroupType.DataRegion, 0));
+    dataRegionSet.setDataNodeLocations(Collections.singletonList(dataNodeLocation));
     plan0.addRegion(dataRegionSet);
-    RegionReplicaSet schemaRegionSet = new RegionReplicaSet();
-    schemaRegionSet.setConsensusGroupId(new SchemaRegionId(1));
-    schemaRegionSet.setDataNodeList(
-        Collections.singletonList(new DataNodeLocation(0, new Endpoint("0.0.0.0", 6667))));
+
+    TRegionReplicaSet schemaRegionSet = new TRegionReplicaSet();
+    schemaRegionSet.setRegionId(new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 1));
+    schemaRegionSet.setDataNodeLocations(Collections.singletonList(dataNodeLocation));
     plan0.addRegion(schemaRegionSet);
 
     plan0.serialize(buffer);
@@ -117,14 +137,20 @@ public class SerializeDeserializeUT {
 
   @Test
   public void CreateSchemaPartitionPlanTest() throws IOException {
+    TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+    dataNodeLocation.setDataNodeId(0);
+    dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+    dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+    dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+    dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010));
+
     String storageGroup = "root.sg0";
-    SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot(10);
-    RegionReplicaSet regionReplicaSet = new RegionReplicaSet();
-    regionReplicaSet.setConsensusGroupId(new SchemaRegionId(0));
-    regionReplicaSet.setDataNodeList(
-        Collections.singletonList(new DataNodeLocation(0, new Endpoint("0.0.0.0", 6667))));
+    TSeriesPartitionSlot seriesPartitionSlot = new TSeriesPartitionSlot(10);
+    TRegionReplicaSet regionReplicaSet = new TRegionReplicaSet();
+    regionReplicaSet.setRegionId(new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 0));
+    regionReplicaSet.setDataNodeLocations(Collections.singletonList(dataNodeLocation));
 
-    Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> assignedSchemaPartition =
+    Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> assignedSchemaPartition =
         new HashMap<>();
     assignedSchemaPartition.put(storageGroup, new HashMap<>());
     assignedSchemaPartition.get(storageGroup).put(seriesPartitionSlot, regionReplicaSet);
@@ -141,9 +167,9 @@ public class SerializeDeserializeUT {
   @Test
   public void GetOrCreateSchemaPartitionPlanTest() throws IOException {
     String storageGroup = "root.sg0";
-    SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot(10);
+    TSeriesPartitionSlot seriesPartitionSlot = new TSeriesPartitionSlot(10);
 
-    Map<String, List<SeriesPartitionSlot>> partitionSlotsMap = new HashMap<>();
+    Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap = new HashMap<>();
     partitionSlotsMap.put(storageGroup, Collections.singletonList(seriesPartitionSlot));
 
     GetOrCreateSchemaPartitionPlan plan0 =
@@ -158,15 +184,21 @@ public class SerializeDeserializeUT {
 
   @Test
   public void CreateDataPartitionPlanTest() throws IOException {
+    TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+    dataNodeLocation.setDataNodeId(0);
+    dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+    dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+    dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+    dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010));
+
     String storageGroup = "root.sg0";
-    SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot(10);
-    TimePartitionSlot timePartitionSlot = new TimePartitionSlot(100);
-    RegionReplicaSet regionReplicaSet = new RegionReplicaSet();
-    regionReplicaSet.setConsensusGroupId(new DataRegionId(0));
-    regionReplicaSet.setDataNodeList(
-        Collections.singletonList(new DataNodeLocation(0, new Endpoint("0.0.0.0", 6667))));
-
-    Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+    TSeriesPartitionSlot seriesPartitionSlot = new TSeriesPartitionSlot(10);
+    TTimePartitionSlot timePartitionSlot = new TTimePartitionSlot(100);
+    TRegionReplicaSet regionReplicaSet = new TRegionReplicaSet();
+    regionReplicaSet.setRegionId(new TConsensusGroupId(TConsensusGroupType.DataRegion, 0));
+    regionReplicaSet.setDataNodeLocations(Collections.singletonList(dataNodeLocation));
+
+    Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
         assignedDataPartition = new HashMap<>();
     assignedDataPartition.put(storageGroup, new HashMap<>());
     assignedDataPartition.get(storageGroup).put(seriesPartitionSlot, new HashMap<>());
@@ -191,10 +223,10 @@ public class SerializeDeserializeUT {
   @Test
   public void GetOrCreateDataPartitionPlanTest() throws IOException {
     String storageGroup = "root.sg0";
-    SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot(10);
-    TimePartitionSlot timePartitionSlot = new TimePartitionSlot(100);
+    TSeriesPartitionSlot seriesPartitionSlot = new TSeriesPartitionSlot(10);
+    TTimePartitionSlot timePartitionSlot = new TTimePartitionSlot(100);
 
-    Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> partitionSlotsMap =
+    Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap =
         new HashMap<>();
     partitionSlotsMap.put(storageGroup, new HashMap<>());
     partitionSlotsMap.get(storageGroup).put(seriesPartitionSlot, new ArrayList<>());
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessorTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessorTest.java
index 4cfd90e65f..e40f601a01 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessorTest.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessorTest.java
@@ -18,23 +18,21 @@
  */
 package org.apache.iotdb.confignode.service.thrift.server;
 
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
 import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.commons.consensus.ConsensusGroupId;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.SchemaRegionId;
 import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
 import org.apache.iotdb.confignode.persistence.DataNodeInfoPersistence;
 import org.apache.iotdb.confignode.persistence.PartitionInfoPersistence;
 import org.apache.iotdb.confignode.persistence.RegionInfoPersistence;
 import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerReq;
 import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerResp;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessage;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessageResp;
+import org.apache.iotdb.confignode.rpc.thrift.TDataNodeLocationResp;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterResp;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
@@ -104,56 +102,71 @@ public class ConfigNodeRPCServerProcessorTest {
         globalConfig.getSeriesPartitionExecutorClass());
   }
 
+  private void registerDataNodes() throws TException {
+    for (int i = 0; i < 3; i++) {
+      TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+      dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667 + i));
+      dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003 + i));
+      dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777 + i));
+      dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010 + i));
+
+      TDataNodeRegisterReq req = new TDataNodeRegisterReq(dataNodeLocation);
+      TDataNodeRegisterResp resp = processor.registerDataNode(req);
+
+      Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
+      Assert.assertEquals(i, resp.getDataNodeId());
+      checkGlobalConfig(resp.getGlobalConfig());
+    }
+  }
+
   @Test
   public void registerAndQueryDataNodeTest() throws TException {
-    TDataNodeRegisterResp resp;
-    TDataNodeRegisterReq registerReq0 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6667));
-    TDataNodeRegisterReq registerReq1 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6668));
-    TDataNodeRegisterReq registerReq2 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6669));
-
-    // test success register
-    resp = processor.registerDataNode(registerReq0);
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
-    Assert.assertEquals(0, resp.getDataNodeID());
-    checkGlobalConfig(resp.getGlobalConfig());
-
-    resp = processor.registerDataNode(registerReq1);
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
-    Assert.assertEquals(1, resp.getDataNodeID());
-    checkGlobalConfig(resp.getGlobalConfig());
-
-    resp = processor.registerDataNode(registerReq2);
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
-    Assert.assertEquals(2, resp.getDataNodeID());
-    checkGlobalConfig(resp.getGlobalConfig());
+    registerDataNodes();
 
     // test success re-register
-    resp = processor.registerDataNode(registerReq1);
+    TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+    dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6668));
+    dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9004));
+    dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8778));
+    dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40011));
+
+    TDataNodeRegisterReq req = new TDataNodeRegisterReq(dataNodeLocation);
+    TDataNodeRegisterResp resp = processor.registerDataNode(req);
     Assert.assertEquals(
         TSStatusCode.DATANODE_ALREADY_REGISTERED.getStatusCode(), resp.getStatus().getCode());
-    Assert.assertEquals(1, resp.getDataNodeID());
+    Assert.assertEquals(1, resp.getDataNodeId());
     checkGlobalConfig(resp.getGlobalConfig());
 
     // test query DataNodeInfo
-    TDataNodeMessageResp msgResp = processor.getDataNodesMessage(-1);
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), msgResp.getStatus().getCode());
-    Map<Integer, TDataNodeMessage> msgMap = msgResp.getDataNodeMessageMap();
-    Assert.assertEquals(3, msgMap.size());
-    List<Map.Entry<Integer, TDataNodeMessage>> messageList = new ArrayList<>(msgMap.entrySet());
-    messageList.sort(Comparator.comparingInt(Map.Entry::getKey));
+    TDataNodeLocationResp locationResp = processor.getDataNodeLocations(-1);
+    Assert.assertEquals(
+        TSStatusCode.SUCCESS_STATUS.getStatusCode(), locationResp.getStatus().getCode());
+    Map<Integer, TDataNodeLocation> locationMap = locationResp.getDataNodeLocationMap();
+    Assert.assertEquals(3, locationMap.size());
+    List<Map.Entry<Integer, TDataNodeLocation>> locationList =
+        new ArrayList<>(locationMap.entrySet());
+    locationList.sort(Comparator.comparingInt(Map.Entry::getKey));
     for (int i = 0; i < 3; i++) {
-      Assert.assertEquals(i, messageList.get(i).getValue().getDataNodeId());
-      Assert.assertEquals("0.0.0.0", messageList.get(i).getValue().getEndPoint().getIp());
-      Assert.assertEquals(6667 + i, messageList.get(i).getValue().getEndPoint().getPort());
+      dataNodeLocation.setDataNodeId(i);
+      dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667 + i));
+      dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003 + i));
+      dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777 + i));
+      dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010 + i));
+      Assert.assertEquals(dataNodeLocation, locationList.get(i).getValue());
     }
 
-    msgResp = processor.getDataNodesMessage(1);
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), msgResp.getStatus().getCode());
-    msgMap = msgResp.getDataNodeMessageMap();
-    Assert.assertEquals(1, msgMap.size());
-    Assert.assertNotNull(msgMap.get(1));
-    Assert.assertEquals("0.0.0.0", msgMap.get(1).getEndPoint().getIp());
-    Assert.assertEquals(6668, msgMap.get(1).getEndPoint().getPort());
+    locationResp = processor.getDataNodeLocations(1);
+    Assert.assertEquals(
+        TSStatusCode.SUCCESS_STATUS.getStatusCode(), locationResp.getStatus().getCode());
+    locationMap = locationResp.getDataNodeLocationMap();
+    Assert.assertEquals(1, locationMap.size());
+    Assert.assertNotNull(locationMap.get(1));
+    dataNodeLocation.setDataNodeId(1);
+    dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6668));
+    dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9004));
+    dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8778));
+    dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40011));
+    Assert.assertEquals(dataNodeLocation, locationMap.get(1));
   }
 
   @Test
@@ -162,21 +175,13 @@ public class ConfigNodeRPCServerProcessorTest {
     final String sg = "root.sg0";
 
     // failed because there are not enough DataNodes
-    TSetStorageGroupReq setReq = new TSetStorageGroupReq(sg);
+    TSetStorageGroupReq setReq = new TSetStorageGroupReq(new TStorageGroupSchema(sg));
     status = processor.setStorageGroup(setReq);
     Assert.assertEquals(TSStatusCode.NOT_ENOUGH_DATA_NODE.getStatusCode(), status.getCode());
     Assert.assertEquals("DataNode is not enough, please register more.", status.getMessage());
 
     // register DataNodes
-    TDataNodeRegisterReq registerReq0 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6667));
-    TDataNodeRegisterReq registerReq1 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6668));
-    TDataNodeRegisterReq registerReq2 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6669));
-    status = processor.registerDataNode(registerReq0).getStatus();
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
-    status = processor.registerDataNode(registerReq1).getStatus();
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
-    status = processor.registerDataNode(registerReq2).getStatus();
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
+    registerDataNodes();
 
     // set StorageGroup
     status = processor.setStorageGroup(setReq);
@@ -185,10 +190,15 @@ public class ConfigNodeRPCServerProcessorTest {
     // query StorageGroupSchema
     TStorageGroupSchemaResp resp = processor.getStorageGroupsSchema();
     Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
-    Map<String, TStorageGroupSchema> msgMap = resp.getStorageGroupSchemaMap();
-    Assert.assertEquals(1, msgMap.size());
-    Assert.assertNotNull(msgMap.get(sg));
-    Assert.assertEquals(sg, msgMap.get(sg).getStorageGroup());
+    Map<String, TStorageGroupSchema> schemaMap = resp.getStorageGroupSchemaMap();
+    Assert.assertEquals(1, schemaMap.size());
+    TStorageGroupSchema storageGroupSchema = schemaMap.get(sg);
+    Assert.assertNotNull(storageGroupSchema);
+    Assert.assertEquals(sg, storageGroupSchema.getName());
+    Assert.assertEquals(Long.MAX_VALUE, storageGroupSchema.getTTL());
+    Assert.assertEquals(3, storageGroupSchema.getSchemaReplicationFactor());
+    Assert.assertEquals(3, storageGroupSchema.getDataReplicationFactor());
+    Assert.assertEquals(604800, storageGroupSchema.getTimePartitionInterval());
 
     // test fail by re-register
     status = processor.setStorageGroup(setReq);
@@ -234,20 +244,12 @@ public class ConfigNodeRPCServerProcessorTest {
     Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap;
 
     // register DataNodes
-    TDataNodeRegisterReq registerReq0 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6667));
-    TDataNodeRegisterReq registerReq1 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6668));
-    TDataNodeRegisterReq registerReq2 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6669));
-    status = processor.registerDataNode(registerReq0).getStatus();
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
-    status = processor.registerDataNode(registerReq1).getStatus();
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
-    status = processor.registerDataNode(registerReq2).getStatus();
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
+    registerDataNodes();
 
     // Set StorageGroups
-    status = processor.setStorageGroup(new TSetStorageGroupReq(sg0));
+    status = processor.setStorageGroup(new TSetStorageGroupReq(new TStorageGroupSchema(sg0)));
     Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
-    status = processor.setStorageGroup(new TSetStorageGroupReq(sg1));
+    status = processor.setStorageGroup(new TSetStorageGroupReq(new TStorageGroupSchema(sg1)));
     Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
 
     // Test getSchemaPartition, the result should be empty
@@ -273,13 +275,9 @@ public class ConfigNodeRPCServerProcessorTest {
           .get(sg + i)
           .forEach(
               (tSeriesPartitionSlot, tRegionReplicaSet) -> {
-                Assert.assertEquals(3, tRegionReplicaSet.getEndpointSize());
-                ConsensusGroupId regionId = null;
-                regionId =
-                    ConsensusGroupId.Factory.create(
-                        ByteBuffer.wrap(tRegionReplicaSet.getRegionId()));
-
-                Assert.assertTrue(regionId instanceof SchemaRegionId);
+                Assert.assertEquals(3, tRegionReplicaSet.getDataNodeLocationsSize());
+                Assert.assertEquals(
+                    TConsensusGroupType.SchemaRegion, tRegionReplicaSet.getRegionId().getType());
               });
     }
 
@@ -300,14 +298,9 @@ public class ConfigNodeRPCServerProcessorTest {
           .get(sg + i)
           .forEach(
               (tSeriesPartitionSlot, tRegionReplicaSet) -> {
-                Assert.assertEquals(3, tRegionReplicaSet.getEndpointSize());
-                ConsensusGroupId regionId = null;
-
-                regionId =
-                    ConsensusGroupId.Factory.create(
-                        ByteBuffer.wrap(tRegionReplicaSet.getRegionId()));
-
-                Assert.assertTrue(regionId instanceof SchemaRegionId);
+                Assert.assertEquals(3, tRegionReplicaSet.getDataNodeLocationsSize());
+                Assert.assertEquals(
+                    TConsensusGroupType.SchemaRegion, tRegionReplicaSet.getRegionId().getType());
               });
     }
 
@@ -327,11 +320,9 @@ public class ConfigNodeRPCServerProcessorTest {
         .get(sg0)
         .forEach(
             (tSeriesPartitionSlot, tRegionReplicaSet) -> {
-              Assert.assertEquals(3, tRegionReplicaSet.getEndpointSize());
-              ConsensusGroupId regionId = null;
-              regionId =
-                  ConsensusGroupId.Factory.create(ByteBuffer.wrap(tRegionReplicaSet.getRegionId()));
-              Assert.assertTrue(regionId instanceof SchemaRegionId);
+              Assert.assertEquals(3, tRegionReplicaSet.getDataNodeLocationsSize());
+              Assert.assertEquals(
+                  TConsensusGroupType.SchemaRegion, tRegionReplicaSet.getRegionId().getType());
             });
     // Check "root.sg1"
     Assert.assertTrue(schemaPartitionMap.containsKey(sg1));
@@ -340,11 +331,9 @@ public class ConfigNodeRPCServerProcessorTest {
         .get(sg1)
         .forEach(
             (tSeriesPartitionSlot, tRegionReplicaSet) -> {
-              Assert.assertEquals(3, tRegionReplicaSet.getEndpointSize());
-              ConsensusGroupId regionId = null;
-              regionId =
-                  ConsensusGroupId.Factory.create(ByteBuffer.wrap(tRegionReplicaSet.getRegionId()));
-              Assert.assertTrue(regionId instanceof SchemaRegionId);
+              Assert.assertEquals(3, tRegionReplicaSet.getDataNodeLocationsSize());
+              Assert.assertEquals(
+                  TConsensusGroupType.SchemaRegion, tRegionReplicaSet.getRegionId().getType());
             });
   }
 
@@ -404,17 +393,15 @@ public class ConfigNodeRPCServerProcessorTest {
                   .get(timePartitionSlot)
                   .size());
           // Is DataRegion
-          ConsensusGroupId regionId = null;
-          regionId =
-              ConsensusGroupId.Factory.create(
-                  ByteBuffer.wrap(
-                      dataPartitionMap
-                          .get(storageGroup)
-                          .get(seriesPartitionSlot)
-                          .get(timePartitionSlot)
-                          .get(0)
-                          .getRegionId()));
-          Assert.assertTrue(regionId instanceof DataRegionId);
+          Assert.assertEquals(
+              TConsensusGroupType.DataRegion,
+              dataPartitionMap
+                  .get(storageGroup)
+                  .get(seriesPartitionSlot)
+                  .get(timePartitionSlot)
+                  .get(0)
+                  .getRegionId()
+                  .getType());
           // Including three RegionReplica
           Assert.assertEquals(
               3,
@@ -423,7 +410,7 @@ public class ConfigNodeRPCServerProcessorTest {
                   .get(seriesPartitionSlot)
                   .get(timePartitionSlot)
                   .get(0)
-                  .getEndpointSize());
+                  .getDataNodeLocationsSize());
         }
       }
     }
@@ -441,15 +428,7 @@ public class ConfigNodeRPCServerProcessorTest {
     TDataPartitionResp dataPartitionResp;
 
     // register DataNodes
-    TDataNodeRegisterReq registerReq0 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6667));
-    TDataNodeRegisterReq registerReq1 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6668));
-    TDataNodeRegisterReq registerReq2 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6669));
-    status = processor.registerDataNode(registerReq0).getStatus();
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
-    status = processor.registerDataNode(registerReq1).getStatus();
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
-    status = processor.registerDataNode(registerReq2).getStatus();
-    Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
+    registerDataNodes();
 
     // Prepare partitionSlotsMap
     Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap0 =
@@ -460,7 +439,7 @@ public class ConfigNodeRPCServerProcessorTest {
 
     // set StorageGroups
     for (int i = 0; i < storageGroupNum; i++) {
-      TSetStorageGroupReq setReq = new TSetStorageGroupReq(sg + i);
+      TSetStorageGroupReq setReq = new TSetStorageGroupReq(new TStorageGroupSchema(sg + i));
       status = processor.setStorageGroup(setReq);
       Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
     }
diff --git a/consensus/pom.xml b/consensus/pom.xml
index 857276360e..f773dd846a 100644
--- a/consensus/pom.xml
+++ b/consensus/pom.xml
@@ -46,11 +46,6 @@
             <artifactId>ratis-grpc</artifactId>
             <version>2.2.0</version>
         </dependency>
-        <dependency>
-            <groupId>org.apache.iotdb</groupId>
-            <artifactId>iotdb-thrift</artifactId>
-            <version>${project.version}</version>
-        </dependency>
         <dependency>
             <groupId>org.apache.iotdb</groupId>
             <artifactId>node-commons</artifactId>
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java b/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java
index 8d218ac9f3..d9364ad13e 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.consensus;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.consensus.statemachine.IStateMachine;
 
 import org.slf4j.Logger;
@@ -37,11 +37,12 @@ public class ConsensusFactory {
   private static final Logger logger = LoggerFactory.getLogger(ConsensusFactory.class);
 
   public static Optional<IConsensus> getConsensusImpl(
-      String className, Endpoint endpoint, File storageDir, IStateMachine.Registry registry) {
+      String className, TEndPoint endpoint, File storageDir, IStateMachine.Registry registry) {
     try {
       Class<?> executor = Class.forName(className);
       Constructor<?> executorConstructor =
-          executor.getDeclaredConstructor(Endpoint.class, File.class, IStateMachine.Registry.class);
+          executor.getDeclaredConstructor(
+              TEndPoint.class, File.class, IStateMachine.Registry.class);
       executorConstructor.setAccessible(true);
       return Optional.of(
           (IConsensus) executorConstructor.newInstance(endpoint, storageDir, registry));
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/Peer.java b/consensus/src/main/java/org/apache/iotdb/consensus/common/Peer.java
index 4e7b8f1743..db537e9022 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/Peer.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/common/Peer.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.consensus.common;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 
 import java.util.Objects;
@@ -28,9 +28,9 @@ import java.util.Objects;
 public class Peer {
 
   private final ConsensusGroupId groupId;
-  private final Endpoint endpoint;
+  private final TEndPoint endpoint;
 
-  public Peer(ConsensusGroupId groupId, Endpoint endpoint) {
+  public Peer(ConsensusGroupId groupId, TEndPoint endpoint) {
     this.groupId = groupId;
     this.endpoint = endpoint;
   }
@@ -39,7 +39,7 @@ public class Peer {
     return groupId;
   }
 
-  public Endpoint getEndpoint() {
+  public TEndPoint getEndpoint() {
     return endpoint;
   }
 
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java
index 327ce09639..29b822da8c 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java
@@ -21,7 +21,6 @@ package org.apache.iotdb.consensus.ratis;
 
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.consensus.IConsensus;
 import org.apache.iotdb.consensus.common.DataSet;
@@ -90,7 +89,7 @@ class RatisConsensus implements IConsensus {
 
   private Logger logger = LoggerFactory.getLogger(RatisConsensus.class);
 
-  public RatisConsensus(Endpoint endpoint, File ratisStorageDir, IStateMachine.Registry registry)
+  public RatisConsensus(TEndPoint endpoint, File ratisStorageDir, IStateMachine.Registry registry)
       throws IOException {
 
     this.clientMap = new ConcurrentHashMap<>();
@@ -180,7 +179,7 @@ class RatisConsensus implements IConsensus {
     }
 
     if (suggestedLeader != null) {
-      Endpoint leaderEndPoint = Utils.getEndpoint(suggestedLeader);
+      TEndPoint leaderEndPoint = Utils.getEndpoint(suggestedLeader);
       writeResult.setRedirectNode(new TEndPoint(leaderEndPoint.getIp(), leaderEndPoint.getPort()));
     }
 
@@ -460,7 +459,7 @@ class RatisConsensus implements IConsensus {
     if (client == null) {
       return null;
     }
-    Endpoint leaderEndpoint = Utils.parseFromRatisId(client.getLeaderId().toString());
+    TEndPoint leaderEndpoint = Utils.parseFromRatisId(client.getLeaderId().toString());
     return new Peer(groupId, leaderEndpoint);
   }
 
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RequestMessage.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RequestMessage.java
index 5607c805ac..9dc73d815c 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RequestMessage.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RequestMessage.java
@@ -34,7 +34,7 @@ public class RequestMessage implements Message {
 
   private final IConsensusRequest actualRequest;
   private volatile ByteString serializedContent;
-  private final int DEFAULT_BUFFER_SIZE = 1024 * 10;
+  private static final int DEFAULT_BUFFER_SIZE = 2048 * 10;
 
   public RequestMessage(IConsensusRequest request) {
     this.actualRequest = request;
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java
index d85efae7e2..ff3210717b 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java
@@ -18,8 +18,8 @@
  */
 package org.apache.iotdb.consensus.ratis;
 
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.DataRegionId;
 import org.apache.iotdb.commons.consensus.PartitionRegionId;
@@ -44,7 +44,7 @@ public class Utils {
   private static final String SchemaRegionAbbr = "SR";
   private static final String PartitionRegionAbbr = "PR";
 
-  public static String IPAddress(Endpoint endpoint) {
+  public static String IPAddress(TEndPoint endpoint) {
     return String.format("%s:%d", endpoint.getIp(), endpoint.getPort());
   }
 
@@ -71,17 +71,17 @@ public class Utils {
     return String.format("%s-%d", groupTypeAbbr, consensusGroupId.getId());
   }
 
-  public static String RatisPeerId(Endpoint endpoint) {
+  public static String RatisPeerId(TEndPoint endpoint) {
     return String.format("%s-%d", endpoint.getIp(), endpoint.getPort());
   }
 
-  public static Endpoint parseFromRatisId(String ratisId) {
+  public static TEndPoint parseFromRatisId(String ratisId) {
     String[] items = ratisId.split("-");
-    return new Endpoint(items[0], Integer.parseInt(items[1]));
+    return new TEndPoint(items[0], Integer.parseInt(items[1]));
   }
 
   // priority is used as ordinal of leader election
-  public static RaftPeer toRaftPeer(Endpoint endpoint, int priority) {
+  public static RaftPeer toRaftPeer(TEndPoint endpoint, int priority) {
     return RaftPeer.newBuilder()
         .setId(RatisPeerId(endpoint))
         .setAddress(IPAddress(endpoint))
@@ -93,10 +93,10 @@ public class Utils {
     return toRaftPeer(peer.getEndpoint(), priority);
   }
 
-  public static Endpoint getEndpoint(RaftPeer raftPeer) {
+  public static TEndPoint getEndpoint(RaftPeer raftPeer) {
     String address = raftPeer.getAddress(); // ip:port
     String[] split = address.split(":");
-    return new Endpoint(split[0], Integer.parseInt(split[1]));
+    return new TEndPoint(split[0], Integer.parseInt(split[1]));
   }
 
   /** Given ConsensusGroupId, generate a deterministic RaftGroupId current scheme: */
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/standalone/StandAloneConsensus.java b/consensus/src/main/java/org/apache/iotdb/consensus/standalone/StandAloneConsensus.java
index 7365bbe855..4650db4c16 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/standalone/StandAloneConsensus.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/standalone/StandAloneConsensus.java
@@ -19,8 +19,8 @@
 
 package org.apache.iotdb.consensus.standalone;
 
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.consensus.IConsensus;
 import org.apache.iotdb.consensus.common.DataSet;
@@ -53,13 +53,13 @@ import java.util.concurrent.atomic.AtomicReference;
  */
 class StandAloneConsensus implements IConsensus {
 
-  private final Endpoint thisNode;
+  private final TEndPoint thisNode;
   private final File storageDir;
   private final IStateMachine.Registry registry;
   private final Map<ConsensusGroupId, StandAloneServerImpl> stateMachineMap =
       new ConcurrentHashMap<>();
 
-  public StandAloneConsensus(Endpoint thisNode, File storageDir, Registry registry) {
+  public StandAloneConsensus(TEndPoint thisNode, File storageDir, Registry registry) {
     this.thisNode = thisNode;
     this.storageDir = storageDir;
     this.registry = registry;
diff --git a/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RatisConsensusTest.java b/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RatisConsensusTest.java
index f0e5093f0a..f132dbbf01 100644
--- a/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RatisConsensusTest.java
+++ b/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RatisConsensusTest.java
@@ -18,8 +18,8 @@
  */
 package org.apache.iotdb.consensus.ratis;
 
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.DataRegionId;
 import org.apache.iotdb.consensus.ConsensusFactory;
@@ -137,9 +137,9 @@ public class RatisConsensusTest {
   public void setUp() throws IOException {
     gid = new DataRegionId(1);
     peers = new ArrayList<>();
-    peer0 = new Peer(gid, new Endpoint("127.0.0.1", 6000));
-    peer1 = new Peer(gid, new Endpoint("127.0.0.1", 6001));
-    peer2 = new Peer(gid, new Endpoint("127.0.0.1", 6002));
+    peer0 = new Peer(gid, new TEndPoint("127.0.0.1", 6000));
+    peer1 = new Peer(gid, new TEndPoint("127.0.0.1", 6001));
+    peer2 = new Peer(gid, new TEndPoint("127.0.0.1", 6002));
     peers.add(peer0);
     peers.add(peer1);
     peers.add(peer2);
diff --git a/consensus/src/test/java/org/apache/iotdb/consensus/standalone/StandAloneConsensusTest.java b/consensus/src/test/java/org/apache/iotdb/consensus/standalone/StandAloneConsensusTest.java
index b7eb47d975..1482f21c04 100644
--- a/consensus/src/test/java/org/apache/iotdb/consensus/standalone/StandAloneConsensusTest.java
+++ b/consensus/src/test/java/org/apache/iotdb/consensus/standalone/StandAloneConsensusTest.java
@@ -19,8 +19,8 @@
 
 package org.apache.iotdb.consensus.standalone;
 
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.DataRegionId;
 import org.apache.iotdb.commons.consensus.PartitionRegionId;
@@ -131,7 +131,7 @@ public class StandAloneConsensusTest {
     consensusImpl =
         ConsensusFactory.getConsensusImpl(
                 STANDALONE_CONSENSUS_CLASS_NAME,
-                new Endpoint("localhost", 6667),
+                new TEndPoint("localhost", 6667),
                 new File("./"),
                 gid -> {
                   switch (gid.getType()) {
@@ -161,14 +161,14 @@ public class StandAloneConsensusTest {
     ConsensusGenericResponse response1 =
         consensusImpl.addConsensusGroup(
             dataRegionId,
-            Collections.singletonList(new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667))));
+            Collections.singletonList(new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667))));
     assertTrue(response1.isSuccess());
     assertNull(response1.getException());
 
     ConsensusGenericResponse response2 =
         consensusImpl.addConsensusGroup(
             dataRegionId,
-            Collections.singletonList(new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667))));
+            Collections.singletonList(new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667))));
     assertFalse(response2.isSuccess());
     assertTrue(response2.getException() instanceof ConsensusGroupAlreadyExistException);
 
@@ -176,15 +176,15 @@ public class StandAloneConsensusTest {
         consensusImpl.addConsensusGroup(
             dataRegionId,
             Arrays.asList(
-                new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667)),
-                new Peer(dataRegionId, new Endpoint("0.0.0.1", 6667))));
+                new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667)),
+                new Peer(dataRegionId, new TEndPoint("0.0.0.1", 6667))));
     assertFalse(response3.isSuccess());
     assertTrue(response3.getException() instanceof IllegalPeerNumException);
 
     ConsensusGenericResponse response4 =
         consensusImpl.addConsensusGroup(
             schemaRegionId,
-            Collections.singletonList(new Peer(schemaRegionId, new Endpoint("0.0.0.0", 6667))));
+            Collections.singletonList(new Peer(schemaRegionId, new TEndPoint("0.0.0.0", 6667))));
     assertTrue(response4.isSuccess());
     assertNull(response4.getException());
   }
@@ -198,7 +198,7 @@ public class StandAloneConsensusTest {
     ConsensusGenericResponse response2 =
         consensusImpl.addConsensusGroup(
             dataRegionId,
-            Collections.singletonList(new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667))));
+            Collections.singletonList(new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667))));
     assertTrue(response2.isSuccess());
     assertNull(response2.getException());
 
@@ -210,7 +210,7 @@ public class StandAloneConsensusTest {
   @Test
   public void addPeer() {
     ConsensusGenericResponse response =
-        consensusImpl.addPeer(dataRegionId, new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667)));
+        consensusImpl.addPeer(dataRegionId, new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667)));
     assertFalse(response.isSuccess());
   }
 
@@ -218,7 +218,7 @@ public class StandAloneConsensusTest {
   public void removePeer() {
     ConsensusGenericResponse response =
         consensusImpl.removePeer(
-            dataRegionId, new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667)));
+            dataRegionId, new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667)));
     assertFalse(response.isSuccess());
   }
 
@@ -227,7 +227,7 @@ public class StandAloneConsensusTest {
     ConsensusGenericResponse response =
         consensusImpl.changePeer(
             dataRegionId,
-            Collections.singletonList(new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667))));
+            Collections.singletonList(new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667))));
     assertFalse(response.isSuccess());
   }
 
@@ -235,7 +235,7 @@ public class StandAloneConsensusTest {
   public void transferLeader() {
     ConsensusGenericResponse response =
         consensusImpl.transferLeader(
-            dataRegionId, new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667)));
+            dataRegionId, new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667)));
     assertFalse(response.isSuccess());
   }
 
@@ -250,20 +250,21 @@ public class StandAloneConsensusTest {
     ConsensusGenericResponse response1 =
         consensusImpl.addConsensusGroup(
             dataRegionId,
-            Collections.singletonList(new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667))));
+            Collections.singletonList(new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667))));
     assertTrue(response1.isSuccess());
     assertNull(response1.getException());
 
     ConsensusGenericResponse response2 =
         consensusImpl.addConsensusGroup(
             schemaRegionId,
-            Collections.singletonList(new Peer(schemaRegionId, new Endpoint("0.0.0.0", 6667))));
+            Collections.singletonList(new Peer(schemaRegionId, new TEndPoint("0.0.0.0", 6667))));
     assertTrue(response2.isSuccess());
     assertNull(response2.getException());
 
     ConsensusGenericResponse response3 =
         consensusImpl.addConsensusGroup(
-            configId, Collections.singletonList(new Peer(configId, new Endpoint("0.0.0.0", 6667))));
+            configId,
+            Collections.singletonList(new Peer(configId, new TEndPoint("0.0.0.0", 6667))));
     assertTrue(response3.isSuccess());
     assertNull(response3.getException());
 
diff --git a/node-commons/pom.xml b/node-commons/pom.xml
index 3997cc6c00..999bbfd373 100644
--- a/node-commons/pom.xml
+++ b/node-commons/pom.xml
@@ -51,6 +51,16 @@
             <artifactId>guava</artifactId>
             <version>[${guava.version},)</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift-confignode</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/DataNodeLocation.java b/node-commons/src/main/java/org/apache/iotdb/commons/cluster/DataNodeLocation.java
deleted file mode 100644
index 6223ac43b2..0000000000
--- a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/DataNodeLocation.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.commons.cluster;
-
-import java.nio.ByteBuffer;
-import java.util.Objects;
-
-public class DataNodeLocation {
-
-  private int dataNodeId;
-  private Endpoint endPoint;
-
-  public DataNodeLocation() {}
-
-  public DataNodeLocation(int dataNodeId, Endpoint endPoint) {
-    this.dataNodeId = dataNodeId;
-    this.endPoint = endPoint;
-  }
-
-  public DataNodeLocation(Endpoint endPoint) {
-    this.endPoint = endPoint;
-  }
-
-  public int getDataNodeId() {
-    return dataNodeId;
-  }
-
-  public void setDataNodeId(int dataNodeId) {
-    this.dataNodeId = dataNodeId;
-  }
-
-  public Endpoint getEndPoint() {
-    return endPoint;
-  }
-
-  public void serializeImpl(ByteBuffer buffer) {
-    buffer.putInt(dataNodeId);
-    endPoint.serializeImpl(buffer);
-  }
-
-  public static DataNodeLocation deserializeImpl(ByteBuffer buffer) {
-    return new DataNodeLocation(buffer.getInt(), Endpoint.deserializeImpl(buffer));
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    DataNodeLocation that = (DataNodeLocation) o;
-    return dataNodeId == that.dataNodeId && Objects.equals(endPoint, that.endPoint);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(dataNodeId, endPoint);
-  }
-
-  public String toString() {
-    return String.format("DataNode[%d, %s]", dataNodeId, endPoint);
-  }
-}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/Endpoint.java b/node-commons/src/main/java/org/apache/iotdb/commons/cluster/Endpoint.java
deleted file mode 100644
index b5c15cc85f..0000000000
--- a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/Endpoint.java
+++ /dev/null
@@ -1,92 +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.commons.cluster;
-
-import java.nio.ByteBuffer;
-import java.util.Objects;
-
-// TODO Use a mature IDL framework such as Protobuf to manage this structure
-public class Endpoint {
-
-  private String ip;
-  private int port;
-
-  public Endpoint() {}
-
-  public Endpoint(String ip, int port) {
-    this.ip = ip;
-    this.port = port;
-  }
-
-  public String getIp() {
-    return ip;
-  }
-
-  public Endpoint setIp(String ip) {
-    this.ip = ip;
-    return this;
-  }
-
-  public int getPort() {
-    return port;
-  }
-
-  public Endpoint setPort(int port) {
-    this.port = port;
-    return this;
-  }
-
-  public void serializeImpl(ByteBuffer buffer) {
-    byte[] bytes = ip.getBytes();
-    buffer.putInt(bytes.length);
-    buffer.put(bytes);
-
-    buffer.putInt(port);
-  }
-
-  public static Endpoint deserializeImpl(ByteBuffer buffer) {
-    int length = buffer.getInt();
-    byte[] bytes = new byte[length];
-    buffer.get(bytes, 0, length);
-    return new Endpoint(new String(bytes, 0, length), buffer.getInt());
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    Endpoint endpoint = (Endpoint) o;
-    return port == endpoint.port && Objects.equals(ip, endpoint.ip);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(ip, port);
-  }
-
-  @Override
-  public String toString() {
-    return String.format("%s:%d", ip, port);
-  }
-}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConsensusGroupId.java b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConsensusGroupId.java
index ba38960173..aee4bcee55 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConsensusGroupId.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConsensusGroupId.java
@@ -19,37 +19,21 @@
 
 package org.apache.iotdb.commons.consensus;
 
-import java.nio.ByteBuffer;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 
 public interface ConsensusGroupId {
 
-  // contains specific id and type
-  void serializeImpl(ByteBuffer buffer);
-
-  // only deserialize specific id
-  void deserializeImpl(ByteBuffer buffer);
-
   // return specific id
   int getId();
 
   void setId(int id);
 
   // return specific type
-  GroupType getType();
+  TConsensusGroupType getType();
 
   class Factory {
-    public static ConsensusGroupId create(ByteBuffer buffer) {
-      int index = buffer.get();
-      if (index >= GroupType.values().length) {
-        throw new IllegalArgumentException("invalid ConsensusGroup type. Ordinal is: " + index);
-      }
-      GroupType type = GroupType.values()[index];
-      ConsensusGroupId groupId = createEmpty(type);
-      groupId.deserializeImpl(buffer);
-      return groupId;
-    }
-
-    public static ConsensusGroupId createEmpty(GroupType type) {
+    public static ConsensusGroupId createEmpty(TConsensusGroupType type) {
       ConsensusGroupId groupId;
       switch (type) {
         case DataRegion:
@@ -67,10 +51,24 @@ public interface ConsensusGroupId {
       return groupId;
     }
 
-    public static ConsensusGroupId create(int id, GroupType type) {
-      ConsensusGroupId groupId = createEmpty(type);
-      groupId.setId(id);
+    public static ConsensusGroupId convertFromTConsensusGroupId(
+        TConsensusGroupId tConsensusGroupId) {
+      ConsensusGroupId groupId = createEmpty(tConsensusGroupId.getType());
+      groupId.setId(tConsensusGroupId.getId());
       return groupId;
     }
+
+    public static TConsensusGroupId convertToTConsensusGroupId(ConsensusGroupId consensusGroupId) {
+      TConsensusGroupId result = new TConsensusGroupId();
+      if (consensusGroupId instanceof SchemaRegionId) {
+        result.setType(TConsensusGroupType.SchemaRegion);
+      } else if (consensusGroupId instanceof DataRegionId) {
+        result.setType(TConsensusGroupType.DataRegion);
+      } else if (consensusGroupId instanceof PartitionRegionId) {
+        result.setType(TConsensusGroupType.PartitionRegion);
+      }
+      result.setId(consensusGroupId.getId());
+      return result;
+    }
   }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/DataRegionId.java b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/DataRegionId.java
index b0c5d1ec46..81fed3fe62 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/DataRegionId.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/DataRegionId.java
@@ -19,7 +19,8 @@
 
 package org.apache.iotdb.commons.consensus;
 
-import java.nio.ByteBuffer;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+
 import java.util.Objects;
 
 public class DataRegionId implements ConsensusGroupId {
@@ -32,18 +33,6 @@ public class DataRegionId implements ConsensusGroupId {
     this.id = id;
   }
 
-  @Override
-  public void serializeImpl(ByteBuffer buffer) {
-    buffer.put((byte) GroupType.DataRegion.ordinal());
-    buffer.putInt(id);
-  }
-
-  @Override
-  public void deserializeImpl(ByteBuffer buffer) {
-    // TODO: (xingtanzjr) should we add validation for the ordinal ?
-    id = buffer.getInt();
-  }
-
   @Override
   public int getId() {
     return id;
@@ -55,8 +44,8 @@ public class DataRegionId implements ConsensusGroupId {
   }
 
   @Override
-  public GroupType getType() {
-    return GroupType.DataRegion;
+  public TConsensusGroupType getType() {
+    return TConsensusGroupType.DataRegion;
   }
 
   @Override
@@ -73,7 +62,7 @@ public class DataRegionId implements ConsensusGroupId {
 
   @Override
   public int hashCode() {
-    return Objects.hash(id, GroupType.DataRegion);
+    return Objects.hash(id, TConsensusGroupType.DataRegion);
   }
 
   public String toString() {
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/GroupType.java b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/GroupType.java
deleted file mode 100644
index 65b5163a0d..0000000000
--- a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/GroupType.java
+++ /dev/null
@@ -1,27 +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.commons.consensus;
-
-// TODO Use a mature IDL framework such as Protobuf to manage this structure
-public enum GroupType {
-  PartitionRegion,
-  DataRegion,
-  SchemaRegion
-}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/PartitionRegionId.java b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/PartitionRegionId.java
index b9f64786c5..6f317547ed 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/PartitionRegionId.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/PartitionRegionId.java
@@ -19,7 +19,8 @@
 
 package org.apache.iotdb.commons.consensus;
 
-import java.nio.ByteBuffer;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+
 import java.util.Objects;
 
 public class PartitionRegionId implements ConsensusGroupId {
@@ -32,18 +33,6 @@ public class PartitionRegionId implements ConsensusGroupId {
     this.id = id;
   }
 
-  @Override
-  public void serializeImpl(ByteBuffer buffer) {
-    buffer.put((byte) GroupType.PartitionRegion.ordinal());
-    buffer.putInt(id);
-  }
-
-  @Override
-  public void deserializeImpl(ByteBuffer buffer) {
-    // TODO: (xingtanzjr) should we add validation for the ordinal ?
-    id = buffer.getInt();
-  }
-
   @Override
   public int getId() {
     return id;
@@ -55,8 +44,8 @@ public class PartitionRegionId implements ConsensusGroupId {
   }
 
   @Override
-  public GroupType getType() {
-    return GroupType.PartitionRegion;
+  public TConsensusGroupType getType() {
+    return TConsensusGroupType.PartitionRegion;
   }
 
   @Override
@@ -73,7 +62,7 @@ public class PartitionRegionId implements ConsensusGroupId {
 
   @Override
   public int hashCode() {
-    return Objects.hash(id, GroupType.PartitionRegion);
+    return Objects.hash(id, TConsensusGroupType.PartitionRegion);
   }
 
   public String toString() {
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/SchemaRegionId.java b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/SchemaRegionId.java
index 9649653c7f..61c4ca80c5 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/SchemaRegionId.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/SchemaRegionId.java
@@ -19,7 +19,8 @@
 
 package org.apache.iotdb.commons.consensus;
 
-import java.nio.ByteBuffer;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+
 import java.util.Objects;
 
 public class SchemaRegionId implements ConsensusGroupId {
@@ -32,18 +33,6 @@ public class SchemaRegionId implements ConsensusGroupId {
     this.id = id;
   }
 
-  @Override
-  public void serializeImpl(ByteBuffer buffer) {
-    buffer.put((byte) GroupType.SchemaRegion.ordinal());
-    buffer.putInt(id);
-  }
-
-  @Override
-  public void deserializeImpl(ByteBuffer buffer) {
-    // TODO: (xingtanzjr) should we add validation for the ordinal ?
-    id = buffer.getInt();
-  }
-
   @Override
   public int getId() {
     return id;
@@ -55,8 +44,8 @@ public class SchemaRegionId implements ConsensusGroupId {
   }
 
   @Override
-  public GroupType getType() {
-    return GroupType.SchemaRegion;
+  public TConsensusGroupType getType() {
+    return TConsensusGroupType.SchemaRegion;
   }
 
   @Override
@@ -73,7 +62,7 @@ public class SchemaRegionId implements ConsensusGroupId {
 
   @Override
   public int hashCode() {
-    return Objects.hash(id, GroupType.SchemaRegion);
+    return Objects.hash(id, TConsensusGroupType.SchemaRegion);
   }
 
   public String toString() {
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java
index e1d24db22d..5156ba47af 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java
@@ -18,6 +18,10 @@
  */
 package org.apache.iotdb.commons.partition;
 
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -28,8 +32,8 @@ import java.util.stream.Collectors;
 
 public class DataPartition extends Partition {
 
-  // Map<StorageGroup, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionMessage>>>>
-  private Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+  // Map<StorageGroup, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionMessage>>>>
+  private Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
       dataPartitionMap;
 
   public DataPartition(String seriesSlotExecutorName, int seriesPartitionSlotNum) {
@@ -37,7 +41,7 @@ public class DataPartition extends Partition {
   }
 
   public DataPartition(
-      Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+      Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
           dataPartitionMap,
       String seriesSlotExecutorName,
       int seriesPartitionSlotNum) {
@@ -45,34 +49,34 @@ public class DataPartition extends Partition {
     this.dataPartitionMap = dataPartitionMap;
   }
 
-  public Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+  public Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
       getDataPartitionMap() {
     return dataPartitionMap;
   }
 
   public void setDataPartitionMap(
-      Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+      Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
           dataPartitionMap) {
     this.dataPartitionMap = dataPartitionMap;
   }
 
-  public List<RegionReplicaSet> getDataRegionReplicaSet(
-      String deviceName, List<TimePartitionSlot> timePartitionSlotList) {
+  public List<TRegionReplicaSet> getDataRegionReplicaSet(
+      String deviceName, List<TTimePartitionSlot> timePartitionSlotList) {
     String storageGroup = getStorageGroupByDevice(deviceName);
-    SeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
+    TSeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
     // TODO: (xingtanzjr) the timePartitionIdList is ignored
     return dataPartitionMap.get(storageGroup).get(seriesPartitionSlot).values().stream()
         .flatMap(Collection::stream)
         .collect(Collectors.toList());
   }
 
-  public List<RegionReplicaSet> getDataRegionReplicaSetForWriting(
-      String deviceName, List<TimePartitionSlot> timePartitionSlotList) {
+  public List<TRegionReplicaSet> getDataRegionReplicaSetForWriting(
+      String deviceName, List<TTimePartitionSlot> timePartitionSlotList) {
     // A list of data region replica sets will store data in a same time partition.
     // We will insert data to the last set in the list.
     // TODO return the latest dataRegionReplicaSet for each time partition
     String storageGroup = getStorageGroupByDevice(deviceName);
-    SeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
+    TSeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
     // IMPORTANT TODO: (xingtanzjr) need to handle the situation for write operation that there are
     // more than 1 Regions for one timeSlot
     return dataPartitionMap.get(storageGroup).get(seriesPartitionSlot).entrySet().stream()
@@ -81,14 +85,14 @@ public class DataPartition extends Partition {
         .collect(Collectors.toList());
   }
 
-  public RegionReplicaSet getDataRegionReplicaSetForWriting(
-      String deviceName, TimePartitionSlot timePartitionSlot) {
+  public TRegionReplicaSet getDataRegionReplicaSetForWriting(
+      String deviceName, TTimePartitionSlot timePartitionSlot) {
     // A list of data region replica sets will store data in a same time partition.
     // We will insert data to the last set in the list.
     // TODO return the latest dataRegionReplicaSet for each time partition
     String storageGroup = getStorageGroupByDevice(deviceName);
-    SeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
-    List<RegionReplicaSet> regions =
+    TSeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
+    List<TRegionReplicaSet> regions =
         dataPartitionMap.get(storageGroup).get(seriesPartitionSlot).entrySet().stream()
             .filter(entry -> entry.getKey().equals(timePartitionSlot))
             .flatMap(entry -> entry.getValue().stream())
@@ -119,31 +123,30 @@ public class DataPartition extends Partition {
    *     Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
    */
   public DataPartition getDataPartition(
-      Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> partitionSlotsMap,
+      Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap,
       String seriesSlotExecutorName,
       int seriesPartitionSlotNum) {
-    Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>> result =
-        new HashMap<>();
+    Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
+        result = new HashMap<>();
 
     for (String storageGroupName : partitionSlotsMap.keySet()) {
       // Compare StorageGroupName
       if (dataPartitionMap.containsKey(storageGroupName)) {
-        Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>
+        Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>
             seriesTimePartitionSlotMap = dataPartitionMap.get(storageGroupName);
-        for (SeriesPartitionSlot seriesPartitionSlot :
+        for (TSeriesPartitionSlot seriesPartitionSlot :
             partitionSlotsMap.get(storageGroupName).keySet()) {
           // Compare SeriesPartitionSlot
           if (seriesTimePartitionSlotMap.containsKey(seriesPartitionSlot)) {
-            Map<TimePartitionSlot, List<RegionReplicaSet>> timePartitionSlotMap =
+            Map<TTimePartitionSlot, List<TRegionReplicaSet>> timePartitionSlotMap =
                 seriesTimePartitionSlotMap.get(seriesPartitionSlot);
-            // TODO: (xingtanzjr) optimize if timeSlotPartition is empty
             if (partitionSlotsMap.get(storageGroupName).get(seriesPartitionSlot).size() == 0) {
               result
                   .computeIfAbsent(storageGroupName, key -> new HashMap<>())
                   .computeIfAbsent(seriesPartitionSlot, key -> new HashMap<>())
                   .putAll(new HashMap<>(timePartitionSlotMap));
             } else {
-              for (TimePartitionSlot timePartitionSlot :
+              for (TTimePartitionSlot timePartitionSlot :
                   partitionSlotsMap.get(storageGroupName).get(seriesPartitionSlot)) {
                 // Compare TimePartitionSlot
                 if (timePartitionSlotMap.containsKey(timePartitionSlot)) {
@@ -172,23 +175,23 @@ public class DataPartition extends Partition {
    * @return Map<StorageGroupName, Map < SeriesPartitionSlot, List < TimePartitionSlot>>>,
    *     unassigned PartitionSlots
    */
-  public Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>>
+  public Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>>
       filterNoAssignedDataPartitionSlots(
-          Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> partitionSlotsMap) {
-    Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> result = new HashMap<>();
+          Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap) {
+    Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> result = new HashMap<>();
 
     for (String storageGroupName : partitionSlotsMap.keySet()) {
       // Compare StorageGroupName
       if (dataPartitionMap.containsKey(storageGroupName)) {
-        Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>
+        Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>
             seriesTimePartitionSlotMap = dataPartitionMap.get(storageGroupName);
-        for (SeriesPartitionSlot seriesPartitionSlot :
+        for (TSeriesPartitionSlot seriesPartitionSlot :
             partitionSlotsMap.get(storageGroupName).keySet()) {
           // Compare SeriesPartitionSlot
           if (seriesTimePartitionSlotMap.containsKey(seriesPartitionSlot)) {
-            Map<TimePartitionSlot, List<RegionReplicaSet>> timePartitionSlotMap =
+            Map<TTimePartitionSlot, List<TRegionReplicaSet>> timePartitionSlotMap =
                 seriesTimePartitionSlotMap.get(seriesPartitionSlot);
-            for (TimePartitionSlot timePartitionSlot :
+            for (TTimePartitionSlot timePartitionSlot :
                 partitionSlotsMap.get(storageGroupName).get(seriesPartitionSlot)) {
               // Compare TimePartitionSlot
               if (!timePartitionSlotMap.containsKey(timePartitionSlot)) {
@@ -220,9 +223,9 @@ public class DataPartition extends Partition {
   /** Create a DataPartition by ConfigNode */
   public void createDataPartition(
       String storageGroup,
-      SeriesPartitionSlot seriesPartitionSlot,
-      TimePartitionSlot timePartitionSlot,
-      RegionReplicaSet regionReplicaSet) {
+      TSeriesPartitionSlot seriesPartitionSlot,
+      TTimePartitionSlot timePartitionSlot,
+      TRegionReplicaSet regionReplicaSet) {
     dataPartitionMap
         .computeIfAbsent(storageGroup, key -> new HashMap<>())
         .computeIfAbsent(seriesPartitionSlot, key -> new HashMap<>())
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionQueryParam.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionQueryParam.java
index 23b61902a9..25ddb433e5 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionQueryParam.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionQueryParam.java
@@ -18,13 +18,15 @@
  */
 package org.apache.iotdb.commons.partition;
 
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+
 import java.util.ArrayList;
 import java.util.List;
 
 public class DataPartitionQueryParam {
 
   private String devicePath;
-  private List<TimePartitionSlot> timePartitionSlotList = new ArrayList<>();
+  private List<TTimePartitionSlot> timePartitionSlotList = new ArrayList<>();
 
   public String getDevicePath() {
     return devicePath;
@@ -34,11 +36,11 @@ public class DataPartitionQueryParam {
     this.devicePath = devicePath;
   }
 
-  public List<TimePartitionSlot> getTimePartitionSlotList() {
+  public List<TTimePartitionSlot> getTimePartitionSlotList() {
     return timePartitionSlotList;
   }
 
-  public void setTimePartitionSlotList(List<TimePartitionSlot> timePartitionSlotList) {
+  public void setTimePartitionSlotList(List<TTimePartitionSlot> timePartitionSlotList) {
     this.timePartitionSlotList = timePartitionSlotList;
   }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/Partition.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/Partition.java
index e0c6e53bd1..839c330f90 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/Partition.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/Partition.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.commons.partition;
 
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
 import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
 
 public abstract class Partition {
@@ -34,7 +35,7 @@ public abstract class Partition {
             seriesSlotExecutorName, seriesPartitionSlotNum);
   }
 
-  protected SeriesPartitionSlot calculateDeviceGroupId(String deviceName) {
+  protected TSeriesPartitionSlot calculateDeviceGroupId(String deviceName) {
     return executor.getSeriesPartitionSlot(deviceName);
   }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/RegionReplicaSet.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/RegionReplicaSet.java
deleted file mode 100644
index 27cf396fce..0000000000
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/RegionReplicaSet.java
+++ /dev/null
@@ -1,141 +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.commons.partition;
-
-import org.apache.iotdb.common.rpc.thrift.TEndPoint;
-import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.consensus.ConsensusGroupId;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-import java.util.stream.Collectors;
-
-public class RegionReplicaSet {
-  private ConsensusGroupId consensusGroupId;
-  private List<DataNodeLocation> dataNodeList;
-
-  public RegionReplicaSet() {}
-
-  public RegionReplicaSet(ConsensusGroupId consensusGroupId, List<DataNodeLocation> dataNodeList) {
-    this.consensusGroupId = consensusGroupId;
-    this.dataNodeList = dataNodeList;
-  }
-
-  public RegionReplicaSet(TRegionReplicaSet respRegionReplicaSet) {
-    this.consensusGroupId = ConsensusGroupId.Factory.create(respRegionReplicaSet.regionId);
-    this.dataNodeList =
-        respRegionReplicaSet.getEndpoint().stream()
-            .map(
-                respEndpoint ->
-                    new DataNodeLocation(
-                        new Endpoint(respEndpoint.getIp(), respEndpoint.getPort())))
-            .collect(Collectors.toList());
-  }
-
-  public List<DataNodeLocation> getDataNodeList() {
-    return dataNodeList;
-  }
-
-  public void setDataNodeList(List<DataNodeLocation> dataNodeList) {
-    this.dataNodeList = dataNodeList;
-  }
-
-  public ConsensusGroupId getConsensusGroupId() {
-    return consensusGroupId;
-  }
-
-  public void setConsensusGroupId(ConsensusGroupId consensusGroupId) {
-    this.consensusGroupId = consensusGroupId;
-  }
-
-  /** Convert RegionReplicaSet to TRegionReplicaSet */
-  public TRegionReplicaSet convertToRPCTRegionReplicaSet() {
-    TRegionReplicaSet tRegionReplicaSet = new TRegionReplicaSet();
-
-    // Convert ConsensusGroupId
-    ByteBuffer buffer = ByteBuffer.allocate(Byte.BYTES + Integer.BYTES);
-    consensusGroupId.serializeImpl(buffer);
-    buffer.flip();
-    tRegionReplicaSet.setRegionId(buffer);
-
-    // Convert EndPoints
-    List<TEndPoint> endPointList = new ArrayList<>();
-    dataNodeList.forEach(
-        dataNodeLocation ->
-            endPointList.add(
-                new TEndPoint(
-                    dataNodeLocation.getEndPoint().getIp(),
-                    dataNodeLocation.getEndPoint().getPort())));
-    tRegionReplicaSet.setEndpoint(endPointList);
-
-    return tRegionReplicaSet;
-  }
-
-  @Override
-  public String toString() {
-    return String.format("RegionReplicaSet[%s]: %s", consensusGroupId, dataNodeList);
-  }
-
-  public void serializeImpl(ByteBuffer buffer) {
-    consensusGroupId.serializeImpl(buffer);
-    buffer.putInt(dataNodeList.size());
-    dataNodeList.forEach(
-        dataNode -> {
-          dataNode.serializeImpl(buffer);
-        });
-  }
-
-  public static RegionReplicaSet deserializeImpl(ByteBuffer buffer) {
-    ConsensusGroupId consensusGroupId = ConsensusGroupId.Factory.create(buffer);
-
-    int size = buffer.getInt();
-    // We should always make dataNodeList as a new Object when deserialization
-    List<DataNodeLocation> dataNodeList = new ArrayList<>();
-    for (int i = 0; i < size; i++) {
-      dataNodeList.add(DataNodeLocation.deserializeImpl(buffer));
-    }
-    return new RegionReplicaSet(consensusGroupId, dataNodeList);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    RegionReplicaSet that = (RegionReplicaSet) o;
-    return Objects.equals(consensusGroupId, that.consensusGroupId)
-        && Objects.equals(dataNodeList, that.dataNodeList);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(consensusGroupId, dataNodeList);
-  }
-
-  public boolean isEmpty() {
-    return dataNodeList == null || dataNodeList.isEmpty();
-  }
-}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/SchemaPartition.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/SchemaPartition.java
index b3f8c61986..a93048ff83 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/SchemaPartition.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/SchemaPartition.java
@@ -18,6 +18,9 @@
  */
 package org.apache.iotdb.commons.partition;
 
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -25,36 +28,36 @@ import java.util.Map;
 
 public class SchemaPartition extends Partition {
 
-  // Map<StorageGroup, Map<SeriesPartitionSlot, SchemaRegionPlaceInfo>>
-  private Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> schemaPartitionMap;
+  // Map<StorageGroup, Map<TSeriesPartitionSlot, TSchemaRegionPlaceInfo>>
+  private Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap;
 
   public SchemaPartition(String seriesSlotExecutorName, int seriesPartitionSlotNum) {
     super(seriesSlotExecutorName, seriesPartitionSlotNum);
   }
 
   public SchemaPartition(
-      Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> schemaPartitionMap,
+      Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap,
       String seriesSlotExecutorName,
       int seriesPartitionSlotNum) {
     this(seriesSlotExecutorName, seriesPartitionSlotNum);
     this.schemaPartitionMap = schemaPartitionMap;
   }
 
-  public Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> getSchemaPartitionMap() {
+  public Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> getSchemaPartitionMap() {
     return schemaPartitionMap;
   }
 
   public void setSchemaPartitionMap(
-      Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> schemaPartitionMap) {
+      Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap) {
     this.schemaPartitionMap = schemaPartitionMap;
   }
 
-  public RegionReplicaSet getSchemaRegionReplicaSet(String deviceName) {
+  public TRegionReplicaSet getSchemaRegionReplicaSet(String deviceName) {
     // A list of data region replica sets will store data in a same time partition.
     // We will insert data to the last set in the list.
     // TODO return the latest dataRegionReplicaSet for each time partition
     String storageGroup = getStorageGroupByDevice(deviceName);
-    SeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
+    TSeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
     return schemaPartitionMap.get(storageGroup).get(seriesPartitionSlot);
   }
 
@@ -78,13 +81,13 @@ public class SchemaPartition extends Partition {
    *     RegionReplicaSet>>
    */
   public SchemaPartition getSchemaPartition(
-      Map<String, List<SeriesPartitionSlot>> partitionSlotsMap) {
+      Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap) {
     if (partitionSlotsMap.isEmpty()) {
       // Return all SchemaPartitions when the partitionSlotsMap is empty
       return new SchemaPartition(
           new HashMap<>(schemaPartitionMap), seriesSlotExecutorName, seriesPartitionSlotNum);
     } else {
-      Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> result = new HashMap<>();
+      Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> result = new HashMap<>();
 
       partitionSlotsMap.forEach(
           (storageGroup, seriesPartitionSlots) -> {
@@ -119,9 +122,9 @@ public class SchemaPartition extends Partition {
    * @param partitionSlotsMap Map<StorageGroupName, List<SeriesPartitionSlot>>
    * @return Map<String, List<SeriesPartitionSlot>>, unassigned PartitionSlots
    */
-  public Map<String, List<SeriesPartitionSlot>> filterNoAssignedSchemaPartitionSlot(
-      Map<String, List<SeriesPartitionSlot>> partitionSlotsMap) {
-    Map<String, List<SeriesPartitionSlot>> result = new HashMap<>();
+  public Map<String, List<TSeriesPartitionSlot>> filterNoAssignedSchemaPartitionSlot(
+      Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap) {
+    Map<String, List<TSeriesPartitionSlot>> result = new HashMap<>();
 
     partitionSlotsMap.forEach(
         (storageGroup, seriesPartitionSlots) -> {
@@ -147,8 +150,8 @@ public class SchemaPartition extends Partition {
   /** Create a SchemaPartition by ConfigNode */
   public void createSchemaPartition(
       String storageGroup,
-      SeriesPartitionSlot seriesPartitionSlot,
-      RegionReplicaSet regionReplicaSet) {
+      TSeriesPartitionSlot seriesPartitionSlot,
+      TRegionReplicaSet regionReplicaSet) {
     schemaPartitionMap
         .computeIfAbsent(storageGroup, key -> new HashMap<>())
         .put(seriesPartitionSlot, regionReplicaSet);
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionSlot.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionSlot.java
deleted file mode 100644
index 553aeff163..0000000000
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionSlot.java
+++ /dev/null
@@ -1,59 +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.commons.partition;
-
-import java.nio.ByteBuffer;
-
-public class SeriesPartitionSlot {
-  private int slotId;
-
-  public SeriesPartitionSlot() {
-    // Empty constructor
-  }
-
-  public SeriesPartitionSlot(int slotId) {
-    this.slotId = slotId;
-  }
-
-  public int getSlotId() {
-    return slotId;
-  }
-
-  public void setSlotId(int slotId) {
-    this.slotId = slotId;
-  }
-
-  @Override
-  public int hashCode() {
-    return new Integer(slotId).hashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    return obj instanceof SeriesPartitionSlot && this.slotId == ((SeriesPartitionSlot) obj).slotId;
-  }
-
-  public void serializeImpl(ByteBuffer buffer) {
-    buffer.putInt(slotId);
-  }
-
-  public void deserializeImpl(ByteBuffer buffer) {
-    slotId = buffer.getInt();
-  }
-}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/TimePartitionSlot.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/TimePartitionSlot.java
deleted file mode 100644
index 69a3416fcf..0000000000
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/TimePartitionSlot.java
+++ /dev/null
@@ -1,64 +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.commons.partition;
-
-import java.nio.ByteBuffer;
-import java.util.Objects;
-
-public class TimePartitionSlot {
-
-  private long startTime;
-
-  public TimePartitionSlot() {
-    // empty constructor
-  }
-
-  public TimePartitionSlot(long startTime) {
-    this.startTime = startTime;
-  }
-
-  public long getStartTime() {
-    return startTime;
-  }
-
-  public void setStartTime(long startTime) {
-    this.startTime = startTime;
-  }
-
-  public void serializeImpl(ByteBuffer buffer) {
-    buffer.putLong(startTime);
-  }
-
-  public void deserializeImpl(ByteBuffer buffer) {
-    startTime = buffer.getLong();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-    TimePartitionSlot that = (TimePartitionSlot) o;
-    return startTime == that.startTime;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(startTime);
-  }
-}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/SeriesPartitionExecutor.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/SeriesPartitionExecutor.java
index 8631951342..7a9de34825 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/SeriesPartitionExecutor.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/SeriesPartitionExecutor.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.commons.partition.executor;
 
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
 
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
@@ -32,7 +32,7 @@ public abstract class SeriesPartitionExecutor {
     this.seriesPartitionSlotNum = seriesPartitionSlotNum;
   }
 
-  public abstract SeriesPartitionSlot getSeriesPartitionSlot(String device);
+  public abstract TSeriesPartitionSlot getSeriesPartitionSlot(String device);
 
   public static SeriesPartitionExecutor getSeriesPartitionExecutor(
       String executorName, int seriesPartitionSlotNum) {
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/APHashExecutor.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/APHashExecutor.java
index 8da263e4ae..cad082bfe2 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/APHashExecutor.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/APHashExecutor.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.commons.partition.executor.hash;
 
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
 import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
 
 public class APHashExecutor extends SeriesPartitionExecutor {
@@ -28,7 +28,7 @@ public class APHashExecutor extends SeriesPartitionExecutor {
   }
 
   @Override
-  public SeriesPartitionSlot getSeriesPartitionSlot(String device) {
+  public TSeriesPartitionSlot getSeriesPartitionSlot(String device) {
     int hash = 0;
 
     for (int i = 0; i < device.length(); i++) {
@@ -40,6 +40,6 @@ public class APHashExecutor extends SeriesPartitionExecutor {
     }
     hash &= Integer.MAX_VALUE;
 
-    return new SeriesPartitionSlot(hash % seriesPartitionSlotNum);
+    return new TSeriesPartitionSlot(hash % seriesPartitionSlotNum);
   }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/BKDRHashExecutor.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/BKDRHashExecutor.java
index a238afb416..a1f1e05503 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/BKDRHashExecutor.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/BKDRHashExecutor.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.commons.partition.executor.hash;
 
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
 import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
 
 public class BKDRHashExecutor extends SeriesPartitionExecutor {
@@ -30,7 +30,7 @@ public class BKDRHashExecutor extends SeriesPartitionExecutor {
   }
 
   @Override
-  public SeriesPartitionSlot getSeriesPartitionSlot(String device) {
+  public TSeriesPartitionSlot getSeriesPartitionSlot(String device) {
     int hash = 0;
 
     for (int i = 0; i < device.length(); i++) {
@@ -38,6 +38,6 @@ public class BKDRHashExecutor extends SeriesPartitionExecutor {
     }
     hash &= Integer.MAX_VALUE;
 
-    return new SeriesPartitionSlot(hash % seriesPartitionSlotNum);
+    return new TSeriesPartitionSlot(hash % seriesPartitionSlotNum);
   }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/JSHashExecutor.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/JSHashExecutor.java
index 13f9468154..19914eed2d 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/JSHashExecutor.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/JSHashExecutor.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.commons.partition.executor.hash;
 
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
 import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
 
 public class JSHashExecutor extends SeriesPartitionExecutor {
@@ -30,7 +30,7 @@ public class JSHashExecutor extends SeriesPartitionExecutor {
   }
 
   @Override
-  public SeriesPartitionSlot getSeriesPartitionSlot(String device) {
+  public TSeriesPartitionSlot getSeriesPartitionSlot(String device) {
     int hash = base;
 
     for (int i = 0; i < device.length(); i++) {
@@ -38,6 +38,6 @@ public class JSHashExecutor extends SeriesPartitionExecutor {
     }
     hash &= Integer.MAX_VALUE;
 
-    return new SeriesPartitionSlot(hash % seriesPartitionSlotNum);
+    return new TSeriesPartitionSlot(hash % seriesPartitionSlotNum);
   }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/SDBMHashExecutor.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/SDBMHashExecutor.java
index 0246d4f8de..105c256e3d 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/SDBMHashExecutor.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/SDBMHashExecutor.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.commons.partition.executor.hash;
 
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
 import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
 
 public class SDBMHashExecutor extends SeriesPartitionExecutor {
@@ -28,7 +28,7 @@ public class SDBMHashExecutor extends SeriesPartitionExecutor {
   }
 
   @Override
-  public SeriesPartitionSlot getSeriesPartitionSlot(String device) {
+  public TSeriesPartitionSlot getSeriesPartitionSlot(String device) {
     int hash = 0;
 
     for (int i = 0; i < device.length(); i++) {
@@ -36,6 +36,6 @@ public class SDBMHashExecutor extends SeriesPartitionExecutor {
     }
     hash &= Integer.MAX_VALUE;
 
-    return new SeriesPartitionSlot(hash % seriesPartitionSlotNum);
+    return new TSeriesPartitionSlot(hash % seriesPartitionSlotNum);
   }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/util/SerializeDeserializeUtil.java b/node-commons/src/main/java/org/apache/iotdb/commons/utils/BasicStructureSerDeUtil.java
similarity index 98%
rename from confignode/src/main/java/org/apache/iotdb/confignode/util/SerializeDeserializeUtil.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/utils/BasicStructureSerDeUtil.java
index 7461827520..215829d396 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/util/SerializeDeserializeUtil.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/utils/BasicStructureSerDeUtil.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.confignode.util;
+package org.apache.iotdb.commons.utils;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -25,10 +25,10 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-public class SerializeDeserializeUtil {
+public class BasicStructureSerDeUtil {
   public static final int INT_LEN = 4;
 
-  private SerializeDeserializeUtil() {}
+  private BasicStructureSerDeUtil() {}
 
   /** read string from byteBuffer. */
   public static String readString(ByteBuffer buffer) {
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/utils/CommonUtils.java b/node-commons/src/main/java/org/apache/iotdb/commons/utils/CommonUtils.java
index b16d7692c8..f153933f43 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/utils/CommonUtils.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/utils/CommonUtils.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.commons.utils;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.commons.exception.BadNodeUrlException;
 
 import org.slf4j.Logger;
@@ -31,17 +31,17 @@ import java.util.List;
 public class CommonUtils {
   private static final Logger logger = LoggerFactory.getLogger(CommonUtils.class);
 
-  public static Endpoint parseNodeUrl(String nodeUrl) throws BadNodeUrlException {
+  public static TEndPoint parseNodeUrl(String nodeUrl) throws BadNodeUrlException {
     String[] split = nodeUrl.split(":");
     if (split.length != 2) {
       logger.warn("Bad node url: {}", nodeUrl);
       throw new BadNodeUrlException(String.format("Bad node url: %s", nodeUrl));
     }
     String ip = split[0];
-    Endpoint result;
+    TEndPoint result;
     try {
       int port = Integer.parseInt(split[1]);
-      result = new Endpoint(ip, port);
+      result = new TEndPoint(ip, port);
     } catch (NumberFormatException e) {
       logger.warn("Bad node url: {}", nodeUrl);
       throw new BadNodeUrlException(String.format("Bad node url: %s", nodeUrl));
@@ -49,8 +49,8 @@ public class CommonUtils {
     return result;
   }
 
-  public static List<Endpoint> parseNodeUrls(List<String> nodeUrls) throws BadNodeUrlException {
-    List<Endpoint> result = new ArrayList<>();
+  public static List<TEndPoint> parseNodeUrls(List<String> nodeUrls) throws BadNodeUrlException {
+    List<TEndPoint> result = new ArrayList<>();
     for (String url : nodeUrls) {
       result.add(parseNodeUrl(url));
     }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtils.java b/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtils.java
new file mode 100644
index 0000000000..1b40fa84a6
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtils.java
@@ -0,0 +1,144 @@
+/*
+ * 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.commons.utils;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+// TODO: Serialize and deserialize by thrift structure
+/** Utils for serialize and deserialize all the data struct defined by thrift-commons */
+public class ThriftCommonsSerDeUtils {
+
+  private ThriftCommonsSerDeUtils() {
+    // Empty constructor
+  }
+
+  public static void writeTEndPoint(TEndPoint endPoint, ByteBuffer buffer) {
+    BasicStructureSerDeUtil.write(endPoint.getIp(), buffer);
+    buffer.putInt(endPoint.getPort());
+  }
+
+  public static TEndPoint readTEndPoint(ByteBuffer buffer) {
+    TEndPoint endPoint = new TEndPoint();
+    endPoint.setIp(BasicStructureSerDeUtil.readString(buffer));
+    endPoint.setPort(buffer.getInt());
+    return endPoint;
+  }
+
+  public static void writeTDataNodeLocation(TDataNodeLocation dataNodeLocation, ByteBuffer buffer) {
+    buffer.putInt(dataNodeLocation.getDataNodeId());
+
+    buffer.put(dataNodeLocation.isSetExternalEndPoint() ? (byte) 1 : (byte) 0);
+    if (dataNodeLocation.isSetExternalEndPoint()) {
+      writeTEndPoint(dataNodeLocation.getExternalEndPoint(), buffer);
+    }
+
+    buffer.put(dataNodeLocation.isSetInternalEndPoint() ? (byte) 1 : (byte) 0);
+    if (dataNodeLocation.isSetInternalEndPoint()) {
+      writeTEndPoint(dataNodeLocation.getInternalEndPoint(), buffer);
+    }
+
+    buffer.put(dataNodeLocation.isSetDataBlockManagerEndPoint() ? (byte) 1 : (byte) 0);
+    if (dataNodeLocation.isSetDataBlockManagerEndPoint()) {
+      writeTEndPoint(dataNodeLocation.getDataBlockManagerEndPoint(), buffer);
+    }
+
+    buffer.put(dataNodeLocation.isSetConsensusEndPoint() ? (byte) 1 : (byte) 0);
+    if (dataNodeLocation.isSetConsensusEndPoint()) {
+      writeTEndPoint(dataNodeLocation.getConsensusEndPoint(), buffer);
+    }
+  }
+
+  public static TDataNodeLocation readTDataNodeLocation(ByteBuffer buffer) {
+    TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+    dataNodeLocation.setDataNodeId(buffer.getInt());
+
+    if (buffer.get() > 0) {
+      dataNodeLocation.setExternalEndPoint(readTEndPoint(buffer));
+    }
+
+    if (buffer.get() > 0) {
+      dataNodeLocation.setInternalEndPoint(readTEndPoint(buffer));
+    }
+
+    if (buffer.get() > 0) {
+      dataNodeLocation.setDataBlockManagerEndPoint(readTEndPoint(buffer));
+    }
+
+    if (buffer.get() > 0) {
+      dataNodeLocation.setConsensusEndPoint(readTEndPoint(buffer));
+    }
+    return dataNodeLocation;
+  }
+
+  public static void writeTSeriesPartitionSlot(
+      TSeriesPartitionSlot seriesPartitionSlot, ByteBuffer buffer) {
+    buffer.putInt(seriesPartitionSlot.getSlotId());
+  }
+
+  public static TSeriesPartitionSlot readTSeriesPartitionSlot(ByteBuffer buffer) {
+    return new TSeriesPartitionSlot(buffer.getInt());
+  }
+
+  public static void writeTTimePartitionSlot(
+      TTimePartitionSlot timePartitionSlot, ByteBuffer buffer) {
+    buffer.putLong(timePartitionSlot.getStartTime());
+  }
+
+  public static TTimePartitionSlot readTTimePartitionSlot(ByteBuffer buffer) {
+    return new TTimePartitionSlot(buffer.getLong());
+  }
+
+  public static void writeTConsensusGroupId(TConsensusGroupId consensusGroupId, ByteBuffer buffer) {
+    buffer.putInt(consensusGroupId.getType().ordinal());
+    buffer.putInt(consensusGroupId.getId());
+  }
+
+  public static TConsensusGroupId readTConsensusGroupId(ByteBuffer buffer) {
+    TConsensusGroupType type = TConsensusGroupType.values()[buffer.getInt()];
+    return new TConsensusGroupId(type, buffer.getInt());
+  }
+
+  public static void writeTRegionReplicaSet(TRegionReplicaSet regionReplicaSet, ByteBuffer buffer) {
+    writeTConsensusGroupId(regionReplicaSet.getRegionId(), buffer);
+    buffer.putInt(regionReplicaSet.getDataNodeLocationsSize());
+    regionReplicaSet
+        .getDataNodeLocations()
+        .forEach(dataNodeLocation -> writeTDataNodeLocation(dataNodeLocation, buffer));
+  }
+
+  public static TRegionReplicaSet readTRegionReplicaSet(ByteBuffer buffer) {
+    TConsensusGroupId consensusGroupId = readTConsensusGroupId(buffer);
+    int dataNodeLocationNum = buffer.getInt();
+    List<TDataNodeLocation> dataNodeLocations = new ArrayList<>();
+    for (int i = 0; i < dataNodeLocationNum; i++) {
+      dataNodeLocations.add(readTDataNodeLocation(buffer));
+    }
+    return new TRegionReplicaSet(consensusGroupId, dataNodeLocations);
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtils.java b/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtils.java
new file mode 100644
index 0000000000..85122f55ad
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtils.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.commons.utils;
+
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+// TODO: Serialize and deserialize by thrift structure
+/** Utils for serialize and deserialize all the data struct defined by thrift-confignode */
+public class ThriftConfigNodeSerDeUtils {
+
+  private ThriftConfigNodeSerDeUtils() {
+    // Empty constructor
+  }
+
+  public static void writeTStorageGroupSchema(
+      TStorageGroupSchema storageGroupSchema, ByteBuffer buffer) {
+    BasicStructureSerDeUtil.write(storageGroupSchema.getName(), buffer);
+    buffer.putLong(storageGroupSchema.getTTL());
+    buffer.putInt(storageGroupSchema.getSchemaReplicationFactor());
+    buffer.putInt(storageGroupSchema.getDataReplicationFactor());
+    buffer.putLong(storageGroupSchema.getTimePartitionInterval());
+
+    buffer.putInt(storageGroupSchema.getSchemaRegionGroupIdsSize());
+    if (storageGroupSchema.getSchemaRegionGroupIdsSize() > 0) {
+      storageGroupSchema
+          .getSchemaRegionGroupIds()
+          .forEach(
+              schemaRegionGroupId ->
+                  ThriftCommonsSerDeUtils.writeTConsensusGroupId(schemaRegionGroupId, buffer));
+    }
+
+    buffer.putInt(storageGroupSchema.getDataRegionGroupIdsSize());
+    if (storageGroupSchema.getDataRegionGroupIdsSize() > 0) {
+      storageGroupSchema
+          .getDataRegionGroupIds()
+          .forEach(
+              dataRegionGroupId ->
+                  ThriftCommonsSerDeUtils.writeTConsensusGroupId(dataRegionGroupId, buffer));
+    }
+  }
+
+  public static TStorageGroupSchema readTStorageGroupSchema(ByteBuffer buffer) {
+    TStorageGroupSchema storageGroupSchema = new TStorageGroupSchema();
+    storageGroupSchema.setName(BasicStructureSerDeUtil.readString(buffer));
+    storageGroupSchema.setTTL(buffer.getLong());
+    storageGroupSchema.setSchemaReplicationFactor(buffer.getInt());
+    storageGroupSchema.setDataReplicationFactor(buffer.getInt());
+    storageGroupSchema.setTimePartitionInterval(buffer.getLong());
+
+    int groupIdNum = buffer.getInt();
+    storageGroupSchema.setSchemaRegionGroupIds(new ArrayList<>());
+    for (int i = 0; i < groupIdNum; i++) {
+      storageGroupSchema
+          .getSchemaRegionGroupIds()
+          .add(ThriftCommonsSerDeUtils.readTConsensusGroupId(buffer));
+    }
+
+    groupIdNum = buffer.getInt();
+    storageGroupSchema.setDataRegionGroupIds(new ArrayList<>());
+    for (int i = 0; i < groupIdNum; i++) {
+      storageGroupSchema
+          .getDataRegionGroupIds()
+          .add(ThriftCommonsSerDeUtils.readTConsensusGroupId(buffer));
+    }
+
+    return storageGroupSchema;
+  }
+}
diff --git a/node-commons/src/test/java/org/apache/iotdb/commons/ConsensusGroupIdTest.java b/node-commons/src/test/java/org/apache/iotdb/commons/ConsensusGroupIdTest.java
index e890449d2d..af19aea35e 100644
--- a/node-commons/src/test/java/org/apache/iotdb/commons/ConsensusGroupIdTest.java
+++ b/node-commons/src/test/java/org/apache/iotdb/commons/ConsensusGroupIdTest.java
@@ -19,35 +19,32 @@
 
 package org.apache.iotdb.commons;
 
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.GroupType;
 import org.apache.iotdb.commons.consensus.SchemaRegionId;
 
 import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 
 public class ConsensusGroupIdTest {
   @Test
   public void TestCreate() throws IOException {
-    ConsensusGroupId dataRegionId = ConsensusGroupId.Factory.create(1, GroupType.DataRegion);
+    ConsensusGroupId dataRegionId =
+        ConsensusGroupId.Factory.convertFromTConsensusGroupId(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 1));
     Assert.assertTrue(dataRegionId instanceof DataRegionId);
     Assert.assertEquals(1, dataRegionId.getId());
-    Assert.assertEquals(GroupType.DataRegion, dataRegionId.getType());
+    Assert.assertEquals(TConsensusGroupType.DataRegion, dataRegionId.getType());
 
-    ConsensusGroupId schemaRegionId = ConsensusGroupId.Factory.create(2, GroupType.SchemaRegion);
+    ConsensusGroupId schemaRegionId =
+        ConsensusGroupId.Factory.convertFromTConsensusGroupId(
+            new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 2));
     Assert.assertTrue(schemaRegionId instanceof SchemaRegionId);
     Assert.assertEquals(2, schemaRegionId.getId());
-    Assert.assertEquals(GroupType.SchemaRegion, schemaRegionId.getType());
-
-    ByteBuffer buffer = ByteBuffer.allocate(1024);
-    schemaRegionId.serializeImpl(buffer);
-    buffer.flip();
-
-    ConsensusGroupId schemaRegionIdClone = ConsensusGroupId.Factory.create(buffer);
-    Assert.assertEquals(schemaRegionId, schemaRegionIdClone);
+    Assert.assertEquals(TConsensusGroupType.SchemaRegion, schemaRegionId.getType());
   }
 }
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/utils/SerializeDeserializeUtilTest.java b/node-commons/src/test/java/org/apache/iotdb/commons/utils/BasicStructureSerDeUtilTest.java
similarity index 82%
rename from confignode/src/test/java/org/apache/iotdb/confignode/utils/SerializeDeserializeUtilTest.java
rename to node-commons/src/test/java/org/apache/iotdb/commons/utils/BasicStructureSerDeUtilTest.java
index 26b2d75aec..9e6a5f78fb 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/utils/SerializeDeserializeUtilTest.java
+++ b/node-commons/src/test/java/org/apache/iotdb/commons/utils/BasicStructureSerDeUtilTest.java
@@ -16,9 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.confignode.utils;
-
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
+package org.apache.iotdb.commons.utils;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -30,7 +28,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-public class SerializeDeserializeUtilTest {
+public class BasicStructureSerDeUtilTest {
   protected static final int DEFAULT_BUFFER_SIZE = 4096;
 
   @Test
@@ -42,10 +40,10 @@ public class SerializeDeserializeUtilTest {
     map.put(key, value);
 
     ByteBuffer bf = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE);
-    SerializeDeserializeUtil.write(map, bf);
+    BasicStructureSerDeUtil.write(map, bf);
     byte[] b = bf.array();
     bf.clear();
-    Map<String, String> result = SerializeDeserializeUtil.readMap(ByteBuffer.wrap(b));
+    Map<String, String> result = BasicStructureSerDeUtil.readMap(ByteBuffer.wrap(b));
     Assert.assertNotNull(result);
     Assert.assertEquals(map, result);
   }
@@ -61,11 +59,11 @@ public class SerializeDeserializeUtilTest {
     stringMapLists.put(key, keyLists);
 
     ByteBuffer bf = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE);
-    SerializeDeserializeUtil.writeStringMapLists(stringMapLists, bf);
+    BasicStructureSerDeUtil.writeStringMapLists(stringMapLists, bf);
     byte[] b = bf.array();
     bf.clear();
     Map<String, List<String>> stringMapListsResult =
-        SerializeDeserializeUtil.readStringMapLists(ByteBuffer.wrap(b));
+        BasicStructureSerDeUtil.readStringMapLists(ByteBuffer.wrap(b));
     Assert.assertNotNull(stringMapListsResult);
     Assert.assertEquals(stringMapLists, stringMapListsResult);
   }
@@ -81,9 +79,9 @@ public class SerializeDeserializeUtilTest {
     ByteBuffer bf = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE);
     byte[] b = bf.array();
 
-    SerializeDeserializeUtil.writeIntMapLists(integerMapLists, bf);
+    BasicStructureSerDeUtil.writeIntMapLists(integerMapLists, bf);
     Map<Integer, List<Integer>> intMapListsResult =
-        SerializeDeserializeUtil.readIntMapLists(ByteBuffer.wrap(b));
+        BasicStructureSerDeUtil.readIntMapLists(ByteBuffer.wrap(b));
     Assert.assertNotNull(intMapListsResult);
     Assert.assertEquals(integerMapLists, intMapListsResult);
   }
diff --git a/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtilsTest.java b/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtilsTest.java
new file mode 100644
index 0000000000..5ae5619dbb
--- /dev/null
+++ b/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtilsTest.java
@@ -0,0 +1,116 @@
+/*
+ * 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.commons.utils;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+public class ThriftCommonsSerDeUtilsTest {
+
+  private static final ByteBuffer buffer = ByteBuffer.allocate(1024 * 10);
+
+  @After
+  public void cleanBuffer() {
+    buffer.clear();
+  }
+
+  @Test
+  public void readWriteTEndPointTest() {
+    TEndPoint endPoint0 = new TEndPoint("0.0.0.0", 6667);
+    ThriftCommonsSerDeUtils.writeTEndPoint(endPoint0, buffer);
+    buffer.flip();
+    TEndPoint endPoint1 = ThriftCommonsSerDeUtils.readTEndPoint(buffer);
+    Assert.assertEquals(endPoint0, endPoint1);
+  }
+
+  @Test
+  public void readWriteTDataNodeLocationTest() {
+    TDataNodeLocation dataNodeLocation0 = new TDataNodeLocation();
+    dataNodeLocation0.setDataNodeId(0);
+    dataNodeLocation0.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+    dataNodeLocation0.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+    dataNodeLocation0.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+    dataNodeLocation0.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010));
+    ThriftCommonsSerDeUtils.writeTDataNodeLocation(dataNodeLocation0, buffer);
+    buffer.flip();
+    TDataNodeLocation dataNodeLocation1 = ThriftCommonsSerDeUtils.readTDataNodeLocation(buffer);
+    Assert.assertEquals(dataNodeLocation0, dataNodeLocation1);
+  }
+
+  @Test
+  public void readWriteTSeriesPartitionSlotTest() {
+    TSeriesPartitionSlot seriesPartitionSlot0 = new TSeriesPartitionSlot(10);
+    ThriftCommonsSerDeUtils.writeTSeriesPartitionSlot(seriesPartitionSlot0, buffer);
+    buffer.flip();
+    TSeriesPartitionSlot seriesPartitionSlot1 =
+        ThriftCommonsSerDeUtils.readTSeriesPartitionSlot(buffer);
+    Assert.assertEquals(seriesPartitionSlot0, seriesPartitionSlot1);
+  }
+
+  @Test
+  public void writeTTimePartitionSlot() {
+    TTimePartitionSlot timePartitionSlot0 = new TTimePartitionSlot(100);
+    ThriftCommonsSerDeUtils.writeTTimePartitionSlot(timePartitionSlot0, buffer);
+    buffer.flip();
+    TTimePartitionSlot timePartitionSlot1 = ThriftCommonsSerDeUtils.readTTimePartitionSlot(buffer);
+    Assert.assertEquals(timePartitionSlot0, timePartitionSlot1);
+  }
+
+  @Test
+  public void readWriteTConsensusGroupIdTest() {
+    TConsensusGroupId consensusGroupId0 =
+        new TConsensusGroupId(TConsensusGroupType.PartitionRegion, 0);
+    ThriftCommonsSerDeUtils.writeTConsensusGroupId(consensusGroupId0, buffer);
+    buffer.flip();
+    TConsensusGroupId consensusGroupId1 = ThriftCommonsSerDeUtils.readTConsensusGroupId(buffer);
+    Assert.assertEquals(consensusGroupId0, consensusGroupId1);
+  }
+
+  @Test
+  public void readWriteTRegionReplicaSetTest() {
+    TRegionReplicaSet regionReplicaSet0 = new TRegionReplicaSet();
+    regionReplicaSet0.setRegionId(new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 0));
+    regionReplicaSet0.setDataNodeLocations(new ArrayList<>());
+    for (int i = 0; i < 3; i++) {
+      TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+      dataNodeLocation.setDataNodeId(i);
+      dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667 + i));
+      dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003 + i));
+      dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777 + i));
+      dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010 + i));
+      regionReplicaSet0.getDataNodeLocations().add(dataNodeLocation);
+    }
+    ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet0, buffer);
+    buffer.flip();
+    TRegionReplicaSet regionReplicaSet1 = ThriftCommonsSerDeUtils.readTRegionReplicaSet(buffer);
+    Assert.assertEquals(regionReplicaSet0, regionReplicaSet1);
+  }
+}
diff --git a/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtilsTest.java b/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtilsTest.java
new file mode 100644
index 0000000000..282ae59233
--- /dev/null
+++ b/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtilsTest.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.commons.utils;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+public class ThriftConfigNodeSerDeUtilsTest {
+
+  private static final ByteBuffer buffer = ByteBuffer.allocate(1024 * 10);
+
+  @After
+  public void cleanBuffer() {
+    buffer.clear();
+  }
+
+  @Test
+  public void readWriteTStorageGroupSchemaTest() {
+    TStorageGroupSchema storageGroupSchema0 = new TStorageGroupSchema();
+    storageGroupSchema0.setName("root.sg");
+    storageGroupSchema0.setTTL(Long.MAX_VALUE);
+    storageGroupSchema0.setSchemaReplicationFactor(3);
+    storageGroupSchema0.setDataReplicationFactor(3);
+    storageGroupSchema0.setTimePartitionInterval(604800);
+
+    storageGroupSchema0.setSchemaRegionGroupIds(new ArrayList<>());
+    storageGroupSchema0.setDataRegionGroupIds(new ArrayList<>());
+    for (int i = 0; i < 3; i++) {
+      storageGroupSchema0
+          .getSchemaRegionGroupIds()
+          .add(new TConsensusGroupId(TConsensusGroupType.SchemaRegion, i * 2));
+      storageGroupSchema0
+          .getDataRegionGroupIds()
+          .add(new TConsensusGroupId(TConsensusGroupType.DataRegion, i * 2 + 1));
+    }
+
+    ThriftConfigNodeSerDeUtils.writeTStorageGroupSchema(storageGroupSchema0, buffer);
+    buffer.flip();
+    TStorageGroupSchema storageGroupSchema1 =
+        ThriftConfigNodeSerDeUtils.readTStorageGroupSchema(buffer);
+    Assert.assertEquals(storageGroupSchema0, storageGroupSchema1);
+  }
+}
diff --git a/pom.xml b/pom.xml
index 90a8dbccb4..06d6555fde 100644
--- a/pom.xml
+++ b/pom.xml
@@ -82,6 +82,7 @@
         <module>tsfile</module>
         <module>antlr</module>
         <module>thrift</module>
+        <module>thrift-commons</module>
         <module>thrift-confignode</module>
         <module>thrift-cluster</module>
         <module>thrift-sync</module>
@@ -742,6 +743,7 @@
                         <!-- put all source folders not in src/main/java here-->
                         <sourceDirectory>antlr/target/generated-sources/antlr4</sourceDirectory>
                         <sourceDirectory>thrift/target/generated-sources/thrift</sourceDirectory>
+                        <sourceDirectory>thrift-commons/target/generated-sources/thrift</sourceDirectory>
                         <sourceDirectory>thrift-confignode/target/generated-sources/thrift</sourceDirectory>
                         <sourceDirectory>thrift-sync/target/generated-sources/thrift</sourceDirectory>
                         <sourceDirectory>thrift-cluster/target/generated-sources/thrift</sourceDirectory>
diff --git a/server/pom.xml b/server/pom.xml
index e6c9c29adf..cf18fa5054 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -50,11 +50,6 @@
             <artifactId>iotdb-consensus</artifactId>
             <version>${project.version}</version>
         </dependency>
-        <dependency>
-            <groupId>org.apache.iotdb</groupId>
-            <artifactId>node-commons</artifactId>
-            <version>${project.version}</version>
-        </dependency>
         <dependency>
             <groupId>org.apache.iotdb</groupId>
             <artifactId>tsfile</artifactId>
@@ -82,11 +77,6 @@
             <artifactId>influxdb-thrift</artifactId>
             <version>${project.version}</version>
         </dependency>
-        <dependency>
-            <groupId>org.apache.iotdb</groupId>
-            <artifactId>iotdb-thrift-confignode</artifactId>
-            <version>${project.version}</version>
-        </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
             <artifactId>commons-collections4</artifactId>
@@ -260,6 +250,11 @@
             <artifactId>rocksdbjni</artifactId>
             <version>6.27.3</version>
         </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>[${guava.version},)</version>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
index 4c2d0d7ed3..06bb8ff613 100644
--- a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
+++ b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
@@ -19,13 +19,13 @@
 
 package org.apache.iotdb.db.client;
 
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
 import org.apache.iotdb.commons.exception.BadNodeUrlException;
 import org.apache.iotdb.commons.utils.CommonUtils;
 import org.apache.iotdb.confignode.rpc.thrift.ConfigIService;
 import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerReq;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessageResp;
+import org.apache.iotdb.confignode.rpc.thrift.TDataNodeLocationResp;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterResp;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
@@ -65,9 +65,9 @@ public class ConfigNodeClient {
 
   private TTransport transport;
 
-  private Endpoint configLeader;
+  private TEndPoint configLeader;
 
-  private List<Endpoint> configNodes;
+  private List<TEndPoint> configNodes;
 
   public ConfigNodeClient() throws BadNodeUrlException, IoTDBConnectionException {
     // Read config nodes from configuration
@@ -76,12 +76,12 @@ public class ConfigNodeClient {
     init();
   }
 
-  public ConfigNodeClient(List<Endpoint> configNodes) throws IoTDBConnectionException {
+  public ConfigNodeClient(List<TEndPoint> configNodes) throws IoTDBConnectionException {
     this.configNodes = configNodes;
     init();
   }
 
-  public ConfigNodeClient(List<Endpoint> configNodes, Endpoint configLeader)
+  public ConfigNodeClient(List<TEndPoint> configNodes, TEndPoint configLeader)
       throws IoTDBConnectionException {
     this.configNodes = configNodes;
     this.configLeader = configLeader;
@@ -92,7 +92,7 @@ public class ConfigNodeClient {
     reconnect();
   }
 
-  public void connect(Endpoint endpoint) throws IoTDBConnectionException {
+  public void connect(TEndPoint endpoint) throws IoTDBConnectionException {
     try {
       transport =
           RpcTransportFactory.INSTANCE.getTransport(
@@ -131,7 +131,7 @@ public class ConfigNodeClient {
       if (tryHostNum == configNodes.size()) {
         break;
       }
-      Endpoint tryEndpoint = configNodes.get(j);
+      TEndPoint tryEndpoint = configNodes.get(j);
       if (j == configNodes.size() - 1) {
         j = -1;
       }
@@ -154,7 +154,7 @@ public class ConfigNodeClient {
     if (status.getCode() == TSStatusCode.NEED_REDIRECTION.getStatusCode()) {
       if (status.isSetRedirectNode()) {
         configLeader =
-            new Endpoint(status.getRedirectNode().getIp(), status.getRedirectNode().getPort());
+            new TEndPoint(status.getRedirectNode().getIp(), status.getRedirectNode().getPort());
       } else {
         configLeader = null;
       }
@@ -180,10 +180,11 @@ public class ConfigNodeClient {
     throw new IoTDBConnectionException(MSG_RECONNECTION_FAIL);
   }
 
-  public TDataNodeMessageResp getDataNodesMessage(int dataNodeID) throws IoTDBConnectionException {
+  public TDataNodeLocationResp getDataNodeLocations(int dataNodeID)
+      throws IoTDBConnectionException {
     for (int i = 0; i < RETRY_NUM; i++) {
       try {
-        TDataNodeMessageResp resp = client.getDataNodesMessage(dataNodeID);
+        TDataNodeLocationResp resp = client.getDataNodeLocations(dataNodeID);
         if (!updateConfigNodeLeader(resp.status)) {
           return resp;
         }
diff --git a/server/src/main/java/org/apache/iotdb/db/consensus/ConsensusImpl.java b/server/src/main/java/org/apache/iotdb/db/consensus/ConsensusImpl.java
index 972221609f..96fd26af89 100644
--- a/server/src/main/java/org/apache/iotdb/db/consensus/ConsensusImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/consensus/ConsensusImpl.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.consensus;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.commons.consensus.DataRegionId;
 import org.apache.iotdb.commons.consensus.SchemaRegionId;
 import org.apache.iotdb.consensus.ConsensusFactory;
@@ -51,7 +51,7 @@ public class ConsensusImpl {
     private static final IConsensus INSTANCE =
         ConsensusFactory.getConsensusImpl(
                 conf.getConsensusProtocolClass(),
-                new Endpoint(conf.getInternalIp(), conf.getConsensusPort()),
+                new TEndPoint(conf.getInternalIp(), conf.getConsensusPort()),
                 new File(conf.getConsensusDir()),
                 gid -> {
                   switch (gid.getType()) {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
index 256278a020..5b89a48e71 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
@@ -19,10 +19,10 @@
 package org.apache.iotdb.db.engine;
 
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.commons.concurrent.ThreadName;
 import org.apache.iotdb.commons.exception.ShutdownException;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
 import org.apache.iotdb.commons.service.IService;
 import org.apache.iotdb.commons.service.ServiceType;
 import org.apache.iotdb.commons.utils.TestOnly;
@@ -182,8 +182,8 @@ public class StorageEngine implements IService {
     return enablePartition ? time / timePartitionInterval : 0;
   }
 
-  public static TimePartitionSlot getTimePartitionSlot(long time) {
-    TimePartitionSlot timePartitionSlot = new TimePartitionSlot();
+  public static TTimePartitionSlot getTimePartitionSlot(long time) {
+    TTimePartitionSlot timePartitionSlot = new TTimePartitionSlot();
     if (enablePartition) {
       timePartitionSlot.setStartTime(time - time % timePartitionInterval);
     } else {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockManager.java b/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockManager.java
index 573771fe81..6a8f34ccf5 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockManager.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.mpp.buffer;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.mpp.execution.FragmentInstanceContext;
 import org.apache.iotdb.db.mpp.memory.LocalMemoryManager;
 import org.apache.iotdb.mpp.rpc.thrift.DataBlockService;
@@ -258,7 +258,7 @@ public class DataBlockManager implements IDataBlockManager {
   @Override
   public ISinkHandle createSinkHandle(
       TFragmentInstanceId localFragmentInstanceId,
-      Endpoint endpoint,
+      TEndPoint endpoint,
       TFragmentInstanceId remoteFragmentInstanceId,
       String remotePlanNodeId,
       FragmentInstanceContext instanceContext) {
@@ -291,7 +291,7 @@ public class DataBlockManager implements IDataBlockManager {
   public ISourceHandle createSourceHandle(
       TFragmentInstanceId localFragmentInstanceId,
       String localPlanNodeId,
-      Endpoint endpoint,
+      TEndPoint endpoint,
       TFragmentInstanceId remoteFragmentInstanceId) {
     if (sourceHandles.containsKey(localFragmentInstanceId)
         && sourceHandles.get(localFragmentInstanceId).containsKey(localPlanNodeId)) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockServiceClientFactory.java b/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockServiceClientFactory.java
index 397755cd4e..a363fed2bb 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockServiceClientFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockServiceClientFactory.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.mpp.buffer;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.mpp.rpc.thrift.DataBlockService;
 import org.apache.iotdb.rpc.RpcTransportFactory;
@@ -47,10 +47,10 @@ public class DataBlockServiceClientFactory {
   private static final Logger logger = LoggerFactory.getLogger(DataBlockServiceClientFactory.class);
 
   // TODO need to be replaced by mature client pool in the future
-  private static final Map<Endpoint, DataBlockService.Iface> dataBlockServiceClientMap =
+  private static final Map<TEndPoint, DataBlockService.Iface> dataBlockServiceClientMap =
       new ConcurrentHashMap<>();
 
-  public DataBlockService.Iface getDataBlockServiceClient(Endpoint endpoint) {
+  public DataBlockService.Iface getDataBlockServiceClient(TEndPoint endpoint) {
 
     return dataBlockServiceClientMap.computeIfAbsent(
         endpoint,
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/buffer/IDataBlockManager.java b/server/src/main/java/org/apache/iotdb/db/mpp/buffer/IDataBlockManager.java
index 0eff0c6620..0d60d346db 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/buffer/IDataBlockManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/buffer/IDataBlockManager.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.mpp.buffer;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.mpp.execution.FragmentInstanceContext;
 import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceId;
 
@@ -36,7 +36,7 @@ public interface IDataBlockManager {
    */
   ISinkHandle createSinkHandle(
       TFragmentInstanceId localFragmentInstanceId,
-      Endpoint endpoint,
+      TEndPoint endpoint,
       TFragmentInstanceId remoteFragmentInstanceId,
       String remotePlanNodeId,
       FragmentInstanceContext instanceContext);
@@ -55,7 +55,7 @@ public interface IDataBlockManager {
   ISourceHandle createSourceHandle(
       TFragmentInstanceId localFragmentInstanceId,
       String localPlanNodeId,
-      Endpoint endpoint,
+      TEndPoint endpoint,
       TFragmentInstanceId remoteFragmentInstanceId);
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/MPPQueryContext.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/MPPQueryContext.java
index b449bdf9e7..da8d42c6de 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/MPPQueryContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/MPPQueryContext.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.common;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.mpp.sql.analyze.QueryType;
 
 /**
@@ -31,14 +31,14 @@ public class MPPQueryContext {
   private SessionInfo session;
   private QueryType queryType = QueryType.READ;
 
-  private Endpoint hostEndpoint;
+  private TEndPoint hostEndpoint;
   private ResultNodeContext resultNodeContext;
 
   public MPPQueryContext(QueryId queryId) {
     this.queryId = queryId;
   }
 
-  public MPPQueryContext(String sql, QueryId queryId, SessionInfo session, Endpoint hostEndpoint) {
+  public MPPQueryContext(String sql, QueryId queryId, SessionInfo session, TEndPoint hostEndpoint) {
     this.sql = sql;
     this.queryId = queryId;
     this.session = session;
@@ -58,7 +58,7 @@ public class MPPQueryContext {
     this.queryType = queryType;
   }
 
-  public Endpoint getHostEndpoint() {
+  public TEndPoint getHostEndpoint() {
     return hostEndpoint;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/ResultNodeContext.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/ResultNodeContext.java
index 4c56646285..dcec963371 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/ResultNodeContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/ResultNodeContext.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.mpp.common;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
 
 public class ResultNodeContext {
@@ -27,7 +27,7 @@ public class ResultNodeContext {
   private final FragmentInstanceId virtualFragmentInstanceId;
   private final PlanNodeId virtualResultNodeId;
 
-  private Endpoint upStreamEndpoint;
+  private TEndPoint upStreamEndpoint;
   private FragmentInstanceId upStreamFragmentInstanceId;
   private PlanNodeId upStreamPlanNodeId;
 
@@ -37,7 +37,7 @@ public class ResultNodeContext {
   }
 
   public void setUpStream(
-      Endpoint upStreamEndpoint,
+      TEndPoint upStreamEndpoint,
       FragmentInstanceId upStreamFragmentInstanceId,
       PlanNodeId upStreamPlanNodeId) {
     this.upStreamEndpoint = upStreamEndpoint;
@@ -53,7 +53,7 @@ public class ResultNodeContext {
     return virtualResultNodeId;
   }
 
-  public Endpoint getUpStreamEndpoint() {
+  public TEndPoint getUpStreamEndpoint() {
     return upStreamEndpoint;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/Coordinator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/Coordinator.java
index a7d09b3ccf..ce589f8575 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/Coordinator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/Coordinator.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.execution;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.mpp.common.MPPQueryContext;
@@ -52,8 +52,8 @@ public class Coordinator {
   private static final String COORDINATOR_SCHEDULED_EXECUTOR_NAME = "MPPCoordinatorScheduled";
   private static final int COORDINATOR_SCHEDULED_EXECUTOR_SIZE = 1;
 
-  private static final Endpoint LOCAL_HOST =
-      new Endpoint(
+  private static final TEndPoint LOCAL_HOST =
+      new TEndPoint(
           IoTDBDescriptor.getInstance().getConfig().getRpcAddress(),
           IoTDBDescriptor.getInstance().getConfig().getInternalPort());
 
@@ -121,7 +121,7 @@ public class Coordinator {
   }
 
   // Get the hostname of current coordinator
-  private Endpoint getHostEndpoint() {
+  private TEndPoint getHostEndpoint() {
     return LOCAL_HOST;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/QueryExecution.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/QueryExecution.java
index a9bd605823..f362a48a72 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/QueryExecution.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/QueryExecution.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.execution;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.mpp.buffer.DataBlockService;
 import org.apache.iotdb.db.mpp.buffer.ISourceHandle;
@@ -280,7 +280,7 @@ public class QueryExecution implements IQueryExecution {
               .createSourceHandle(
                   context.getResultNodeContext().getVirtualFragmentInstanceId().toThrift(),
                   context.getResultNodeContext().getVirtualResultNodeId().getId(),
-                  new Endpoint(
+                  new TEndPoint(
                       context.getResultNodeContext().getUpStreamEndpoint().getIp(),
                       IoTDBDescriptor.getInstance().getConfig().getDataBlockManagerPort()),
                   context.getResultNodeContext().getVirtualFragmentInstanceId().toThrift());
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/config/SetStorageGroupTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/config/SetStorageGroupTask.java
index 934d671e27..a28c7924c4 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/config/SetStorageGroupTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/config/SetStorageGroupTask.java
@@ -22,6 +22,7 @@ package org.apache.iotdb.db.mpp.execution.config;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.exception.BadNodeUrlException;
 import org.apache.iotdb.confignode.rpc.thrift.TSetStorageGroupReq;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.db.client.ConfigNodeClient;
 import org.apache.iotdb.db.mpp.sql.statement.metadata.SetStorageGroupStatement;
 import org.apache.iotdb.rpc.IoTDBConnectionException;
@@ -45,8 +46,9 @@ public class SetStorageGroupTask implements IConfigTask {
   public ListenableFuture<ConfigTaskResult> execute() {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     // Construct request using statement
-    TSetStorageGroupReq req =
-        new TSetStorageGroupReq(setStorageGroupStatement.getStorageGroupPath().getFullPath());
+    TStorageGroupSchema storageGroupSchema = new TStorageGroupSchema();
+    storageGroupSchema.setName(setStorageGroupStatement.getStorageGroupPath().getFullPath());
+    TSetStorageGroupReq req = new TSetStorageGroupReq(storageGroupSchema);
 
     ConfigNodeClient configNodeClient = null;
     try {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/AbstractFragInsStateTracker.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/AbstractFragInsStateTracker.java
index 82abc70c3f..71706b4857 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/AbstractFragInsStateTracker.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/AbstractFragInsStateTracker.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.mpp.execution.scheduler;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.mpp.execution.FragmentInstanceState;
 import org.apache.iotdb.db.mpp.execution.QueryStateMachine;
@@ -61,7 +61,7 @@ public abstract class AbstractFragInsStateTracker implements IFragInstanceStateT
     // TODO (jackie tien) change the port
     InternalService.Iface client =
         InternalServiceClientFactory.getInternalServiceClient(
-            new Endpoint(
+            new TEndPoint(
                 instance.getHostEndpoint().getIp(),
                 IoTDBDescriptor.getInstance().getConfig().getInternalPort()));
     TFragmentInstanceStateResp resp =
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/InternalServiceClientFactory.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/InternalServiceClientFactory.java
index c1c76f1e07..aa35c3854b 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/InternalServiceClientFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/InternalServiceClientFactory.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.mpp.execution.scheduler;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.mpp.rpc.thrift.InternalService;
 import org.apache.iotdb.rpc.RpcTransportFactory;
 
@@ -44,10 +44,10 @@ public class InternalServiceClientFactory {
   private static final Logger logger = LoggerFactory.getLogger(InternalServiceClientFactory.class);
 
   // TODO need to be replaced by mature client pool in the future
-  private static final Map<Endpoint, InternalService.Iface> internalServiceClientMap =
+  private static final Map<TEndPoint, InternalService.Iface> internalServiceClientMap =
       new ConcurrentHashMap<>();
 
-  public static InternalService.Iface getInternalServiceClient(Endpoint endpoint)
+  public static InternalService.Iface getInternalServiceClient(TEndPoint endpoint)
       throws TTransportException {
     return internalServiceClientMap.computeIfAbsent(
         endpoint,
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleFragInstanceDispatcher.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleFragInstanceDispatcher.java
index 09ff1171b9..b2e9252f93 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleFragInstanceDispatcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleFragInstanceDispatcher.java
@@ -19,11 +19,11 @@
 
 package org.apache.iotdb.db.mpp.execution.scheduler;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.mpp.sql.planner.plan.FragmentInstance;
 import org.apache.iotdb.mpp.rpc.thrift.InternalService;
-import org.apache.iotdb.mpp.rpc.thrift.TConsensusGroupId;
 import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstance;
 import org.apache.iotdb.mpp.rpc.thrift.TSendFragmentInstanceReq;
 import org.apache.iotdb.mpp.rpc.thrift.TSendFragmentInstanceResp;
@@ -50,19 +50,17 @@ public class SimpleFragInstanceDispatcher implements IFragInstanceDispatcher {
         () -> {
           TSendFragmentInstanceResp resp = new TSendFragmentInstanceResp(false);
           for (FragmentInstance instance : instances) {
+            // TODO: (jackie tien) change the port
             InternalService.Iface client =
                 InternalServiceClientFactory.getInternalServiceClient(
-                    new Endpoint(
+                    new TEndPoint(
                         instance.getHostEndpoint().getIp(),
                         IoTDBDescriptor.getInstance().getConfig().getInternalPort()));
             // TODO: (xingtanzjr) consider how to handle the buffer here
             ByteBuffer buffer = ByteBuffer.allocate(1024 * 1024);
             instance.serializeRequest(buffer);
             buffer.flip();
-            TConsensusGroupId groupId =
-                new TConsensusGroupId(
-                    instance.getRegionReplicaSet().getConsensusGroupId().getId(),
-                    instance.getRegionReplicaSet().getConsensusGroupId().getType().toString());
+            TConsensusGroupId groupId = instance.getRegionReplicaSet().getRegionId();
             TSendFragmentInstanceReq req =
                 new TSendFragmentInstanceReq(
                     new TFragmentInstance(buffer), groupId, instance.getType().toString());
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleQueryTerminator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleQueryTerminator.java
index 5ffe02080a..856d8d6e6d 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleQueryTerminator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleQueryTerminator.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.mpp.execution.scheduler;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.mpp.common.QueryId;
 import org.apache.iotdb.db.mpp.sql.planner.plan.FragmentInstance;
@@ -50,16 +50,16 @@ public class SimpleQueryTerminator implements IQueryTerminator {
 
   @Override
   public Future<Boolean> terminate() {
-    List<Endpoint> relatedHost = getRelatedHost(fragmentInstances);
+    List<TEndPoint> relatedHost = getRelatedHost(fragmentInstances);
 
     return executor.submit(
         () -> {
           try {
-            for (Endpoint endpoint : relatedHost) {
+            for (TEndPoint endpoint : relatedHost) {
               // TODO (jackie tien) change the port
               InternalService.Iface client =
                   InternalServiceClientFactory.getInternalServiceClient(
-                      new Endpoint(
+                      new TEndPoint(
                           endpoint.getIp(),
                           IoTDBDescriptor.getInstance().getConfig().getInternalPort()));
               client.cancelQuery(new TCancelQueryReq(queryId.getId()));
@@ -71,7 +71,7 @@ public class SimpleQueryTerminator implements IQueryTerminator {
         });
   }
 
-  private List<Endpoint> getRelatedHost(List<FragmentInstance> instances) {
+  private List<TEndPoint> getRelatedHost(List<FragmentInstance> instances) {
     return instances.stream()
         .map(FragmentInstance::getHostEndpoint)
         .distinct()
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/Analysis.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/Analysis.java
index a9e945ab4a..9b1aa3c651 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/Analysis.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/Analysis.java
@@ -19,8 +19,8 @@
 
 package org.apache.iotdb.db.mpp.sql.analyze;
 
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.commons.partition.DataPartition;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
 import org.apache.iotdb.commons.partition.SchemaPartition;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
@@ -58,7 +58,7 @@ public class Analysis {
 
   public Analysis() {}
 
-  public List<RegionReplicaSet> getPartitionInfo(PartialPath seriesPath, Filter timefilter) {
+  public List<TRegionReplicaSet> getPartitionInfo(PartialPath seriesPath, Filter timefilter) {
     // TODO: (xingtanzjr) implement the calculation of timePartitionIdList
     return dataPartition.getDataRegionReplicaSet(seriesPath.getDevice(), null);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterPartitionFetcher.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterPartitionFetcher.java
index f8a9d348be..b2ecaa612c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterPartitionFetcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterPartitionFetcher.java
@@ -24,10 +24,7 @@ import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.commons.exception.BadNodeUrlException;
 import org.apache.iotdb.commons.partition.DataPartition;
 import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
 import org.apache.iotdb.commons.partition.SchemaPartition;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
 import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionResp;
@@ -195,17 +192,17 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
   }
 
   private SchemaPartition parseSchemaPartitionResp(TSchemaPartitionResp schemaPartitionResp) {
-    Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> schemaPartitionMap = new HashMap<>();
+    Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap = new HashMap<>();
     for (Map.Entry<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> sgEntry :
         schemaPartitionResp.getSchemaRegionMap().entrySet()) {
       // for each sg
       String storageGroupName = sgEntry.getKey();
-      Map<SeriesPartitionSlot, RegionReplicaSet> deviceToSchemaRegionMap = new HashMap<>();
+      Map<TSeriesPartitionSlot, TRegionReplicaSet> deviceToSchemaRegionMap = new HashMap<>();
       for (Map.Entry<TSeriesPartitionSlot, TRegionReplicaSet> deviceEntry :
           sgEntry.getValue().entrySet()) {
         deviceToSchemaRegionMap.put(
-            new SeriesPartitionSlot(deviceEntry.getKey().getSlotId()),
-            new RegionReplicaSet(deviceEntry.getValue()));
+            new TSeriesPartitionSlot(deviceEntry.getKey().getSlotId()),
+            new TRegionReplicaSet(deviceEntry.getValue()));
       }
       schemaPartitionMap.put(storageGroupName, deviceToSchemaRegionMap);
     }
@@ -216,7 +213,7 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
   }
 
   private DataPartition parseDataPartitionResp(TDataPartitionResp dataPartitionResp) {
-    Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+    Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
         dataPartitionMap = new HashMap<>();
     for (Map.Entry<
             String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
@@ -225,23 +222,24 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
       String storageGroupName = sgEntry.getKey();
       Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>
           respDeviceToRegionsMap = sgEntry.getValue();
-      Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>> deviceToRegionsMap =
-          new HashMap<>();
+      Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>
+          deviceToRegionsMap = new HashMap<>();
       for (Map.Entry<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>
           deviceEntry : respDeviceToRegionsMap.entrySet()) {
         // for each device
-        Map<TimePartitionSlot, List<RegionReplicaSet>> timePartitionToRegionsMap = new HashMap<>();
+        Map<TTimePartitionSlot, List<TRegionReplicaSet>> timePartitionToRegionsMap =
+            new HashMap<>();
         for (Map.Entry<TTimePartitionSlot, List<TRegionReplicaSet>> timePartitionEntry :
             deviceEntry.getValue().entrySet()) {
           // for each time partition
           timePartitionToRegionsMap.put(
-              new TimePartitionSlot(timePartitionEntry.getKey().getStartTime()),
+              new TTimePartitionSlot(timePartitionEntry.getKey().getStartTime()),
               timePartitionEntry.getValue().stream()
-                  .map(RegionReplicaSet::new)
+                  .map(TRegionReplicaSet::new)
                   .collect(Collectors.toList()));
         }
         deviceToRegionsMap.put(
-            new SeriesPartitionSlot(deviceEntry.getKey().getSlotId()), timePartitionToRegionsMap);
+            new TSeriesPartitionSlot(deviceEntry.getKey().getSlotId()), timePartitionToRegionsMap);
       }
       dataPartitionMap.put(storageGroupName, deviceToRegionsMap);
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterSchemaFetcher.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterSchemaFetcher.java
index 929ca01a55..969549998b 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterSchemaFetcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterSchemaFetcher.java
@@ -18,9 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.analyze;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
 import org.apache.iotdb.commons.partition.SchemaPartition;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.mpp.common.QueryId;
 import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
@@ -62,7 +62,7 @@ public class ClusterSchemaFetcher implements ISchemaFetcher {
   @Override
   public SchemaTree fetchSchema(PathPatternTree patternTree) {
     SchemaPartition schemaPartition = partitionFetcher.getSchemaPartition(patternTree);
-    Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> schemaPartitionMap =
+    Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap =
         schemaPartition.getSchemaPartitionMap();
     List<String> storageGroups = new ArrayList<>(schemaPartitionMap.keySet());
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/FakePartitionFetcherImpl.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/FakePartitionFetcherImpl.java
index 3e72186310..eb0d8bec34 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/FakePartitionFetcherImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/FakePartitionFetcherImpl.java
@@ -19,14 +19,24 @@
 
 package org.apache.iotdb.db.mpp.sql.analyze;
 
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.*;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+import org.apache.iotdb.commons.partition.DataPartition;
+import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
+import org.apache.iotdb.commons.partition.SchemaPartition;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 public class FakePartitionFetcherImpl implements IPartitionFetcher {
 
@@ -51,56 +61,76 @@ public class FakePartitionFetcherImpl implements IPartitionFetcher {
         new DataPartition(
             IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionExecutorClass(),
             IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionSlotNum());
-    Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+    Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
         dataPartitionMap = new HashMap<>();
-    Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>> sgPartitionMap =
+    Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>> sgPartitionMap =
         new HashMap<>();
 
-    List<RegionReplicaSet> d1DataRegions = new ArrayList<>();
+    List<TRegionReplicaSet> d1DataRegions = new ArrayList<>();
     d1DataRegions.add(
-        new RegionReplicaSet(
-            new DataRegionId(1),
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 1),
             Arrays.asList(
-                new DataNodeLocation(11, new Endpoint("192.0.1.1", 9000)),
-                new DataNodeLocation(12, new Endpoint("192.0.1.2", 9000)))));
+                new TDataNodeLocation()
+                    .setDataNodeId(11)
+                    .setExternalEndPoint(new TEndPoint("192.0.1.1", 9000)),
+                new TDataNodeLocation()
+                    .setDataNodeId(12)
+                    .setExternalEndPoint(new TEndPoint("192.0.1.2", 9000)))));
     d1DataRegions.add(
-        new RegionReplicaSet(
-            new DataRegionId(2),
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 2),
             Arrays.asList(
-                new DataNodeLocation(21, new Endpoint("192.0.2.1", 9000)),
-                new DataNodeLocation(22, new Endpoint("192.0.2.2", 9000)))));
-    Map<TimePartitionSlot, List<RegionReplicaSet>> d1DataRegionMap = new HashMap<>();
-    d1DataRegionMap.put(new TimePartitionSlot(), d1DataRegions);
-
-    List<RegionReplicaSet> d2DataRegions = new ArrayList<>();
+                new TDataNodeLocation()
+                    .setDataNodeId(21)
+                    .setExternalEndPoint(new TEndPoint("192.0.2.1", 9000)),
+                new TDataNodeLocation()
+                    .setDataNodeId(22)
+                    .setExternalEndPoint(new TEndPoint("192.0.2.2", 9000)))));
+    Map<TTimePartitionSlot, List<TRegionReplicaSet>> d1DataRegionMap = new HashMap<>();
+    d1DataRegionMap.put(new TTimePartitionSlot(), d1DataRegions);
+
+    List<TRegionReplicaSet> d2DataRegions = new ArrayList<>();
     d2DataRegions.add(
-        new RegionReplicaSet(
-            new DataRegionId(3),
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 3),
             Arrays.asList(
-                new DataNodeLocation(31, new Endpoint("192.0.3.1", 9000)),
-                new DataNodeLocation(32, new Endpoint("192.0.3.2", 9000)))));
-    Map<TimePartitionSlot, List<RegionReplicaSet>> d2DataRegionMap = new HashMap<>();
-    d2DataRegionMap.put(new TimePartitionSlot(), d2DataRegions);
-
-    List<RegionReplicaSet> d3DataRegions = new ArrayList<>();
+                new TDataNodeLocation()
+                    .setDataNodeId(31)
+                    .setExternalEndPoint(new TEndPoint("192.0.3.1", 9000)),
+                new TDataNodeLocation()
+                    .setDataNodeId(32)
+                    .setExternalEndPoint(new TEndPoint("192.0.3.2", 9000)))));
+    Map<TTimePartitionSlot, List<TRegionReplicaSet>> d2DataRegionMap = new HashMap<>();
+    d2DataRegionMap.put(new TTimePartitionSlot(), d2DataRegions);
+
+    List<TRegionReplicaSet> d3DataRegions = new ArrayList<>();
     d3DataRegions.add(
-        new RegionReplicaSet(
-            new DataRegionId(1),
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 1),
             Arrays.asList(
-                new DataNodeLocation(11, new Endpoint("192.0.1.1", 9000)),
-                new DataNodeLocation(12, new Endpoint("192.0.1.2", 9000)))));
+                new TDataNodeLocation()
+                    .setDataNodeId(11)
+                    .setExternalEndPoint(new TEndPoint("192.0.1.1", 9000)),
+                new TDataNodeLocation()
+                    .setDataNodeId(12)
+                    .setExternalEndPoint(new TEndPoint("192.0.1.2", 9000)))));
     d3DataRegions.add(
-        new RegionReplicaSet(
-            new DataRegionId(4),
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 4),
             Arrays.asList(
-                new DataNodeLocation(41, new Endpoint("192.0.4.1", 9000)),
-                new DataNodeLocation(42, new Endpoint("192.0.4.2", 9000)))));
-    Map<TimePartitionSlot, List<RegionReplicaSet>> d3DataRegionMap = new HashMap<>();
-    d3DataRegionMap.put(new TimePartitionSlot(), d3DataRegions);
-
-    sgPartitionMap.put(new SeriesPartitionSlot(device1.length()), d1DataRegionMap);
-    sgPartitionMap.put(new SeriesPartitionSlot(device2.length()), d2DataRegionMap);
-    sgPartitionMap.put(new SeriesPartitionSlot(device3.length()), d3DataRegionMap);
+                new TDataNodeLocation()
+                    .setDataNodeId(41)
+                    .setExternalEndPoint(new TEndPoint("192.0.4.1", 9000)),
+                new TDataNodeLocation()
+                    .setDataNodeId(42)
+                    .setExternalEndPoint(new TEndPoint("192.0.4.2", 9000)))));
+    Map<TTimePartitionSlot, List<TRegionReplicaSet>> d3DataRegionMap = new HashMap<>();
+    d3DataRegionMap.put(new TTimePartitionSlot(), d3DataRegions);
+
+    sgPartitionMap.put(new TSeriesPartitionSlot(device1.length()), d1DataRegionMap);
+    sgPartitionMap.put(new TSeriesPartitionSlot(device2.length()), d2DataRegionMap);
+    sgPartitionMap.put(new TSeriesPartitionSlot(device3.length()), d3DataRegionMap);
 
     dataPartitionMap.put("root.sg", sgPartitionMap);
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/DistributionPlanner.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/DistributionPlanner.java
index 83b5e950d6..b8a42f462b 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/DistributionPlanner.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/DistributionPlanner.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.mpp.common.MPPQueryContext;
 import org.apache.iotdb.db.mpp.common.PlanFragmentId;
 import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
@@ -149,8 +149,8 @@ public class DistributionPlanner {
     public PlanNode visitSchemaMerge(SchemaMergeNode node, DistributionPlanContext context) {
       SchemaMergeNode root = (SchemaMergeNode) node.clone();
       SchemaScanNode seed = (SchemaScanNode) node.getChildren().get(0);
-      TreeSet<RegionReplicaSet> schemaRegions =
-          new TreeSet<>(Comparator.comparingInt(region -> region.getConsensusGroupId().getId()));
+      TreeSet<TRegionReplicaSet> schemaRegions =
+          new TreeSet<>(Comparator.comparingInt(region -> region.getRegionId().getId()));
       analysis
           .getSchemaPartitionInfo()
           .getSchemaPartitionMap()
@@ -177,7 +177,7 @@ public class DistributionPlanner {
     // TODO: (xingtanzjr) a temporary way to resolve the distribution of single SeriesScanNode issue
     @Override
     public PlanNode visitSeriesScan(SeriesScanNode node, DistributionPlanContext context) {
-      List<RegionReplicaSet> dataDistribution =
+      List<TRegionReplicaSet> dataDistribution =
           analysis.getPartitionInfo(node.getSeriesPath(), node.getTimeFilter());
       if (dataDistribution.size() == 1) {
         node.setRegionReplicaSet(dataDistribution.get(0));
@@ -185,7 +185,7 @@ public class DistributionPlanner {
       }
       TimeJoinNode timeJoinNode =
           new TimeJoinNode(context.queryContext.getQueryId().genPlanNodeId(), node.getScanOrder());
-      for (RegionReplicaSet dataRegion : dataDistribution) {
+      for (TRegionReplicaSet dataRegion : dataDistribution) {
         SeriesScanNode split = (SeriesScanNode) node.clone();
         split.setPlanNodeId(context.queryContext.getQueryId().genPlanNodeId());
         split.setRegionReplicaSet(dataRegion);
@@ -206,11 +206,11 @@ public class DistributionPlanner {
           // If the child is SeriesScanNode, we need to check whether this node should be seperated
           // into several splits.
           SeriesScanNode handle = (SeriesScanNode) child;
-          List<RegionReplicaSet> dataDistribution =
+          List<TRegionReplicaSet> dataDistribution =
               analysis.getPartitionInfo(handle.getSeriesPath(), handle.getTimeFilter());
           // If the size of dataDistribution is m, this SeriesScanNode should be seperated into m
           // SeriesScanNode.
-          for (RegionReplicaSet dataRegion : dataDistribution) {
+          for (TRegionReplicaSet dataRegion : dataDistribution) {
             SeriesScanNode split = (SeriesScanNode) handle.clone();
             split.setPlanNodeId(context.queryContext.getQueryId().genPlanNodeId());
             split.setRegionReplicaSet(dataRegion);
@@ -230,7 +230,7 @@ public class DistributionPlanner {
       }
 
       // Step 2: For the source nodes, group them by the DataRegion.
-      Map<RegionReplicaSet, List<SeriesScanNode>> sourceGroup =
+      Map<TRegionReplicaSet, List<SeriesScanNode>> sourceGroup =
           sources.stream().collect(Collectors.groupingBy(SeriesScanNode::getRegionReplicaSet));
       // Step 3: For the source nodes which belong to same data region, add a TimeJoinNode for them
       // and make the
@@ -365,7 +365,7 @@ public class DistributionPlanner {
                 visitedChildren.add(visit(child, context));
               });
 
-      RegionReplicaSet dataRegion = calculateDataRegionByChildren(visitedChildren, context);
+      TRegionReplicaSet dataRegion = calculateDataRegionByChildren(visitedChildren, context);
       NodeDistributionType distributionType =
           nodeDistributionIsSame(visitedChildren, context)
               ? NodeDistributionType.SAME_WITH_ALL_CHILDREN
@@ -395,10 +395,10 @@ public class DistributionPlanner {
       return newNode;
     }
 
-    private RegionReplicaSet calculateDataRegionByChildren(
+    private TRegionReplicaSet calculateDataRegionByChildren(
         List<PlanNode> children, NodeGroupContext context) {
       // Step 1: calculate the count of children group by DataRegion.
-      Map<RegionReplicaSet, Long> groupByRegion =
+      Map<TRegionReplicaSet, Long> groupByRegion =
           children.stream()
               .collect(
                   Collectors.groupingBy(
@@ -408,7 +408,7 @@ public class DistributionPlanner {
       return Collections.max(groupByRegion.entrySet(), Map.Entry.comparingByValue()).getKey();
     }
 
-    private RegionReplicaSet calculateSchemaRegionByChildren(
+    private TRegionReplicaSet calculateSchemaRegionByChildren(
         List<PlanNode> children, NodeGroupContext context) {
       // We always make the schemaRegion of MetaMergeNode to be the same as its first child.
       return context.getNodeDistribution(children.get(0).getPlanNodeId()).region;
@@ -458,9 +458,9 @@ public class DistributionPlanner {
 
   private class NodeDistribution {
     private NodeDistributionType type;
-    private RegionReplicaSet region;
+    private TRegionReplicaSet region;
 
-    private NodeDistribution(NodeDistributionType type, RegionReplicaSet region) {
+    private NodeDistribution(NodeDistributionType type, TRegionReplicaSet region) {
       this.type = type;
       this.region = region;
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/LocalExecutionPlanner.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/LocalExecutionPlanner.java
index e5d60439bd..51e7abc63d 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/LocalExecutionPlanner.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/LocalExecutionPlanner.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.storagegroup.DataRegion;
 import org.apache.iotdb.db.metadata.path.PartialPath;
@@ -321,13 +321,13 @@ public class LocalExecutionPlanner {
               SeriesScanOperator.class.getSimpleName());
       FragmentInstanceId localInstanceId = context.instanceContext.getId();
       FragmentInstanceId remoteInstanceId = node.getUpstreamInstanceId();
-      Endpoint source = node.getUpstreamEndpoint();
+      TEndPoint source = node.getUpstreamEndpoint();
 
       ISourceHandle sourceHandle =
           DATA_BLOCK_MANAGER.createSourceHandle(
               localInstanceId.toThrift(),
               node.getPlanNodeId().getId(),
-              new Endpoint(
+              new TEndPoint(
                   source.getIp(),
                   IoTDBDescriptor.getInstance().getConfig().getDataBlockManagerPort()),
               remoteInstanceId.toThrift());
@@ -337,13 +337,13 @@ public class LocalExecutionPlanner {
     @Override
     public Operator visitFragmentSink(FragmentSinkNode node, LocalExecutionPlanContext context) {
       Operator child = node.getChild().accept(this, context);
-      Endpoint target = node.getDownStreamEndpoint();
+      TEndPoint target = node.getDownStreamEndpoint();
       FragmentInstanceId localInstanceId = context.instanceContext.getId();
       FragmentInstanceId targetInstanceId = node.getDownStreamInstanceId();
       ISinkHandle sinkHandle =
           DATA_BLOCK_MANAGER.createSinkHandle(
               localInstanceId.toThrift(),
-              new Endpoint(
+              new TEndPoint(
                   target.getIp(),
                   IoTDBDescriptor.getInstance().getConfig().getDataBlockManagerPort()),
               targetInstanceId.toThrift(),
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/FragmentInstance.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/FragmentInstance.java
index 7058301ce8..b4a7819589 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/FragmentInstance.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/FragmentInstance.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
 import org.apache.iotdb.consensus.common.request.IConsensusRequest;
 import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
 import org.apache.iotdb.db.mpp.sql.analyze.QueryType;
@@ -41,9 +42,9 @@ public class FragmentInstance implements IConsensusRequest {
   private final PlanFragment fragment;
 
   // The Region where the FragmentInstance should run
-  private RegionReplicaSet regionReplicaSet;
+  private TRegionReplicaSet regionReplicaSet;
 
-  private Endpoint hostEndpoint;
+  private TEndPoint hostEndpoint;
 
   private Filter timeFilter;
 
@@ -58,26 +59,26 @@ public class FragmentInstance implements IConsensusRequest {
     this.type = type;
   }
 
-  public RegionReplicaSet getDataRegionId() {
+  public TRegionReplicaSet getDataRegionId() {
     return regionReplicaSet;
   }
 
-  public void setDataRegionAndHost(RegionReplicaSet regionReplicaSet) {
+  public void setDataRegionAndHost(TRegionReplicaSet regionReplicaSet) {
     this.regionReplicaSet = regionReplicaSet;
     // TODO: (xingtanzjr) We select the first Endpoint as the default target host for current
     // instance
-    this.hostEndpoint = regionReplicaSet.getDataNodeList().get(0).getEndPoint();
+    this.hostEndpoint = regionReplicaSet.getDataNodeLocations().get(0).getConsensusEndPoint();
   }
 
-  public RegionReplicaSet getRegionReplicaSet() {
+  public TRegionReplicaSet getRegionReplicaSet() {
     return regionReplicaSet;
   }
 
-  public void setRegionReplicaSet(RegionReplicaSet regionReplicaSet) {
+  public void setRegionReplicaSet(TRegionReplicaSet regionReplicaSet) {
     this.regionReplicaSet = regionReplicaSet;
   }
 
-  public Endpoint getHostEndpoint() {
+  public TEndPoint getHostEndpoint() {
     return hostEndpoint;
   }
 
@@ -122,9 +123,7 @@ public class FragmentInstance implements IConsensusRequest {
     ret.append(
         String.format(
             "Region: %s",
-            getRegionReplicaSet() == null
-                ? "Not set"
-                : getRegionReplicaSet().getConsensusGroupId()));
+            getRegionReplicaSet() == null ? "Not set" : getRegionReplicaSet().getRegionId()));
     ret.append("---- Plan Node Tree ----\n");
     ret.append(PlanNodeUtil.nodeToString(getFragment().getRoot()));
     return ret.toString();
@@ -138,8 +137,8 @@ public class FragmentInstance implements IConsensusRequest {
     QueryType queryType = QueryType.values()[ReadWriteIOUtils.readInt(buffer)];
     FragmentInstance fragmentInstance =
         new FragmentInstance(planFragment, id, timeFilter, queryType);
-    fragmentInstance.regionReplicaSet = RegionReplicaSet.deserializeImpl(buffer);
-    fragmentInstance.hostEndpoint = Endpoint.deserializeImpl(buffer);
+    fragmentInstance.regionReplicaSet = ThriftCommonsSerDeUtils.readTRegionReplicaSet(buffer);
+    fragmentInstance.hostEndpoint = ThriftCommonsSerDeUtils.readTEndPoint(buffer);
 
     return fragmentInstance;
   }
@@ -153,8 +152,8 @@ public class FragmentInstance implements IConsensusRequest {
       timeFilter.serialize(buffer);
     }
     ReadWriteIOUtils.write(type.ordinal(), buffer);
-    regionReplicaSet.serializeImpl(buffer);
-    hostEndpoint.serializeImpl(buffer);
+    ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet, buffer);
+    ThriftCommonsSerDeUtils.writeTEndPoint(hostEndpoint, buffer);
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/PlanFragment.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/PlanFragment.java
index 38365ec4af..a287884a99 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/PlanFragment.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/PlanFragment.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.mpp.common.PlanFragmentId;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
@@ -61,16 +61,16 @@ public class PlanFragment {
   // In current version, one PlanFragment should contain at least one SourceNode,
   // and the DataRegions of all SourceNodes should be same in one PlanFragment.
   // So we can use the DataRegion of one SourceNode as the PlanFragment's DataRegion.
-  public RegionReplicaSet getTargetRegion() {
+  public TRegionReplicaSet getTargetRegion() {
     return getNodeRegion(root);
   }
 
-  private RegionReplicaSet getNodeRegion(PlanNode root) {
+  private TRegionReplicaSet getNodeRegion(PlanNode root) {
     if (root instanceof SourceNode) {
       return ((SourceNode) root).getRegionReplicaSet();
     }
     for (PlanNode child : root.getChildren()) {
-      RegionReplicaSet result = getNodeRegion(child);
+      TRegionReplicaSet result = getNodeRegion(child);
       if (result != null) {
         return result;
       }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/SimpleFragmentParallelPlanner.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/SimpleFragmentParallelPlanner.java
index 631b217695..f3413df263 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/SimpleFragmentParallelPlanner.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/SimpleFragmentParallelPlanner.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.mpp.common.MPPQueryContext;
 import org.apache.iotdb.db.mpp.common.PlanFragmentId;
 import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
@@ -93,7 +93,7 @@ public class SimpleFragmentParallelPlanner implements IFragmentParallelPlaner {
 
     // Get the target region for origin PlanFragment, then its instance will be distributed one
     // of them.
-    RegionReplicaSet regionReplicaSet = fragment.getTargetRegion();
+    TRegionReplicaSet regionReplicaSet = fragment.getTargetRegion();
 
     // Set DataRegion and target host for the instance
     // We need to store all the replica host in case of the scenario that the instance need to be
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanGraphPrinter.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanGraphPrinter.java
index d74a6c49f1..fbcef0ec72 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanGraphPrinter.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanGraphPrinter.java
@@ -60,7 +60,7 @@ public class PlanGraphPrinter extends PlanVisitor<List<String>, PlanGraphPrinter
     List<String> boxValue = new ArrayList<>();
     boxValue.add(String.format("SeriesScanNode-%s", node.getPlanNodeId().getId()));
     boxValue.add(String.format("Series: %s", node.getSeriesPath()));
-    boxValue.add(String.format("Partition: %s", node.getRegionReplicaSet().getConsensusGroupId()));
+    boxValue.add(String.format("Partition: %s", node.getRegionReplicaSet().getRegionId()));
     return render(node, boxValue, context);
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/WritePlanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/WritePlanNode.java
index c37b8e1d7f..434b8bb9a5 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/WritePlanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/WritePlanNode.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.mpp.sql.planner.plan.node;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
 
 import java.util.List;
@@ -30,7 +30,7 @@ public abstract class WritePlanNode extends PlanNode {
     super(id);
   }
 
-  public abstract RegionReplicaSet getRegionReplicaSet();
+  public abstract TRegionReplicaSet getRegionReplicaSet();
 
   public abstract List<WritePlanNode> splitByPartition(Analysis analysis);
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/read/SchemaScanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/read/SchemaScanNode.java
index 90812b842d..d1605ba8fd 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/read/SchemaScanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/read/SchemaScanNode.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.read;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.source.SourceNode;
@@ -32,7 +32,7 @@ public abstract class SchemaScanNode extends SourceNode {
   private boolean hasLimit;
   protected boolean isPrefixPath;
 
-  private RegionReplicaSet schemaRegionReplicaSet;
+  private TRegionReplicaSet schemaRegionReplicaSet;
 
   protected SchemaScanNode(PlanNodeId id) {
     super(id);
@@ -81,12 +81,12 @@ public abstract class SchemaScanNode extends SourceNode {
   }
 
   @Override
-  public RegionReplicaSet getRegionReplicaSet() {
+  public TRegionReplicaSet getRegionReplicaSet() {
     return schemaRegionReplicaSet;
   }
 
   @Override
-  public void setRegionReplicaSet(RegionReplicaSet schemaRegionReplicaSet) {
+  public void setRegionReplicaSet(TRegionReplicaSet schemaRegionReplicaSet) {
     this.schemaRegionReplicaSet = schemaRegionReplicaSet;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/write/CreateTimeSeriesNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/write/CreateTimeSeriesNode.java
index e5d1959dc3..ddeefcef63 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/write/CreateTimeSeriesNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/write/CreateTimeSeriesNode.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
@@ -54,7 +54,7 @@ public class CreateTimeSeriesNode extends WritePlanNode {
   private Map<String, String> attributes = null;
   private long tagOffset = -1;
 
-  private RegionReplicaSet regionReplicaSet;
+  private TRegionReplicaSet regionReplicaSet;
 
   public CreateTimeSeriesNode(
       PlanNodeId id,
@@ -314,19 +314,19 @@ public class CreateTimeSeriesNode extends WritePlanNode {
   }
 
   @Override
-  public RegionReplicaSet getRegionReplicaSet() {
+  public TRegionReplicaSet getRegionReplicaSet() {
     return regionReplicaSet;
   }
 
   @Override
   public List<WritePlanNode> splitByPartition(Analysis analysis) {
-    RegionReplicaSet regionReplicaSet =
+    TRegionReplicaSet regionReplicaSet =
         analysis.getSchemaPartitionInfo().getSchemaRegionReplicaSet(path.getDevice());
     setRegionReplicaSet(regionReplicaSet);
     return ImmutableList.of(this);
   }
 
-  public void setRegionReplicaSet(RegionReplicaSet regionReplicaSet) {
+  public void setRegionReplicaSet(TRegionReplicaSet regionReplicaSet) {
     this.regionReplicaSet = regionReplicaSet;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/process/ExchangeNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/process/ExchangeNode.java
index d3f82fd328..dc61098d7a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/process/ExchangeNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/process/ExchangeNode.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.process;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
 import org.apache.iotdb.db.mpp.sql.planner.plan.PlanFragment;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
@@ -44,7 +44,7 @@ public class ExchangeNode extends PlanNode {
   // In current version, one ExchangeNode will only have one source.
   // And the fragment which the sourceNode belongs to will only have one instance.
   // Thus, by nodeId and endpoint, the ExchangeNode can know where its source from.
-  private Endpoint upstreamEndpoint;
+  private TEndPoint upstreamEndpoint;
   private FragmentInstanceId upstreamInstanceId;
   private PlanNodeId upstreamPlanNodeId;
 
@@ -86,7 +86,7 @@ public class ExchangeNode extends PlanNode {
     return CHILD_COUNT_NO_LIMIT;
   }
 
-  public void setUpstream(Endpoint endPoint, FragmentInstanceId instanceId, PlanNodeId nodeId) {
+  public void setUpstream(TEndPoint endPoint, FragmentInstanceId instanceId, PlanNodeId nodeId) {
     this.upstreamEndpoint = endPoint;
     this.upstreamInstanceId = instanceId;
     this.upstreamPlanNodeId = nodeId;
@@ -95,8 +95,9 @@ public class ExchangeNode extends PlanNode {
   public static ExchangeNode deserialize(ByteBuffer byteBuffer) {
     FragmentSinkNode fragmentSinkNode =
         (FragmentSinkNode) PlanFragment.deserializeHelper(byteBuffer);
-    Endpoint endPoint =
-        new Endpoint(ReadWriteIOUtils.readString(byteBuffer), ReadWriteIOUtils.readInt(byteBuffer));
+    TEndPoint endPoint =
+        new TEndPoint(
+            ReadWriteIOUtils.readString(byteBuffer), ReadWriteIOUtils.readInt(byteBuffer));
     FragmentInstanceId fragmentInstanceId = FragmentInstanceId.deserialize(byteBuffer);
     PlanNodeId upstreamPlanNodeId = PlanNodeId.deserialize(byteBuffer);
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
@@ -151,7 +152,7 @@ public class ExchangeNode extends PlanNode {
     this.child = null;
   }
 
-  public Endpoint getUpstreamEndpoint() {
+  public TEndPoint getUpstreamEndpoint() {
     return upstreamEndpoint;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/sink/FragmentSinkNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/sink/FragmentSinkNode.java
index a738c8a502..c32b2b2062 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/sink/FragmentSinkNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/sink/FragmentSinkNode.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.sink;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
@@ -36,7 +36,7 @@ import java.util.Objects;
 public class FragmentSinkNode extends SinkNode {
   private PlanNode child;
 
-  private Endpoint downStreamEndpoint;
+  private TEndPoint downStreamEndpoint;
   private FragmentInstanceId downStreamInstanceId;
   private PlanNodeId downStreamPlanNodeId;
 
@@ -84,8 +84,9 @@ public class FragmentSinkNode extends SinkNode {
   }
 
   public static FragmentSinkNode deserialize(ByteBuffer byteBuffer) {
-    Endpoint downStreamEndpoint =
-        new Endpoint(ReadWriteIOUtils.readString(byteBuffer), ReadWriteIOUtils.readInt(byteBuffer));
+    TEndPoint downStreamEndpoint =
+        new TEndPoint(
+            ReadWriteIOUtils.readString(byteBuffer), ReadWriteIOUtils.readInt(byteBuffer));
     FragmentInstanceId downStreamInstanceId = FragmentInstanceId.deserialize(byteBuffer);
     PlanNodeId downStreamPlanNodeId = PlanNodeId.deserialize(byteBuffer);
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
@@ -134,7 +135,7 @@ public class FragmentSinkNode extends SinkNode {
         getDownStreamEndpoint().getIp(), getDownStreamInstanceId(), getDownStreamPlanNodeId());
   }
 
-  public void setDownStream(Endpoint endPoint, FragmentInstanceId instanceId, PlanNodeId nodeId) {
+  public void setDownStream(TEndPoint endPoint, FragmentInstanceId instanceId, PlanNodeId nodeId) {
     this.downStreamEndpoint = endPoint;
     this.downStreamInstanceId = instanceId;
     this.downStreamPlanNodeId = nodeId;
@@ -144,7 +145,7 @@ public class FragmentSinkNode extends SinkNode {
     this.downStreamPlanNodeId = downStreamPlanNodeId;
   }
 
-  public Endpoint getDownStreamEndpoint() {
+  public TEndPoint getDownStreamEndpoint() {
     return downStreamEndpoint;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesAggregateScanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesAggregateScanNode.java
index fde2b903a1..88bb0742bb 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesAggregateScanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesAggregateScanNode.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.source;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.metadata.path.PathDeserializeUtil;
 import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
@@ -90,7 +91,7 @@ public class SeriesAggregateScanNode extends SourceNode implements IOutputPlanNo
   private List<ColumnHeader> columnHeaders;
 
   // The id of DataRegion where the node will run
-  private RegionReplicaSet regionReplicaSet;
+  private TRegionReplicaSet regionReplicaSet;
 
   public SeriesAggregateScanNode(
       PlanNodeId id,
@@ -169,12 +170,12 @@ public class SeriesAggregateScanNode extends SourceNode implements IOutputPlanNo
   public void open() throws Exception {}
 
   @Override
-  public RegionReplicaSet getRegionReplicaSet() {
+  public TRegionReplicaSet getRegionReplicaSet() {
     return this.regionReplicaSet;
   }
 
   @Override
-  public void setRegionReplicaSet(RegionReplicaSet regionReplicaSet) {
+  public void setRegionReplicaSet(TRegionReplicaSet regionReplicaSet) {
     this.regionReplicaSet = regionReplicaSet;
   }
 
@@ -208,7 +209,7 @@ public class SeriesAggregateScanNode extends SourceNode implements IOutputPlanNo
     ReadWriteIOUtils.write(scanOrder.ordinal(), byteBuffer);
     timeFilter.serialize(byteBuffer);
     groupByTimeParameter.serialize(byteBuffer);
-    regionReplicaSet.serializeImpl(byteBuffer);
+    ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet, byteBuffer);
   }
 
   public static SeriesAggregateScanNode deserialize(ByteBuffer byteBuffer) {
@@ -227,7 +228,7 @@ public class SeriesAggregateScanNode extends SourceNode implements IOutputPlanNo
     Filter timeFilter = FilterFactory.deserialize(byteBuffer);
 
     GroupByTimeComponent groupByTimeComponent = GroupByTimeComponent.deserialize(byteBuffer);
-    RegionReplicaSet regionReplicaSet = RegionReplicaSet.deserializeImpl(byteBuffer);
+    TRegionReplicaSet regionReplicaSet = ThriftCommonsSerDeUtils.readTRegionReplicaSet(byteBuffer);
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
     SeriesAggregateScanNode seriesAggregateScanNode =
         new SeriesAggregateScanNode(
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesScanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesScanNode.java
index bf0df051bc..76327e002f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesScanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesScanNode.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.source;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.metadata.path.PathDeserializeUtil;
 import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
@@ -79,7 +80,7 @@ public class SeriesScanNode extends SourceNode implements IOutputPlanNode {
   private ColumnHeader columnHeader;
 
   // The id of DataRegion where the node will run
-  private RegionReplicaSet regionReplicaSet;
+  private TRegionReplicaSet regionReplicaSet;
 
   public SeriesScanNode(PlanNodeId id, PartialPath seriesPath) {
     super(id);
@@ -95,7 +96,7 @@ public class SeriesScanNode extends SourceNode implements IOutputPlanNode {
     this.columnHeader = new ColumnHeader(seriesPath.getFullPath(), seriesPath.getSeriesType());
   }
 
-  public SeriesScanNode(PlanNodeId id, PartialPath seriesPath, RegionReplicaSet regionReplicaSet) {
+  public SeriesScanNode(PlanNodeId id, PartialPath seriesPath, TRegionReplicaSet regionReplicaSet) {
     this(id, seriesPath);
     this.regionReplicaSet = regionReplicaSet;
   }
@@ -115,12 +116,12 @@ public class SeriesScanNode extends SourceNode implements IOutputPlanNode {
   public void open() throws Exception {}
 
   @Override
-  public RegionReplicaSet getRegionReplicaSet() {
+  public TRegionReplicaSet getRegionReplicaSet() {
     return regionReplicaSet;
   }
 
   @Override
-  public void setRegionReplicaSet(RegionReplicaSet dataRegion) {
+  public void setRegionReplicaSet(TRegionReplicaSet dataRegion) {
     this.regionReplicaSet = dataRegion;
   }
 
@@ -209,7 +210,8 @@ public class SeriesScanNode extends SourceNode implements IOutputPlanNode {
     if (isNull == 1) valueFilter = FilterFactory.deserialize(byteBuffer);
     int limit = ReadWriteIOUtils.readInt(byteBuffer);
     int offset = ReadWriteIOUtils.readInt(byteBuffer);
-    RegionReplicaSet dataRegionReplicaSet = RegionReplicaSet.deserializeImpl(byteBuffer);
+    TRegionReplicaSet dataRegionReplicaSet =
+        ThriftCommonsSerDeUtils.readTRegionReplicaSet(byteBuffer);
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
     SeriesScanNode seriesScanNode = new SeriesScanNode(planNodeId, partialPath);
     seriesScanNode.allSensors = allSensors;
@@ -250,7 +252,7 @@ public class SeriesScanNode extends SourceNode implements IOutputPlanNode {
     }
     ReadWriteIOUtils.write(limit, byteBuffer);
     ReadWriteIOUtils.write(offset, byteBuffer);
-    regionReplicaSet.serializeImpl(byteBuffer);
+    ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet, byteBuffer);
   }
 
   public PartialPath getSeriesPath() {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SourceNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SourceNode.java
index 5d674371e9..9fe3e58140 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SourceNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SourceNode.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.source;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
 
@@ -30,7 +30,7 @@ public abstract class SourceNode extends PlanNode implements AutoCloseable {
 
   public abstract void open() throws Exception;
 
-  public abstract RegionReplicaSet getRegionReplicaSet();
+  public abstract TRegionReplicaSet getRegionReplicaSet();
 
-  public abstract void setRegionReplicaSet(RegionReplicaSet regionReplicaSet);
+  public abstract void setRegionReplicaSet(TRegionReplicaSet regionReplicaSet);
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertMultiTabletsNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertMultiTabletsNode.java
index 3e2fd699d5..0aec9e691c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertMultiTabletsNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertMultiTabletsNode.java
@@ -18,8 +18,8 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.write;
 
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
 import org.apache.iotdb.commons.utils.StatusUtils;
 import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
@@ -110,12 +110,12 @@ public class InsertMultiTabletsNode extends InsertNode {
 
   @Override
   public List<WritePlanNode> splitByPartition(Analysis analysis) {
-    Map<RegionReplicaSet, InsertMultiTabletsNode> splitMap = new HashMap<>();
+    Map<TRegionReplicaSet, InsertMultiTabletsNode> splitMap = new HashMap<>();
     for (int i = 0; i < insertTabletNodeList.size(); i++) {
       InsertTabletNode insertTabletNode = insertTabletNodeList.get(i);
       List<WritePlanNode> tmpResult = insertTabletNode.splitByPartition(analysis);
       for (WritePlanNode subNode : tmpResult) {
-        RegionReplicaSet dataRegionReplicaSet = ((InsertNode) subNode).getDataRegionReplicaSet();
+        TRegionReplicaSet dataRegionReplicaSet = ((InsertNode) subNode).getDataRegionReplicaSet();
         if (splitMap.containsKey(dataRegionReplicaSet)) {
           InsertMultiTabletsNode tmpNode = splitMap.get(dataRegionReplicaSet);
           tmpNode.addInsertTabletNode((InsertTabletNode) subNode, i);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertNode.java
index 08c861913d..62cbffb55b 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertNode.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.write;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
@@ -53,7 +53,7 @@ public abstract class InsertNode extends WritePlanNode {
   protected IDeviceID deviceID;
 
   /** Physical address of data region after splitting */
-  RegionReplicaSet dataRegionReplicaSet;
+  TRegionReplicaSet dataRegionReplicaSet;
 
   protected InsertNode(PlanNodeId id) {
     super(id);
@@ -79,11 +79,11 @@ public abstract class InsertNode extends WritePlanNode {
     }
   }
 
-  public RegionReplicaSet getDataRegionReplicaSet() {
+  public TRegionReplicaSet getDataRegionReplicaSet() {
     return dataRegionReplicaSet;
   }
 
-  public void setDataRegionReplicaSet(RegionReplicaSet dataRegionReplicaSet) {
+  public void setDataRegionReplicaSet(TRegionReplicaSet dataRegionReplicaSet) {
     this.dataRegionReplicaSet = dataRegionReplicaSet;
   }
 
@@ -131,7 +131,7 @@ public abstract class InsertNode extends WritePlanNode {
     this.deviceID = deviceID;
   }
 
-  public RegionReplicaSet getRegionReplicaSet() {
+  public TRegionReplicaSet getRegionReplicaSet() {
     return dataRegionReplicaSet;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowNode.java
index e999046ff8..cf144c9d51 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowNode.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.write;
 
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
@@ -76,7 +76,7 @@ public class InsertRowNode extends InsertNode implements WALEntryValue {
 
   @Override
   public List<WritePlanNode> splitByPartition(Analysis analysis) {
-    TimePartitionSlot timePartitionSlot = StorageEngine.getTimePartitionSlot(time);
+    TTimePartitionSlot timePartitionSlot = StorageEngine.getTimePartitionSlot(time);
     this.dataRegionReplicaSet =
         analysis
             .getDataPartitionInfo()
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsNode.java
index b82370ecfa..62fad82b81 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsNode.java
@@ -18,8 +18,8 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.write;
 
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
 import org.apache.iotdb.commons.utils.StatusUtils;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
@@ -132,11 +132,11 @@ public class InsertRowsNode extends InsertNode {
 
   @Override
   public List<WritePlanNode> splitByPartition(Analysis analysis) {
-    Map<RegionReplicaSet, InsertRowsNode> splitMap = new HashMap<>();
+    Map<TRegionReplicaSet, InsertRowsNode> splitMap = new HashMap<>();
     for (int i = 0; i < insertRowNodeList.size(); i++) {
       InsertRowNode insertRowNode = insertRowNodeList.get(i);
       // data region for insert row node
-      RegionReplicaSet dataRegionReplicaSet =
+      TRegionReplicaSet dataRegionReplicaSet =
           analysis
               .getDataPartitionInfo()
               .getDataRegionReplicaSetForWriting(
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsOfOneDeviceNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsOfOneDeviceNode.java
index a20beb3876..f83912efa9 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsOfOneDeviceNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsOfOneDeviceNode.java
@@ -18,8 +18,8 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.write;
 
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
 import org.apache.iotdb.commons.utils.StatusUtils;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
@@ -113,11 +113,11 @@ public class InsertRowsOfOneDeviceNode extends InsertNode {
 
   @Override
   public List<WritePlanNode> splitByPartition(Analysis analysis) {
-    Map<RegionReplicaSet, InsertRowsNode> splitMap = new HashMap<>();
+    Map<TRegionReplicaSet, InsertRowsNode> splitMap = new HashMap<>();
     for (int i = 0; i < insertRowNodeList.size(); i++) {
       InsertRowNode insertRowNode = insertRowNodeList.get(i);
       // data region for insert row node
-      RegionReplicaSet dataRegionReplicaSet =
+      TRegionReplicaSet dataRegionReplicaSet =
           analysis
               .getDataPartitionInfo()
               .getDataRegionReplicaSetForWriting(
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertTabletNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertTabletNode.java
index f53e716680..ef8a2e3b93 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertTabletNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertTabletNode.java
@@ -18,8 +18,8 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.write;
 
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.PartialPath;
@@ -552,10 +552,10 @@ public class InsertTabletNode extends InsertNode implements WALEntryValue {
         (times[0] / StorageEngine.getTimePartitionInterval())
             * StorageEngine.getTimePartitionInterval(); // included
     long endTime = startTime + StorageEngine.getTimePartitionInterval(); // excluded
-    TimePartitionSlot timePartitionSlot = StorageEngine.getTimePartitionSlot(times[0]);
+    TTimePartitionSlot timePartitionSlot = StorageEngine.getTimePartitionSlot(times[0]);
     int startLoc = 0; // included
 
-    List<TimePartitionSlot> timePartitionSlots = new ArrayList<>();
+    List<TTimePartitionSlot> timePartitionSlots = new ArrayList<>();
     // for each List in split, they are range1.start, range1.end, range2.start, range2.end, ...
     List<Integer> ranges = new ArrayList<>();
     for (int i = 1; i < times.length; i++) { // times are sorted in session API.
@@ -580,12 +580,12 @@ public class InsertTabletNode extends InsertNode implements WALEntryValue {
     timePartitionSlots.add(timePartitionSlot);
 
     // data region for each time partition
-    List<RegionReplicaSet> dataRegionReplicaSets =
+    List<TRegionReplicaSet> dataRegionReplicaSets =
         analysis
             .getDataPartitionInfo()
             .getDataRegionReplicaSetForWriting(devicePath.getFullPath(), timePartitionSlots);
 
-    Map<RegionReplicaSet, List<Integer>> splitMap = new HashMap<>();
+    Map<TRegionReplicaSet, List<Integer>> splitMap = new HashMap<>();
     for (int i = 0; i < dataRegionReplicaSets.size(); i++) {
       List<Integer> sub_ranges =
           splitMap.computeIfAbsent(dataRegionReplicaSets.get(i), x -> new ArrayList<>());
@@ -594,7 +594,7 @@ public class InsertTabletNode extends InsertNode implements WALEntryValue {
     }
 
     List<Integer> locs;
-    for (Map.Entry<RegionReplicaSet, List<Integer>> entry : splitMap.entrySet()) {
+    for (Map.Entry<TRegionReplicaSet, List<Integer>> entry : splitMap.entrySet()) {
       // generate a new times and values
       locs = entry.getValue();
       int count = 0;
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowStatement.java
index 80862ab70e..858df6a94e 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowStatement.java
@@ -19,8 +19,8 @@
 
 package org.apache.iotdb.db.mpp.sql.statement.crud;
 
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
@@ -176,7 +176,7 @@ public class InsertRowStatement extends InsertBaseStatement {
     dataTypes[index] = null;
   }
 
-  public List<TimePartitionSlot> getTimePartitionSlots() {
+  public List<TTimePartitionSlot> getTimePartitionSlots() {
     return Collections.singletonList(StorageEngine.getTimePartitionSlot(time));
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowsOfOneDeviceStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowsOfOneDeviceStatement.java
index 2d4079b21c..f83f42c27f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowsOfOneDeviceStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowsOfOneDeviceStatement.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.mpp.sql.statement.crud;
 
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.mpp.common.schematree.SchemaTree;
@@ -61,8 +61,8 @@ public class InsertRowsOfOneDeviceStatement extends InsertBaseStatement {
     dataTypes = measurementsAndDataType.values().toArray(new TSDataType[0]);
   }
 
-  public List<TimePartitionSlot> getTimePartitionSlots() {
-    Set<TimePartitionSlot> timePartitionSlotSet = new HashSet<>();
+  public List<TTimePartitionSlot> getTimePartitionSlots() {
+    Set<TTimePartitionSlot> timePartitionSlotSet = new HashSet<>();
     for (InsertRowStatement insertRowStatement : insertRowStatementList) {
       timePartitionSlotSet.add(StorageEngine.getTimePartitionSlot(insertRowStatement.getTime()));
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertTabletStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertTabletStatement.java
index 23d740315e..12aa64db34 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertTabletStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertTabletStatement.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.statement.crud;
 
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
@@ -82,13 +82,13 @@ public class InsertTabletStatement extends InsertBaseStatement {
     bitMaps[index] = null;
   }
 
-  public List<TimePartitionSlot> getTimePartitionSlots() {
-    List<TimePartitionSlot> result = new ArrayList<>();
+  public List<TTimePartitionSlot> getTimePartitionSlots() {
+    List<TTimePartitionSlot> result = new ArrayList<>();
     long startTime =
         (times[0] / StorageEngine.getTimePartitionInterval())
             * StorageEngine.getTimePartitionInterval(); // included
     long endTime = startTime + StorageEngine.getTimePartitionInterval(); // excluded
-    TimePartitionSlot timePartitionSlot = StorageEngine.getTimePartitionSlot(times[0]);
+    TTimePartitionSlot timePartitionSlot = StorageEngine.getTimePartitionSlot(times[0]);
     for (int i = 1; i < times.length; i++) { // times are sorted in session API.
       if (times[i] >= endTime) {
         result.add(timePartitionSlot);
diff --git a/server/src/main/java/org/apache/iotdb/db/service/DataNode.java b/server/src/main/java/org/apache/iotdb/db/service/DataNode.java
index f2285abbc3..7774b43826 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/DataNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/DataNode.java
@@ -18,8 +18,8 @@
  */
 package org.apache.iotdb.db.service;
 
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
-import org.apache.iotdb.commons.cluster.Endpoint;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.exception.BadNodeUrlException;
 import org.apache.iotdb.commons.exception.ConfigurationException;
@@ -54,7 +54,7 @@ public class DataNode implements DataNodeMBean {
    */
   private static final int DEFAULT_JOIN_RETRY = 10;
 
-  private Endpoint thisNode = new Endpoint();
+  private TEndPoint thisNode = new TEndPoint();
 
   private int dataNodeID;
 
@@ -123,14 +123,25 @@ public class DataNode implements DataNodeMBean {
     while (retry > 0) {
       logger.info("start joining the cluster.");
       try {
-        TDataNodeRegisterResp dataNodeRegisterResp =
-            configNodeClient.registerDataNode(
-                new TDataNodeRegisterReq(new TEndPoint(thisNode.getIp(), thisNode.getPort())));
+        IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+        TDataNodeRegisterReq req = new TDataNodeRegisterReq();
+        TDataNodeLocation location = new TDataNodeLocation();
+        location.setDataNodeId(-1);
+        location.setExternalEndPoint(new TEndPoint(config.getRpcAddress(), config.getRpcPort()));
+        location.setInternalEndPoint(
+            new TEndPoint(config.getInternalIp(), config.getInternalPort()));
+        location.setDataBlockManagerEndPoint(
+            new TEndPoint(config.getInternalIp(), config.getDataBlockManagerPort()));
+        location.setConsensusEndPoint(
+            new TEndPoint(config.getInternalIp(), config.getConsensusPort()));
+        req.setDataNodeLocation(location);
+
+        TDataNodeRegisterResp dataNodeRegisterResp = configNodeClient.registerDataNode(req);
         if (dataNodeRegisterResp.getStatus().getCode()
                 == TSStatusCode.SUCCESS_STATUS.getStatusCode()
             || dataNodeRegisterResp.getStatus().getCode()
                 == TSStatusCode.DATANODE_ALREADY_REGISTERED.getStatusCode()) {
-          dataNodeID = dataNodeRegisterResp.getDataNodeID();
+          dataNodeID = dataNodeRegisterResp.getDataNodeId();
           IoTDBDescriptor.getInstance().loadGlobalConfig(dataNodeRegisterResp.globalConfig);
           logger.info("Joined the cluster successfully");
           return;
diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/InternalServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/InternalServiceImpl.java
index b101e9bd78..eb77ccb6cd 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/InternalServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/InternalServiceImpl.java
@@ -19,13 +19,12 @@
 
 package org.apache.iotdb.db.service.thrift.impl;
 
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.GroupType;
 import org.apache.iotdb.commons.consensus.SchemaRegionId;
 import org.apache.iotdb.consensus.IConsensus;
 import org.apache.iotdb.consensus.common.Peer;
@@ -66,7 +65,6 @@ import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -85,8 +83,7 @@ public class InternalServiceImpl implements InternalService.Iface {
   public TSendFragmentInstanceResp sendFragmentInstance(TSendFragmentInstanceReq req) {
     QueryType type = QueryType.valueOf(req.queryType);
     ConsensusGroupId groupId =
-        ConsensusGroupId.Factory.create(
-            req.consensusGroupId.id, GroupType.valueOf(req.consensusGroupId.type));
+        ConsensusGroupId.Factory.convertFromTConsensusGroupId(req.getConsensusGroupId());
     switch (type) {
       case READ:
         ConsensusReadResponse readResp =
@@ -146,16 +143,14 @@ public class InternalServiceImpl implements InternalService.Iface {
     try {
       PartialPath storageGroupPartitionPath = new PartialPath(req.getStorageGroup());
       TRegionReplicaSet regionReplicaSet = req.getRegionReplicaSet();
-      SchemaRegionId schemaRegionId =
-          (SchemaRegionId)
-              ConsensusGroupId.Factory.create(ByteBuffer.wrap(regionReplicaSet.getRegionId()));
-      LOGGER.info("SchemaRegionId: " + schemaRegionId.getId());
+      SchemaRegionId schemaRegionId = new SchemaRegionId(regionReplicaSet.getRegionId().getId());
       schemaEngine.createSchemaRegion(storageGroupPartitionPath, schemaRegionId);
       List<Peer> peers = new ArrayList<>();
-      for (TEndPoint endPoint : regionReplicaSet.getEndpoint()) {
-        Endpoint endpoint = new Endpoint(endPoint.getIp(), endPoint.getPort());
-        // TODO: Expend Peer and RegisterDataNodeReq
-        endpoint.setPort(endpoint.getPort() + 31007);
+      for (TDataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeLocations()) {
+        TEndPoint endpoint =
+            new TEndPoint(
+                dataNodeLocation.getConsensusEndPoint().getIp(),
+                dataNodeLocation.getConsensusEndPoint().getPort());
         peers.add(new Peer(schemaRegionId, endpoint));
       }
       ConsensusGenericResponse consensusGenericResponse =
@@ -186,16 +181,14 @@ public class InternalServiceImpl implements InternalService.Iface {
     TSStatus tsStatus;
     try {
       TRegionReplicaSet regionReplicaSet = req.getRegionReplicaSet();
-      DataRegionId dataRegionId =
-          (DataRegionId)
-              ConsensusGroupId.Factory.create(ByteBuffer.wrap(regionReplicaSet.getRegionId()));
-      LOGGER.info("DataRegionId: " + dataRegionId.getId());
+      DataRegionId dataRegionId = new DataRegionId(regionReplicaSet.getRegionId().getId());
       storageEngine.createDataRegion(dataRegionId, req.storageGroup, req.ttl);
       List<Peer> peers = new ArrayList<>();
-      for (TEndPoint endPoint : regionReplicaSet.getEndpoint()) {
-        Endpoint endpoint = new Endpoint(endPoint.getIp(), endPoint.getPort());
-        // TODO: Expend Peer and RegisterDataNodeReq
-        endpoint.setPort(endpoint.getPort() + 31007);
+      for (TDataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeLocations()) {
+        TEndPoint endpoint =
+            new TEndPoint(
+                dataNodeLocation.getConsensusEndPoint().getIp(),
+                dataNodeLocation.getConsensusEndPoint().getPort());
         peers.add(new Peer(dataRegionId, endpoint));
       }
       ConsensusGenericResponse consensusGenericResponse =
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/DistributionPlannerTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/DistributionPlannerTest.java
index ee4d5dedc9..08d80476cb 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/DistributionPlannerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/DistributionPlannerTest.java
@@ -19,15 +19,15 @@
 
 package org.apache.iotdb.db.mpp.sql.plan;
 
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.SchemaRegionId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.commons.partition.DataPartition;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
 import org.apache.iotdb.commons.partition.SchemaPartition;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
 import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
@@ -80,7 +80,7 @@ public class DistributionPlannerTest {
 
     Analysis analysis = constructAnalysis();
 
-    MPPQueryContext context = new MPPQueryContext("", queryId, null, new Endpoint());
+    MPPQueryContext context = new MPPQueryContext("", queryId, null, new TEndPoint());
     DistributionPlanner planner =
         new DistributionPlanner(analysis, new LogicalQueryPlan(context, root));
     DistributedQueryPlan plan = planner.planFragments();
@@ -100,7 +100,7 @@ public class DistributionPlannerTest {
 
     Analysis analysis = constructAnalysis();
 
-    MPPQueryContext context = new MPPQueryContext("", queryId, null, new Endpoint());
+    MPPQueryContext context = new MPPQueryContext("", queryId, null, new TEndPoint());
     DistributionPlanner planner =
         new DistributionPlanner(analysis, new LogicalQueryPlan(context, root));
     PlanNode rootAfterRewrite = planner.rewriteSource();
@@ -257,7 +257,7 @@ public class DistributionPlannerTest {
 
     Analysis analysis = constructAnalysis();
 
-    MPPQueryContext context = new MPPQueryContext("", queryId, null, new Endpoint());
+    MPPQueryContext context = new MPPQueryContext("", queryId, null, new TEndPoint());
     DistributionPlanner planner =
         new DistributionPlanner(analysis, new LogicalQueryPlan(context, root));
     PlanNode rootAfterRewrite = planner.rewriteSource();
@@ -294,7 +294,7 @@ public class DistributionPlannerTest {
 
     Analysis analysis = constructAnalysis();
 
-    MPPQueryContext context = new MPPQueryContext("", queryId, null, new Endpoint());
+    MPPQueryContext context = new MPPQueryContext("", queryId, null, new TEndPoint());
     DistributionPlanner planner =
         new DistributionPlanner(analysis, new LogicalQueryPlan(context, root));
     DistributedQueryPlan plan = planner.planFragments();
@@ -318,7 +318,7 @@ public class DistributionPlannerTest {
 
     Analysis analysis = constructAnalysis();
 
-    MPPQueryContext context = new MPPQueryContext("", queryId, null, new Endpoint());
+    MPPQueryContext context = new MPPQueryContext("", queryId, null, new TEndPoint());
     context.setQueryType(QueryType.WRITE);
     DistributionPlanner planner =
         new DistributionPlanner(analysis, new LogicalQueryPlan(context, insertRowNode));
@@ -360,7 +360,7 @@ public class DistributionPlannerTest {
 
     Analysis analysis = constructAnalysis();
 
-    MPPQueryContext context = new MPPQueryContext("", queryId, null, new Endpoint());
+    MPPQueryContext context = new MPPQueryContext("", queryId, null, new TEndPoint());
     context.setQueryType(QueryType.WRITE);
     DistributionPlanner planner =
         new DistributionPlanner(analysis, new LogicalQueryPlan(context, node));
@@ -385,56 +385,73 @@ public class DistributionPlannerTest {
         new DataPartition(
             IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionExecutorClass(),
             IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionSlotNum());
-    Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+
+    Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
         dataPartitionMap = new HashMap<>();
-    Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>> sgPartitionMap =
+    Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>> sgPartitionMap =
         new HashMap<>();
 
-    List<RegionReplicaSet> d1DataRegion1 = new ArrayList<>();
-    d1DataRegion1.add(
-        new RegionReplicaSet(
-            new DataRegionId(1),
+    List<TRegionReplicaSet> d1DataRegions = new ArrayList<>();
+    d1DataRegions.add(
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 1),
             Arrays.asList(
-                new DataNodeLocation(11, new Endpoint("192.0.1.1", 9000)),
-                new DataNodeLocation(12, new Endpoint("192.0.1.2", 9000)))));
-
-    List<RegionReplicaSet> d1DataRegion2 = new ArrayList<>();
-    d1DataRegion2.add(
-        new RegionReplicaSet(
-            new DataRegionId(2),
+                new TDataNodeLocation()
+                    .setDataNodeId(11)
+                    .setExternalEndPoint(new TEndPoint("192.0.1.1", 9000)),
+                new TDataNodeLocation()
+                    .setDataNodeId(12)
+                    .setExternalEndPoint(new TEndPoint("192.0.1.2", 9000)))));
+    d1DataRegions.add(
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 2),
             Arrays.asList(
-                new DataNodeLocation(21, new Endpoint("192.0.2.1", 9000)),
-                new DataNodeLocation(22, new Endpoint("192.0.2.2", 9000)))));
-
-    Map<TimePartitionSlot, List<RegionReplicaSet>> d1DataRegionMap = new HashMap<>();
-    d1DataRegionMap.put(new TimePartitionSlot(0), d1DataRegion1);
-    d1DataRegionMap.put(new TimePartitionSlot(1), d1DataRegion2);
-
-    List<RegionReplicaSet> d2DataRegions = new ArrayList<>();
+                new TDataNodeLocation()
+                    .setDataNodeId(21)
+                    .setExternalEndPoint(new TEndPoint("192.0.2.1", 9000)),
+                new TDataNodeLocation()
+                    .setDataNodeId(22)
+                    .setExternalEndPoint(new TEndPoint("192.0.2.2", 9000)))));
+    Map<TTimePartitionSlot, List<TRegionReplicaSet>> d1DataRegionMap = new HashMap<>();
+    d1DataRegionMap.put(new TTimePartitionSlot(), d1DataRegions);
+
+    List<TRegionReplicaSet> d2DataRegions = new ArrayList<>();
     d2DataRegions.add(
-        new RegionReplicaSet(
-            new DataRegionId(3),
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 3),
             Arrays.asList(
-                new DataNodeLocation(31, new Endpoint("192.0.3.1", 9000)),
-                new DataNodeLocation(32, new Endpoint("192.0.3.2", 9000)))));
-    Map<TimePartitionSlot, List<RegionReplicaSet>> d2DataRegionMap = new HashMap<>();
-    d2DataRegionMap.put(new TimePartitionSlot(0), d2DataRegions);
-
-    List<RegionReplicaSet> d3DataRegions = new ArrayList<>();
+                new TDataNodeLocation()
+                    .setDataNodeId(31)
+                    .setExternalEndPoint(new TEndPoint("192.0.3.1", 9000)),
+                new TDataNodeLocation()
+                    .setDataNodeId(32)
+                    .setExternalEndPoint(new TEndPoint("192.0.3.2", 9000)))));
+    Map<TTimePartitionSlot, List<TRegionReplicaSet>> d2DataRegionMap = new HashMap<>();
+    d2DataRegionMap.put(new TTimePartitionSlot(), d2DataRegions);
+
+    List<TRegionReplicaSet> d3DataRegions = new ArrayList<>();
     d3DataRegions.add(
-        new RegionReplicaSet(
-            new DataRegionId(1),
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 1),
             Arrays.asList(
-                new DataNodeLocation(11, new Endpoint("192.0.1.1", 9000)),
-                new DataNodeLocation(12, new Endpoint("192.0.1.2", 9000)))));
+                new TDataNodeLocation()
+                    .setDataNodeId(11)
+                    .setExternalEndPoint(new TEndPoint("192.0.1.1", 9000)),
+                new TDataNodeLocation()
+                    .setDataNodeId(12)
+                    .setExternalEndPoint(new TEndPoint("192.0.1.2", 9000)))));
     d3DataRegions.add(
-        new RegionReplicaSet(
-            new DataRegionId(4),
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 4),
             Arrays.asList(
-                new DataNodeLocation(41, new Endpoint("192.0.4.1", 9000)),
-                new DataNodeLocation(42, new Endpoint("192.0.4.2", 9000)))));
-    Map<TimePartitionSlot, List<RegionReplicaSet>> d3DataRegionMap = new HashMap<>();
-    d3DataRegionMap.put(new TimePartitionSlot(0), d3DataRegions);
+                new TDataNodeLocation()
+                    .setDataNodeId(41)
+                    .setExternalEndPoint(new TEndPoint("192.0.4.1", 9000)),
+                new TDataNodeLocation()
+                    .setDataNodeId(42)
+                    .setExternalEndPoint(new TEndPoint("192.0.4.2", 9000)))));
+    Map<TTimePartitionSlot, List<TRegionReplicaSet>> d3DataRegionMap = new HashMap<>();
+    d3DataRegionMap.put(new TTimePartitionSlot(), d3DataRegions);
 
     sgPartitionMap.put(executor.getSeriesPartitionSlot(device1), d1DataRegionMap);
     sgPartitionMap.put(executor.getSeriesPartitionSlot(device2), d2DataRegionMap);
@@ -451,22 +468,30 @@ public class DistributionPlannerTest {
         new SchemaPartition(
             IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionExecutorClass(),
             IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionSlotNum());
-    Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> schemaPartitionMap = new HashMap<>();
+    Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap = new HashMap<>();
 
-    RegionReplicaSet schemaRegion1 =
-        new RegionReplicaSet(
-            new SchemaRegionId(11),
+    TRegionReplicaSet schemaRegion1 =
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 11),
             Arrays.asList(
-                new DataNodeLocation(11, new Endpoint("192.0.1.1", 9000)),
-                new DataNodeLocation(12, new Endpoint("192.0.1.2", 9000))));
-    Map<SeriesPartitionSlot, RegionReplicaSet> schemaRegionMap = new HashMap<>();
-
-    RegionReplicaSet schemaRegion2 =
-        new RegionReplicaSet(
-            new SchemaRegionId(21),
+                new TDataNodeLocation()
+                    .setDataNodeId(11)
+                    .setExternalEndPoint(new TEndPoint("192.0.1.1", 9000)),
+                new TDataNodeLocation()
+                    .setDataNodeId(12)
+                    .setExternalEndPoint(new TEndPoint("192.0.1.2", 9000))));
+    Map<TSeriesPartitionSlot, TRegionReplicaSet> schemaRegionMap = new HashMap<>();
+
+    TRegionReplicaSet schemaRegion2 =
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 21),
             Arrays.asList(
-                new DataNodeLocation(21, new Endpoint("192.0.1.1", 9000)),
-                new DataNodeLocation(22, new Endpoint("192.0.1.2", 9000))));
+                new TDataNodeLocation()
+                    .setDataNodeId(21)
+                    .setExternalEndPoint(new TEndPoint("192.0.1.1", 9000)),
+                new TDataNodeLocation()
+                    .setDataNodeId(22)
+                    .setExternalEndPoint(new TEndPoint("192.0.1.2", 9000))));
 
     schemaRegionMap.put(executor.getSeriesPartitionSlot(device1), schemaRegion1);
     schemaRegionMap.put(executor.getSeriesPartitionSlot(device2), schemaRegion2);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/FragmentInstanceSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/FragmentInstanceSerdeTest.java
index 4bf6a477e2..5bf598cf14 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/FragmentInstanceSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/FragmentInstanceSerdeTest.java
@@ -18,10 +18,11 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan;
 
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
 import org.apache.iotdb.db.mpp.common.PlanFragmentId;
@@ -56,6 +57,13 @@ public class FragmentInstanceSerdeTest {
 
   @Test
   public void TestSerializeAndDeserializeForTree1() throws IllegalPathException {
+    TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+    dataNodeLocation.setDataNodeId(0);
+    dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+    dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+    dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+    dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010));
+
     PlanFragmentId planFragmentId = new PlanFragmentId("test", -1);
     FragmentInstance fragmentInstance =
         new FragmentInstance(
@@ -63,10 +71,10 @@ public class FragmentInstanceSerdeTest {
             planFragmentId.genFragmentInstanceId(),
             new GroupByFilter(1, 2, 3, 4),
             QueryType.READ);
-    RegionReplicaSet regionReplicaSet =
-        new RegionReplicaSet(
-            new DataRegionId(1),
-            ImmutableList.of(new DataNodeLocation(0, new Endpoint("127.0.0.1", 6666))));
+    TRegionReplicaSet regionReplicaSet =
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 1),
+            ImmutableList.of(dataNodeLocation));
     fragmentInstance.setDataRegionAndHost(regionReplicaSet);
 
     ByteBuffer byteBuffer = ByteBuffer.allocate(1024);
@@ -78,6 +86,13 @@ public class FragmentInstanceSerdeTest {
 
   @Test
   public void TestSerializeAndDeserializeWithNullFilter() throws IllegalPathException {
+    TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+    dataNodeLocation.setDataNodeId(0);
+    dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+    dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+    dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+    dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010));
+
     PlanFragmentId planFragmentId = new PlanFragmentId("test2", 1);
     FragmentInstance fragmentInstance =
         new FragmentInstance(
@@ -85,10 +100,10 @@ public class FragmentInstanceSerdeTest {
             planFragmentId.genFragmentInstanceId(),
             null,
             QueryType.READ);
-    RegionReplicaSet regionReplicaSet =
-        new RegionReplicaSet(
-            new DataRegionId(1),
-            ImmutableList.of(new DataNodeLocation(0, new Endpoint("127.0.0.2", 6667))));
+    TRegionReplicaSet regionReplicaSet =
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 1),
+            ImmutableList.of(dataNodeLocation));
     fragmentInstance.setDataRegionAndHost(regionReplicaSet);
 
     ByteBuffer byteBuffer = ByteBuffer.allocate(1024);
@@ -125,17 +140,20 @@ public class FragmentInstanceSerdeTest {
     SeriesScanNode seriesScanNode1 =
         new SeriesScanNode(new PlanNodeId("SeriesScanNode1"), new MeasurementPath("root.sg.d1.s2"));
     seriesScanNode1.setRegionReplicaSet(
-        new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     seriesScanNode1.setScanOrder(OrderBy.TIMESTAMP_DESC);
     SeriesScanNode seriesScanNode2 =
         new SeriesScanNode(new PlanNodeId("SeriesScanNode2"), new MeasurementPath("root.sg.d2.s1"));
     seriesScanNode2.setRegionReplicaSet(
-        new RegionReplicaSet(new DataRegionId(2), new ArrayList<>()));
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 2), new ArrayList<>()));
     seriesScanNode2.setScanOrder(OrderBy.TIMESTAMP_DESC);
     SeriesScanNode seriesScanNode3 =
         new SeriesScanNode(new PlanNodeId("SeriesScanNode3"), new MeasurementPath("root.sg.d2.s2"));
     seriesScanNode3.setRegionReplicaSet(
-        new RegionReplicaSet(new DataRegionId(3), new ArrayList<>()));
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 3), new ArrayList<>()));
     seriesScanNode3.setScanOrder(OrderBy.TIMESTAMP_DESC);
 
     // build tree
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/QueryPlannerTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/QueryPlannerTest.java
index a0b28041d4..f10596f775 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/QueryPlannerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/QueryPlannerTest.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.mpp.sql.plan;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.db.mpp.common.MPPQueryContext;
 import org.apache.iotdb.db.mpp.common.QueryId;
@@ -50,7 +50,8 @@ public class QueryPlannerTest {
     QueryExecution queryExecution =
         new QueryExecution(
             stmt,
-            new MPPQueryContext(querySql, new QueryId("query1"), new SessionInfo(), new Endpoint()),
+            new MPPQueryContext(
+                querySql, new QueryId("query1"), new SessionInfo(), new TEndPoint()),
             IoTDBThreadPoolFactory.newSingleThreadExecutor("test_query"),
             IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("test_query_scheduled"),
             new FakePartitionFetcherImpl(),
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java
index eadfb2258c..185ba5a4bc 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.metadata.read;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
@@ -56,13 +56,13 @@ public class DeviceSchemaScanNodeSerdeTest {
     FragmentSinkNode fragmentSinkNode = new FragmentSinkNode(new PlanNodeId("fragmentSink"));
     fragmentSinkNode.addChild(devicesSchemaScanNode);
     fragmentSinkNode.setDownStream(
-        new Endpoint("127.0.0.1", 6667),
+        new TEndPoint("127.0.0.1", 6667),
         new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
         new PlanNodeId("test"));
     exchangeNode.addChild(schemaMergeNode);
     exchangeNode.setRemoteSourceNode(fragmentSinkNode);
     exchangeNode.setUpstream(
-        new Endpoint("127.0.0.1", 6667),
+        new TEndPoint("127.0.0.1", 6667),
         new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
         new PlanNodeId("test"));
     offsetNode.addChild(exchangeNode);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java
index 3680c12251..e3c49ffa92 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.metadata.read;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
@@ -59,13 +59,13 @@ public class TimeSeriesSchemaScanNodeSerdeTest {
     FragmentSinkNode fragmentSinkNode = new FragmentSinkNode(new PlanNodeId("fragmentSink"));
     fragmentSinkNode.addChild(timeSeriesSchemaScanNode);
     fragmentSinkNode.setDownStream(
-        new Endpoint("127.0.0.1", 6667),
+        new TEndPoint("127.0.0.1", 6667),
         new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
         new PlanNodeId("test"));
     exchangeNode.addChild(schemaMergeNode);
     exchangeNode.setRemoteSourceNode(fragmentSinkNode);
     exchangeNode.setUpstream(
-        new Endpoint("127.0.0.1", 6667),
+        new TEndPoint("127.0.0.1", 6667),
         new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
         new PlanNodeId("test"));
     offsetNode.addChild(exchangeNode);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/AggregateNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/AggregateNodeSerdeTest.java
index ab20e99e8e..4f1c52508d 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/AggregateNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/AggregateNodeSerdeTest.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.process;
 
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.AlignedPath;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -57,7 +58,8 @@ public class AggregateNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new AlignedPath("s1"),
-            new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+            new TRegionReplicaSet(
+                new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     aggregateNode.addChild(seriesScanNode);
     ByteBuffer byteBuffer = ByteBuffer.allocate(1024);
     aggregateNode.serialize(byteBuffer);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/DeviceMergeNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/DeviceMergeNodeSerdeTest.java
index 66463f1aa5..dc8646a993 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/DeviceMergeNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/DeviceMergeNodeSerdeTest.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.process;
 
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.AlignedPath;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -59,7 +60,8 @@ public class DeviceMergeNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new AlignedPath("s1"),
-            new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+            new TRegionReplicaSet(
+                new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     aggregateNode.addChild(seriesScanNode);
 
     DeviceMergeNode deviceMergeNode =
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/ExchangeNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/ExchangeNodeSerdeTest.java
index 3dc47fb4f5..76563e29b9 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/ExchangeNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/ExchangeNodeSerdeTest.java
@@ -18,9 +18,10 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.process;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.AlignedPath;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -65,7 +66,8 @@ public class ExchangeNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new AlignedPath("s1"),
-            new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+            new TRegionReplicaSet(
+                new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     aggregateNode.addChild(seriesScanNode);
 
     DeviceMergeNode deviceMergeNode =
@@ -91,14 +93,14 @@ public class ExchangeNodeSerdeTest {
     FragmentSinkNode fragmentSinkNode =
         new FragmentSinkNode(new PlanNodeId("TestFragmentSinkNode"));
     fragmentSinkNode.setDownStream(
-        new Endpoint("127.0.0.1", 6666),
+        new TEndPoint("127.0.0.1", 6666),
         new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
         new PlanNodeId("test"));
     fragmentSinkNode.addChild(seriesScanNode);
     exchangeNode.setRemoteSourceNode(fragmentSinkNode);
     exchangeNode.addChild(deviceMergeNode);
     exchangeNode.setUpstream(
-        new Endpoint("127.0.0.1", 6666),
+        new TEndPoint("127.0.0.1", 6666),
         new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
         new PlanNodeId("test"));
 
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FillNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FillNodeSerdeTest.java
index c282d26626..df3a0f3d95 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FillNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FillNodeSerdeTest.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.process;
 
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.AlignedPath;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -69,7 +70,8 @@ public class FillNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new AlignedPath("s1"),
-            new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+            new TRegionReplicaSet(
+                new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     aggregateNode.addChild(seriesScanNode);
     deviceMergeNode.addChildDeviceNode("device", aggregateNode);
 
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNodeSerdeTest.java
index 74f416e2c1..b33fe9088b 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNodeSerdeTest.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.process;
 
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.AlignedPath;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -80,7 +81,8 @@ public class FilterNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new AlignedPath("s1"),
-            new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+            new TRegionReplicaSet(
+                new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     aggregateNode.addChild(seriesScanNode);
     deviceMergeNode.addChildDeviceNode("device", aggregateNode);
 
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNullNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNullNodeSerdeTest.java
index 9740b0d530..f34d1ae761 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNullNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNullNodeSerdeTest.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.process;
 
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.AlignedPath;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -82,7 +83,8 @@ public class FilterNullNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new AlignedPath("s1"),
-            new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+            new TRegionReplicaSet(
+                new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     aggregateNode.addChild(seriesScanNode);
     deviceMergeNode.addChildDeviceNode("device", aggregateNode);
 
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/GroupByLevelNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/GroupByLevelNodeSerdeTest.java
index cdbd5015cc..8f4dc9a71a 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/GroupByLevelNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/GroupByLevelNodeSerdeTest.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.process;
 
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.AlignedPath;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -84,7 +85,8 @@ public class GroupByLevelNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new AlignedPath("s1"),
-            new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+            new TRegionReplicaSet(
+                new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     aggregateNode.addChild(seriesScanNode);
     deviceMergeNode.addChildDeviceNode("device", aggregateNode);
 
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/LimitNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/LimitNodeSerdeTest.java
index c693aa735e..2e6e84f4b1 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/LimitNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/LimitNodeSerdeTest.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.process;
 
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.AlignedPath;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -85,7 +86,8 @@ public class LimitNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new AlignedPath("s1"),
-            new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+            new TRegionReplicaSet(
+                new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     aggregateNode.addChild(seriesScanNode);
     deviceMergeNode.addChildDeviceNode("device", aggregateNode);
 
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/OffsetNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/OffsetNodeSerdeTest.java
index eb2f482b99..5cfa8dc015 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/OffsetNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/OffsetNodeSerdeTest.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.process;
 
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.AlignedPath;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -91,7 +92,8 @@ public class OffsetNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new AlignedPath("s1"),
-            new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+            new TRegionReplicaSet(
+                new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     aggregateNode.addChild(seriesScanNode);
     deviceMergeNode.addChildDeviceNode("device", aggregateNode);
 
@@ -161,17 +163,20 @@ public class OffsetNodeSerdeTest {
     SeriesScanNode seriesScanNode1 =
         new SeriesScanNode(new PlanNodeId("SeriesScanNode1"), new MeasurementPath("root.sg.d1.s2"));
     seriesScanNode1.setRegionReplicaSet(
-        new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     seriesScanNode1.setScanOrder(OrderBy.TIMESTAMP_DESC);
     SeriesScanNode seriesScanNode2 =
         new SeriesScanNode(new PlanNodeId("SeriesScanNode2"), new MeasurementPath("root.sg.d2.s1"));
     seriesScanNode2.setRegionReplicaSet(
-        new RegionReplicaSet(new DataRegionId(2), new ArrayList<>()));
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 2), new ArrayList<>()));
     seriesScanNode2.setScanOrder(OrderBy.TIMESTAMP_DESC);
     SeriesScanNode seriesScanNode3 =
         new SeriesScanNode(new PlanNodeId("SeriesScanNode3"), new MeasurementPath("root.sg.d2.s2"));
     seriesScanNode3.setRegionReplicaSet(
-        new RegionReplicaSet(new DataRegionId(3), new ArrayList<>()));
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 3), new ArrayList<>()));
     seriesScanNode3.setScanOrder(OrderBy.TIMESTAMP_DESC);
 
     // build tree
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/SortNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/SortNodeSerdeTest.java
index d2385f117a..98f733b41d 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/SortNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/SortNodeSerdeTest.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.process;
 
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.AlignedPath;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -89,7 +90,8 @@ public class SortNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new AlignedPath("s1"),
-            new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+            new TRegionReplicaSet(
+                new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     aggregateNode.addChild(seriesScanNode);
     deviceMergeNode.addChildDeviceNode("device", aggregateNode);
 
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/TimeJoinNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/TimeJoinNodeSerdeTest.java
index 9c99507611..c98ddb2ef2 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/TimeJoinNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/TimeJoinNodeSerdeTest.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.process;
 
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.AlignedPath;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -90,7 +91,8 @@ public class TimeJoinNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new AlignedPath("s1"),
-            new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+            new TRegionReplicaSet(
+                new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     aggregateNode.addChild(seriesScanNode);
     deviceMergeNode.addChildDeviceNode("device", aggregateNode);
 
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/sink/FragmentSinkNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/sink/FragmentSinkNodeSerdeTest.java
index a2804b8c2e..cde85f4e97 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/sink/FragmentSinkNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/sink/FragmentSinkNodeSerdeTest.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.sink;
 
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
@@ -44,7 +44,7 @@ public class FragmentSinkNodeSerdeTest {
         new DevicesSchemaScanNode(
             new PlanNodeId("deviceSchema"), new PartialPath("root.sg.device"), 0, 0, false, false));
     fragmentSinkNode.setDownStream(
-        new Endpoint("127.0.0.1", 6666),
+        new TEndPoint("127.0.0.1", 6666),
         new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
         new PlanNodeId("test"));
 
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesAggregateScanNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesAggregateScanNodeSerdeTest.java
index 67fb28feab..258968d6da 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesAggregateScanNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesAggregateScanNodeSerdeTest.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.source;
 
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -64,7 +65,8 @@ public class SeriesAggregateScanNodeSerdeTest {
             new In<String>(st, VALUE_FILTER, true),
             groupByTimeComponent);
     seriesAggregateScanNode.setRegionReplicaSet(
-        new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+        new TRegionReplicaSet(
+            new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
 
     ByteBuffer byteBuffer = ByteBuffer.allocate(2048);
     seriesAggregateScanNode.serialize(byteBuffer);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesScanNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesScanNodeSerdeTest.java
index 44e93c9629..8ddc7ebf5b 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesScanNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesScanNodeSerdeTest.java
@@ -18,8 +18,9 @@
  */
 package org.apache.iotdb.db.mpp.sql.plan.node.source;
 
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.path.AlignedPath;
@@ -44,7 +45,8 @@ public class SeriesScanNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new AlignedPath("s1"),
-            new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+            new TRegionReplicaSet(
+                new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
     seriesScanNode.setTimeFilter(new GroupByFilter(1, 2, 3, 4));
     seriesScanNode.setScanOrder(OrderBy.TIMESTAMP_ASC);
     ByteBuffer byteBuffer = ByteBuffer.allocate(2048);
diff --git a/server/src/test/java/org/apache/iotdb/db/service/InternalServiceImplTest.java b/server/src/test/java/org/apache/iotdb/db/service/InternalServiceImplTest.java
index 5def9d7f05..bf0dbad703 100644
--- a/server/src/test/java/org/apache/iotdb/db/service/InternalServiceImplTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/service/InternalServiceImplTest.java
@@ -19,10 +19,13 @@
 
 package org.apache.iotdb.db.service;
 
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.SchemaRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
 import org.apache.iotdb.consensus.common.Peer;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -40,7 +43,6 @@ import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.CreateAligne
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.CreateTimeSeriesNode;
 import org.apache.iotdb.db.service.thrift.impl.InternalServiceImpl;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.mpp.rpc.thrift.TConsensusGroupId;
 import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstance;
 import org.apache.iotdb.mpp.rpc.thrift.TSendFragmentInstanceReq;
 import org.apache.iotdb.mpp.rpc.thrift.TSendFragmentInstanceResp;
@@ -80,16 +82,20 @@ public class InternalServiceImplTest {
 
   @Before
   public void setUp() throws Exception {
-    RegionReplicaSet regionReplicaSet = genRegionReplicaSet();
+    TRegionReplicaSet regionReplicaSet = genRegionReplicaSet();
     ConsensusImpl.getInstance()
-        .addConsensusGroup(regionReplicaSet.getConsensusGroupId(), genPeerList(regionReplicaSet));
+        .addConsensusGroup(
+            ConsensusGroupId.Factory.convertFromTConsensusGroupId(regionReplicaSet.getRegionId()),
+            genPeerList(regionReplicaSet));
     internalServiceImpl = new InternalServiceImpl();
   }
 
   @After
   public void tearDown() throws Exception {
-    RegionReplicaSet regionReplicaSet = genRegionReplicaSet();
-    ConsensusImpl.getInstance().removeConsensusGroup(regionReplicaSet.getConsensusGroupId());
+    TRegionReplicaSet regionReplicaSet = genRegionReplicaSet();
+    ConsensusImpl.getInstance()
+        .removeConsensusGroup(
+            ConsensusGroupId.Factory.convertFromTConsensusGroupId(regionReplicaSet.getRegionId()));
     FileUtils.deleteFully(new File(conf.getConsensusDir()));
   }
 
@@ -128,7 +134,7 @@ public class InternalServiceImplTest {
             },
             "meter1");
 
-    RegionReplicaSet regionReplicaSet = genRegionReplicaSet();
+    TRegionReplicaSet regionReplicaSet = genRegionReplicaSet();
     PlanFragment planFragment = new PlanFragment(new PlanFragmentId("2", 3), createTimeSeriesNode);
     FragmentInstance fragmentInstance =
         new FragmentInstance(
@@ -148,10 +154,7 @@ public class InternalServiceImplTest {
     TFragmentInstance tFragmentInstance = new TFragmentInstance();
     tFragmentInstance.setBody(byteBuffer);
     request.setFragmentInstance(tFragmentInstance);
-    request.setConsensusGroupId(
-        new TConsensusGroupId(
-            regionReplicaSet.getConsensusGroupId().getId(),
-            regionReplicaSet.getConsensusGroupId().getType().toString()));
+    request.setConsensusGroupId(regionReplicaSet.getRegionId());
     request.setQueryType(QueryType.WRITE.toString());
 
     // Use consensus layer to execute request
@@ -219,7 +222,7 @@ public class InternalServiceImplTest {
               }
             });
 
-    RegionReplicaSet regionReplicaSet = genRegionReplicaSet();
+    TRegionReplicaSet regionReplicaSet = genRegionReplicaSet();
     PlanFragment planFragment =
         new PlanFragment(new PlanFragmentId("2", 3), createAlignedTimeSeriesNode);
     FragmentInstance fragmentInstance =
@@ -240,10 +243,7 @@ public class InternalServiceImplTest {
     TFragmentInstance tFragmentInstance = new TFragmentInstance();
     tFragmentInstance.setBody(byteBuffer);
     request.setFragmentInstance(tFragmentInstance);
-    request.setConsensusGroupId(
-        new TConsensusGroupId(
-            regionReplicaSet.getConsensusGroupId().getId(),
-            regionReplicaSet.getConsensusGroupId().getType().toString()));
+    request.setConsensusGroupId(regionReplicaSet.getRegionId());
     request.setQueryType(QueryType.WRITE.toString());
 
     // Use consensus layer to execute request
@@ -252,21 +252,24 @@ public class InternalServiceImplTest {
     Assert.assertTrue(response.accepted);
   }
 
-  private RegionReplicaSet genRegionReplicaSet() {
-    List<DataNodeLocation> dataNodeList = new ArrayList<>();
+  private TRegionReplicaSet genRegionReplicaSet() {
+    List<TDataNodeLocation> dataNodeList = new ArrayList<>();
     dataNodeList.add(
-        new DataNodeLocation(
-            conf.getConsensusPort(), new Endpoint(conf.getInternalIp(), conf.getConsensusPort())));
+        new TDataNodeLocation()
+            .setConsensusEndPoint(new TEndPoint(conf.getInternalIp(), conf.getConsensusPort())));
 
     // construct fragmentInstance
-    SchemaRegionId schemaRegionId = new SchemaRegionId(0);
-    return new RegionReplicaSet(schemaRegionId, dataNodeList);
+    return new TRegionReplicaSet(
+        new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 0), dataNodeList);
   }
 
-  private List<Peer> genPeerList(RegionReplicaSet regionReplicaSet) {
+  private List<Peer> genPeerList(TRegionReplicaSet regionReplicaSet) {
     List<Peer> peerList = new ArrayList<>();
-    for (DataNodeLocation node : regionReplicaSet.getDataNodeList()) {
-      peerList.add(new Peer(regionReplicaSet.getConsensusGroupId(), node.getEndPoint()));
+    for (TDataNodeLocation node : regionReplicaSet.getDataNodeLocations()) {
+      peerList.add(
+          new Peer(
+              new SchemaRegionId(regionReplicaSet.getRegionId().getId()),
+              node.getConsensusEndPoint()));
     }
     return peerList;
   }
diff --git a/thrift-cluster/pom.xml b/thrift-cluster/pom.xml
index 236417bd87..ff6aeca0ed 100644
--- a/thrift-cluster/pom.xml
+++ b/thrift-cluster/pom.xml
@@ -40,6 +40,11 @@
             <artifactId>iotdb-thrift</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift-commons</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/thrift-cluster/pom.xml b/thrift-commons/pom.xml
similarity index 87%
copy from thrift-cluster/pom.xml
copy to thrift-commons/pom.xml
index 236417bd87..942b35df8a 100644
--- a/thrift-cluster/pom.xml
+++ b/thrift-commons/pom.xml
@@ -27,19 +27,14 @@
         <version>0.14.0-SNAPSHOT</version>
         <relativePath>../pom.xml</relativePath>
     </parent>
-    <artifactId>iotdb-thrift-cluster</artifactId>
-    <name>rpc-thrift-cluster</name>
-    <description>RPC (Thrift) framework among servers.</description>
+    <artifactId>iotdb-thrift-commons</artifactId>
+    <name>rpc-thrift-commons</name>
+    <description>RPC (Thrift) common framework.</description>
     <dependencies>
         <dependency>
             <groupId>org.apache.thrift</groupId>
             <artifactId>libthrift</artifactId>
         </dependency>
-        <dependency>
-            <groupId>org.apache.iotdb</groupId>
-            <artifactId>iotdb-thrift</artifactId>
-            <version>${project.version}</version>
-        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/thrift/src/main/thrift/common.thrift b/thrift-commons/src/main/thrift/common.thrift
similarity index 65%
rename from thrift/src/main/thrift/common.thrift
rename to thrift-commons/src/main/thrift/common.thrift
index 0d20e7d6f3..9b09bc3743 100644
--- a/thrift/src/main/thrift/common.thrift
+++ b/thrift-commons/src/main/thrift/common.thrift
@@ -33,9 +33,15 @@ struct TSStatus {
   4: optional TEndPoint redirectNode
 }
 
-struct TRegionReplicaSet {
-  1: required binary regionId
-  2: required list<TEndPoint> endpoint
+enum TConsensusGroupType {
+  PartitionRegion,
+  DataRegion,
+  SchemaRegion
+}
+
+struct TConsensusGroupId {
+  1: required TConsensusGroupType type
+  2: required i32 id
 }
 
 struct TSeriesPartitionSlot {
@@ -44,4 +50,21 @@ struct TSeriesPartitionSlot {
 
 struct TTimePartitionSlot {
   1: required i64 startTime
+}
+
+struct TRegionReplicaSet {
+  1: required TConsensusGroupId regionId
+  2: required list<TDataNodeLocation> dataNodeLocations
+}
+
+struct TDataNodeLocation {
+  1: required i32 dataNodeId
+  // TEndPoint for DataNode's external rpc
+  2: required TEndPoint externalEndPoint
+  // TEndPoint for DataNode's internal rpc
+  3: required TEndPoint internalEndPoint
+  // TEndPoint for transfering data between DataNodes
+  4: required TEndPoint dataBlockManagerEndPoint
+  // TEndPoint for DataNode's ConsensusLayer
+  5: required TEndPoint consensusEndPoint
 }
\ No newline at end of file
diff --git a/thrift-confignode/pom.xml b/thrift-confignode/pom.xml
index a3501f8095..fb3f5e4467 100644
--- a/thrift-confignode/pom.xml
+++ b/thrift-confignode/pom.xml
@@ -37,7 +37,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.iotdb</groupId>
-            <artifactId>iotdb-thrift</artifactId>
+            <artifactId>iotdb-thrift-commons</artifactId>
             <version>${project.version}</version>
         </dependency>
     </dependencies>
diff --git a/thrift-confignode/src/main/thrift/confignode.thrift b/thrift-confignode/src/main/thrift/confignode.thrift
index ac4d8d9d96..7ac920f073 100644
--- a/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/thrift-confignode/src/main/thrift/confignode.thrift
@@ -23,7 +23,7 @@ namespace py iotdb.thrift.confignode
 
 // DataNode
 struct TDataNodeRegisterReq {
-  1: required common.TEndPoint endPoint
+  1: required common.TDataNodeLocation dataNodeLocation
   // Map<StorageGroupName, TStorageGroupSchema>
   // DataNode can use statusMap to report its status to the ConfigNode when restart
   2: optional map<string, TStorageGroupSchema> statusMap
@@ -38,25 +38,19 @@ struct TGlobalConfig {
 
 struct TDataNodeRegisterResp {
   1: required common.TSStatus status
-  2: optional i32 dataNodeID
+  2: optional i32 dataNodeId
   3: optional TGlobalConfig globalConfig
 }
 
-struct TDataNodeMessageResp {
+struct TDataNodeLocationResp {
   1: required common.TSStatus status
-  // map<DataNodeId, DataNodeMessage>
-  2: optional map<i32, TDataNodeMessage> dataNodeMessageMap
-}
-
-struct TDataNodeMessage {
-  1: required i32 dataNodeId
-  2: required common.TEndPoint endPoint
+  // map<DataNodeId, DataNodeLocation>
+  2: optional map<i32, common.TDataNodeLocation> dataNodeLocationMap
 }
 
 // StorageGroup
 struct TSetStorageGroupReq {
-  1: required string storageGroup
-  2: optional i64 TTL
+  1: required TStorageGroupSchema storageGroup
 }
 
 struct TDeleteStorageGroupReq {
@@ -68,6 +62,11 @@ struct TSetTTLReq {
   2: required i64 TTL
 }
 
+struct TSetTimePartitionIntervalReq {
+  1: required string storageGroup
+  2: required i64 timePartitionInterval
+}
+
 struct TStorageGroupSchemaResp {
   1: required common.TSStatus status
   // map<string, StorageGroupMessage>
@@ -75,12 +74,13 @@ struct TStorageGroupSchemaResp {
 }
 
 struct TStorageGroupSchema {
-  1: required string storageGroup
+  1: required string name
   2: optional i64 TTL
-  // list<DataRegionId>
-  3: optional list<binary> dataRegionGroupIds
-  // list<SchemaRegionId>
-  4: optional list<binary> schemaRegionGroupIds
+  3: optional i32 schemaReplicationFactor
+  4: optional i32 dataReplicationFactor
+  5: optional i64 timePartitionInterval
+  6: optional list<common.TConsensusGroupId> dataRegionGroupIds
+  7: optional list<common.TConsensusGroupId> schemaRegionGroupIds
 }
 
 // Schema
@@ -128,7 +128,7 @@ service ConfigIService {
 
   TDataNodeRegisterResp registerDataNode(TDataNodeRegisterReq req)
 
-  TDataNodeMessageResp getDataNodesMessage(i32 dataNodeID)
+  TDataNodeLocationResp getDataNodeLocations(i32 dataNodeId)
 
   /* StorageGroup */
 
@@ -138,6 +138,8 @@ service ConfigIService {
 
   common.TSStatus setTTL(TSetTTLReq req)
 
+  common.TSStatus setTimePartitionInterval(TSetTimePartitionIntervalReq req)
+
   TStorageGroupSchemaResp getStorageGroupsSchema()
 
   /* Schema */
diff --git a/thrift/pom.xml b/thrift/pom.xml
index 59adc6b3f9..92306b53a1 100644
--- a/thrift/pom.xml
+++ b/thrift/pom.xml
@@ -35,6 +35,11 @@
             <groupId>org.apache.thrift</groupId>
             <artifactId>libthrift</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift-commons</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/thrift/src/main/thrift/mpp.thrift b/thrift/src/main/thrift/mpp.thrift
index d7339e4100..0c2acf6cb2 100644
--- a/thrift/src/main/thrift/mpp.thrift
+++ b/thrift/src/main/thrift/mpp.thrift
@@ -83,14 +83,9 @@ struct TFragmentInstance {
   1: required binary body
 }
 
-struct TConsensusGroupId {
-  1: required i32 id
-  2: required string type
-}
-
 struct TSendFragmentInstanceReq {
   1: required TFragmentInstance fragmentInstance
-  2: required TConsensusGroupId consensusGroupId
+  2: required common.TConsensusGroupId consensusGroupId
   3: required string queryType
 }