You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by zy...@apache.org on 2023/03/07 00:35:07 UTC

[iotdb] branch rc/1.1.0 updated (697fd43565 -> 32f9db0108)

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

zyk pushed a change to branch rc/1.1.0
in repository https://gitbox.apache.org/repos/asf/iotdb.git


    from 697fd43565 [IOTDB-5563] Extract and decouple the logic of window segmentation in Aggregator (#9207)
     new 449e86d3ec [IOTDB-5596] Rename ConfigNodeRegion to ConfigRegion (#9168) (#9215)
     new f7d535584b [IOTDB-5613] Remove unnecessary serialization in IoTConsensus when replicaNum is 1 to improve write performance (#9204) (#9216)
     new 32f9db0108 [To rel/1.1][IOTDB-5620] Fix flush stuck when there is a lot of time partitions in each DataRegion (#9225)

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../iotdb/confignode/conf/ConfigNodeConfig.java    | 10 +--
 ...eMachine.java => ConfigRegionStateMachine.java} | 89 ++++++++-----------
 .../iotdb/confignode/manager/ConfigManager.java    |  6 +-
 .../manager/consensus/ConsensusManager.java        | 20 ++---
 .../partition/DatabasePartitionTable.java          |  2 +-
 .../procedure/env/ConfigNodeProcedureEnv.java      |  2 +-
 .../procedure/env/DataNodeRemoveHandler.java       |  2 +-
 .../thrift/ConfigNodeRPCServiceProcessor.java      |  3 +-
 .../request/ConfigPhysicalPlanSerDeTest.java       |  4 +-
 consensus/README.md                                |  2 +-
 .../common/request/IndexedConsensusRequest.java    | 19 +++--
 .../consensus/iot/IoTConsensusServerImpl.java      |  5 +-
 .../consensus/iot/logdispatcher/LogDispatcher.java | 33 +++++---
 .../consensus/iot/util/FakeConsensusReqReader.java |  4 +-
 .../apache/iotdb/consensus/ratis/UtilsTest.java    |  4 +-
 .../consensus/simple/SimpleConsensusTest.java      |  4 +-
 .../iotdb/it/env/cluster/MppCommonConfig.java      |  8 +-
 .../it/env/cluster/MppSharedCommonConfig.java      | 13 ++-
 .../iotdb/it/env/remote/RemoteCommonConfig.java    |  7 +-
 .../org/apache/iotdb/itbase/env/CommonConfig.java  |  4 +-
 .../confignode/it/IoTDBSnapshotTransferIT.java     |  4 +-
 .../it/load/IoTDBConfigNodeSwitchLeaderIT.java     |  2 +-
 .../it/partition/IoTDBPartitionGetterIT.java       |  6 +-
 ...ionIT.java => IoTDBInsertMultiPartitionIT.java} | 41 ++++-----
 ...ConfigNodeRegionId.java => ConfigRegionId.java} |  6 +-
 .../iotdb/commons/consensus/ConsensusGroupId.java  |  8 +-
 .../commons/utils/ThriftCommonsSerDeUtilsTest.java |  2 +-
 .../iotdb/db/auth/ClusterAuthorityFetcher.java     | 14 +--
 .../apache/iotdb/db/client/ConfigNodeClient.java   | 22 ++---
 .../iotdb/db/client/ConfigNodeClientManager.java   |  8 +-
 .../org/apache/iotdb/db/client/ConfigNodeInfo.java |  8 +-
 .../iotdb/db/client/DataNodeClientPoolFactory.java | 14 +--
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |  1 -
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |  2 +-
 .../engine/storagegroup/TimePartitionManager.java  |  9 +-
 .../metadata/template/ClusterTemplateManager.java  | 16 ++--
 .../iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java  |  2 +-
 .../mpp/plan/analyze/ClusterPartitionFetcher.java  | 18 ++--
 .../db/mpp/plan/analyze/cache/PartitionCache.java  | 10 +--
 .../config/executor/ClusterConfigTaskExecutor.java | 99 +++++++++++-----------
 .../java/org/apache/iotdb/db/service/DataNode.java | 10 +--
 .../db/service/DataNodeServerCommandLine.java      |  6 +-
 .../apache/iotdb/db/service/IoTDBShutdownHook.java |  2 +-
 .../iotdb/db/service/RegionMigrateService.java     |  2 +-
 .../db/sync/common/ClusterSyncInfoFetcher.java     | 12 +--
 .../db/trigger/executor/TriggerFireVisitor.java    |  8 +-
 .../trigger/service/TriggerInformationUpdater.java |  8 +-
 thrift-commons/src/main/thrift/common.thrift       |  2 +-
 48 files changed, 300 insertions(+), 283 deletions(-)
 rename confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/{ConfigNodeRegionStateMachine.java => ConfigRegionStateMachine.java} (93%)
 copy integration-test/src/test/java/org/apache/iotdb/db/it/{aligned/IoTDBLastQueryWithoutLastCacheWithDeletionIT.java => IoTDBInsertMultiPartitionIT.java} (73%)
 rename node-commons/src/main/java/org/apache/iotdb/commons/consensus/{ConfigNodeRegionId.java => ConfigRegionId.java} (87%)


[iotdb] 02/03: [IOTDB-5613] Remove unnecessary serialization in IoTConsensus when replicaNum is 1 to improve write performance (#9204) (#9216)

Posted by zy...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zyk pushed a commit to branch rc/1.1.0
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit f7d535584b616880f04b6eb77c8a7cc5fb3c0f42
Author: Potato <ta...@apache.org>
AuthorDate: Mon Mar 6 14:19:56 2023 +0800

    [IOTDB-5613] Remove unnecessary serialization in IoTConsensus when replicaNum is 1 to improve write performance (#9204) (#9216)
    
    * finish
    
    
    
    * remove unused code
    
    
    
    * add wal & UT buildSerializedRequest
    
    
    
    * fix UT
    
    
    
    ---------
    
    Signed-off-by: OneSizeFitQuorum <ta...@apache.org>
---
 .../common/request/IndexedConsensusRequest.java    | 19 ++++++++-----
 .../consensus/iot/IoTConsensusServerImpl.java      |  5 ++--
 .../consensus/iot/logdispatcher/LogDispatcher.java | 33 +++++++++++++---------
 .../consensus/iot/util/FakeConsensusReqReader.java |  4 ++-
 4 files changed, 36 insertions(+), 25 deletions(-)

diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IndexedConsensusRequest.java b/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IndexedConsensusRequest.java
index 0be89ce1be..58789dbd0e 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IndexedConsensusRequest.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IndexedConsensusRequest.java
@@ -32,19 +32,14 @@ public class IndexedConsensusRequest implements IConsensusRequest {
 
   private final long syncIndex;
   private final List<IConsensusRequest> requests;
-  private final List<ByteBuffer> serializedRequests = new ArrayList<>();
+  private final List<ByteBuffer> serializedRequests;
   private long serializedSize = 0;
 
   public IndexedConsensusRequest(long searchIndex, List<IConsensusRequest> requests) {
     this.searchIndex = searchIndex;
     this.requests = requests;
     this.syncIndex = -1L;
-    this.requests.forEach(
-        r -> {
-          ByteBuffer buffer = r.serializeToByteBuffer();
-          this.serializedRequests.add(buffer);
-          this.serializedSize += buffer.capacity();
-        });
+    this.serializedRequests = new ArrayList<>(requests.size());
   }
 
   public IndexedConsensusRequest(
@@ -52,6 +47,16 @@ public class IndexedConsensusRequest implements IConsensusRequest {
     this.searchIndex = searchIndex;
     this.requests = requests;
     this.syncIndex = syncIndex;
+    this.serializedRequests = new ArrayList<>(requests.size());
+  }
+
+  public void buildSerializedRequests() {
+    this.requests.forEach(
+        r -> {
+          ByteBuffer buffer = r.serializeToByteBuffer();
+          this.serializedRequests.add(buffer);
+          this.serializedSize += buffer.capacity();
+        });
   }
 
   @Override
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensusServerImpl.java b/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensusServerImpl.java
index 2ab9746a31..16e378cde4 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensusServerImpl.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensusServerImpl.java
@@ -199,8 +199,6 @@ public class IoTConsensusServerImpl {
         }
       }
       long writeToStateMachineStartTime = System.nanoTime();
-      IndexedConsensusRequest indexedConsensusRequest =
-          buildIndexedConsensusRequestForLocalRequest(request);
       // statistic the time of checking write block
       MetricService.getInstance()
           .getOrCreateHistogram(
@@ -213,6 +211,8 @@ public class IoTConsensusServerImpl {
               Tag.REGION.toString(),
               this.consensusGroupId)
           .update(writeToStateMachineStartTime - getStateMachineLockTime);
+      IndexedConsensusRequest indexedConsensusRequest =
+          buildIndexedConsensusRequestForLocalRequest(request);
       if (indexedConsensusRequest.getSearchIndex() % 1000 == 0) {
         logger.info(
             "DataRegion[{}]: index after build: safeIndex:{}, searchIndex: {}",
@@ -220,7 +220,6 @@ public class IoTConsensusServerImpl {
             getCurrentSafelyDeletedSearchIndex(),
             indexedConsensusRequest.getSearchIndex());
       }
-      // TODO wal and memtable
       IConsensusRequest planNode = stateMachine.deserializeRequest(indexedConsensusRequest);
       TSStatus result = stateMachine.write(planNode);
       long writeToStateMachineEndTime = System.nanoTime();
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/iot/logdispatcher/LogDispatcher.java b/consensus/src/main/java/org/apache/iotdb/consensus/iot/logdispatcher/LogDispatcher.java
index 68a988558e..f6da0efd52 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/iot/logdispatcher/LogDispatcher.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/iot/logdispatcher/LogDispatcher.java
@@ -156,22 +156,26 @@ public class LogDispatcher {
   }
 
   public void offer(IndexedConsensusRequest request) {
-    synchronized (this) {
-      threads.forEach(
-          thread -> {
-            logger.debug(
-                "{}->{}: Push a log to the queue, where the queue length is {}",
-                impl.getThisNode().getGroupId(),
-                thread.getPeer().getEndpoint().getIp(),
-                thread.getPendingEntriesSize());
-            if (!thread.offer(request)) {
+    // we don't need to serialize and offer request when replicaNum is 1.
+    if (!threads.isEmpty()) {
+      request.buildSerializedRequests();
+      synchronized (this) {
+        threads.forEach(
+            thread -> {
               logger.debug(
-                  "{}: Log queue of {} is full, ignore the log to this node, searchIndex: {}",
+                  "{}->{}: Push a log to the queue, where the queue length is {}",
                   impl.getThisNode().getGroupId(),
-                  thread.getPeer(),
-                  request.getSearchIndex());
-            }
-          });
+                  thread.getPeer().getEndpoint().getIp(),
+                  thread.getPendingEntriesSize());
+              if (!thread.offer(request)) {
+                logger.debug(
+                    "{}: Log queue of {} is full, ignore the log to this node, searchIndex: {}",
+                    impl.getThisNode().getGroupId(),
+                    thread.getPeer(),
+                    request.getSearchIndex());
+              }
+            });
+      }
     }
   }
 
@@ -510,6 +514,7 @@ public class LogDispatcher {
           }
         }
         targetIndex = data.getSearchIndex() + 1;
+        data.buildSerializedRequests();
         // construct request from wal
         logBatches.addTLogEntry(
             new TLogEntry(data.getSerializedRequests(), data.getSearchIndex(), true));
diff --git a/consensus/src/test/java/org/apache/iotdb/consensus/iot/util/FakeConsensusReqReader.java b/consensus/src/test/java/org/apache/iotdb/consensus/iot/util/FakeConsensusReqReader.java
index c2560201a5..3a8c8e4e8c 100644
--- a/consensus/src/test/java/org/apache/iotdb/consensus/iot/util/FakeConsensusReqReader.java
+++ b/consensus/src/test/java/org/apache/iotdb/consensus/iot/util/FakeConsensusReqReader.java
@@ -53,6 +53,7 @@ public class FakeConsensusReqReader implements ConsensusReqReader, DataSet {
   }
 
   private class FakeConsensusReqIterator implements ConsensusReqReader.ReqIterator {
+
     private long nextSearchIndex;
 
     public FakeConsensusReqIterator(long startIndex) {
@@ -70,7 +71,8 @@ public class FakeConsensusReqReader implements ConsensusReqReader, DataSet {
         for (IndexedConsensusRequest indexedConsensusRequest : requestSets.getRequestSet()) {
           if (indexedConsensusRequest.getSearchIndex() == nextSearchIndex) {
             nextSearchIndex++;
-            return indexedConsensusRequest;
+            return new IndexedConsensusRequest(
+                indexedConsensusRequest.getSearchIndex(), indexedConsensusRequest.getRequests());
           }
         }
         return null;


[iotdb] 03/03: [To rel/1.1][IOTDB-5620] Fix flush stuck when there is a lot of time partitions in each DataRegion (#9225)

Posted by zy...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zyk pushed a commit to branch rc/1.1.0
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 32f9db010848cbbc251b1fecbc68f4b3e5404420
Author: Haonan <hh...@outlook.com>
AuthorDate: Mon Mar 6 20:42:15 2023 +0800

    [To rel/1.1][IOTDB-5620] Fix flush stuck when there is a lot of time partitions in each DataRegion (#9225)
---
 .../iotdb/it/env/cluster/MppCommonConfig.java      |  6 ++
 .../it/env/cluster/MppSharedCommonConfig.java      |  7 +++
 .../iotdb/it/env/remote/RemoteCommonConfig.java    |  5 ++
 .../org/apache/iotdb/itbase/env/CommonConfig.java  |  2 +
 .../iotdb/db/it/IoTDBInsertMultiPartitionIT.java   | 71 ++++++++++++++++++++++
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |  1 -
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |  2 +-
 .../engine/storagegroup/TimePartitionManager.java  |  9 ++-
 8 files changed, 100 insertions(+), 3 deletions(-)

diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java
index c1bcbde4ec..fe9f3a6767 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java
@@ -334,4 +334,10 @@ public class MppCommonConfig extends MppBaseConfig implements CommonConfig {
     setProperty("schema_memory_allocate_proportion", String.valueOf(schemaMemoryAllocate));
     return this;
   }
+
+  @Override
+  public CommonConfig setWriteMemoryProportion(String writeMemoryProportion) {
+    setProperty("write_memory_proportion", writeMemoryProportion);
+    return this;
+  }
 }
diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java
index ddaba5a331..275ed1f8f8 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java
@@ -343,4 +343,11 @@ public class MppSharedCommonConfig implements CommonConfig {
     cnConfig.setSchemaMemoryAllocate(schemaMemoryAllocate);
     return this;
   }
+
+  @Override
+  public CommonConfig setWriteMemoryProportion(String writeMemoryProportion) {
+    dnConfig.setWriteMemoryProportion(writeMemoryProportion);
+    cnConfig.setWriteMemoryProportion(writeMemoryProportion);
+    return this;
+  }
 }
diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java
index 440b38253e..526df5dbab 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java
@@ -246,4 +246,9 @@ public class RemoteCommonConfig implements CommonConfig {
   public CommonConfig setSchemaMemoryAllocate(String schemaMemoryAllocate) {
     return this;
   }
+
+  @Override
+  public CommonConfig setWriteMemoryProportion(String writeMemoryProportion) {
+    return this;
+  }
 }
diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
index 59dcd778ab..4792b8b3e7 100644
--- a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
@@ -110,4 +110,6 @@ public interface CommonConfig {
   CommonConfig setSeriesSlotNum(int seriesSlotNum);
 
   CommonConfig setSchemaMemoryAllocate(String schemaMemoryAllocate);
+
+  CommonConfig setWriteMemoryProportion(String writeMemoryProportion);
 }
diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBInsertMultiPartitionIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBInsertMultiPartitionIT.java
new file mode 100644
index 0000000000..fac1175f74
--- /dev/null
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBInsertMultiPartitionIT.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.it;
+
+import org.apache.iotdb.it.env.EnvFactory;
+import org.apache.iotdb.it.framework.IoTDBTestRunner;
+import org.apache.iotdb.itbase.category.ClusterIT;
+import org.apache.iotdb.itbase.category.LocalStandaloneIT;
+import org.apache.iotdb.itbase.category.RemoteIT;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.sql.Connection;
+import java.sql.Statement;
+
+import static org.junit.Assert.fail;
+
+@RunWith(IoTDBTestRunner.class)
+@Category({LocalStandaloneIT.class, ClusterIT.class, RemoteIT.class})
+public class IoTDBInsertMultiPartitionIT {
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    EnvFactory.getEnv()
+        .getConfig()
+        .getDataNodeCommonConfig()
+        .setWriteMemoryProportion("10000000:1");
+    EnvFactory.getEnv().initClusterEnvironment();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    EnvFactory.getEnv().cleanClusterEnvironment();
+  }
+
+  @Test
+  public void testInsertMultiPartition() {
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      statement.execute("insert into root.sg.d1(time,s1) values(1,2)");
+      statement.execute("flush");
+      statement.execute("insert into root.sg.d1(time,s1) values(2,2)");
+      statement.execute("insert into root.sg.d1(time,s1) values(604800001,2)");
+      statement.execute("flush");
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index 75a461b006..70a0b0f3da 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -1785,7 +1785,6 @@ public class IoTDBConfig {
 
   public void setAllocateMemoryForStorageEngine(long allocateMemoryForStorageEngine) {
     this.allocateMemoryForStorageEngine = allocateMemoryForStorageEngine;
-    this.allocateMemoryForTimePartitionInfo = allocateMemoryForStorageEngine * 50 / 1001;
   }
 
   public long getAllocateMemoryForSchema() {
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index a4c40333ab..0432754ab7 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -1612,7 +1612,7 @@ public class IoTDBDescriptor {
     double memtableProportionForWrite =
         ((double) (proportionForMemTable)
             / (double) (proportionForMemTable + proportionForTimePartitionInfo));
-    Double.parseDouble(properties.getProperty("flush_time_memory_proportion", "0.05"));
+
     double timePartitionInfoForWrite =
         ((double) (proportionForTimePartitionInfo)
             / (double) (proportionForMemTable + proportionForTimePartitionInfo));
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TimePartitionManager.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TimePartitionManager.java
index 1f77b36b82..640f8dfb95 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TimePartitionManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TimePartitionManager.java
@@ -24,6 +24,9 @@ import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.util.HashMap;
 import java.util.Map;
 import java.util.TreeMap;
@@ -31,6 +34,7 @@ import java.util.TreeSet;
 
 /** Manage all the time partitions for all data regions and control the total memory of them */
 public class TimePartitionManager {
+  private static final Logger logger = LoggerFactory.getLogger(TimePartitionManager.class);
   final Map<DataRegionId, Map<Long, TimePartitionInfo>> timePartitionInfoMap;
 
   long memCost = 0;
@@ -102,7 +106,10 @@ public class TimePartitionManager {
       }
 
       while (memCost > timePartitionInfoMemoryThreshold) {
-        TimePartitionInfo timePartitionInfo = treeSet.first();
+        TimePartitionInfo timePartitionInfo = treeSet.pollFirst();
+        if (timePartitionInfo == null) {
+          return;
+        }
         memCost -= timePartitionInfo.memSize;
         DataRegion dataRegion =
             StorageEngine.getInstance().getDataRegion(timePartitionInfo.dataRegionId);


[iotdb] 01/03: [IOTDB-5596] Rename ConfigNodeRegion to ConfigRegion (#9168) (#9215)

Posted by zy...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zyk pushed a commit to branch rc/1.1.0
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 449e86d3ec906a2b75917951a422f2dbb006d048
Author: Potato <ta...@apache.org>
AuthorDate: Mon Mar 6 14:19:21 2023 +0800

    [IOTDB-5596] Rename ConfigNodeRegion to ConfigRegion (#9168) (#9215)
---
 .../iotdb/confignode/conf/ConfigNodeConfig.java    | 10 +--
 ...eMachine.java => ConfigRegionStateMachine.java} | 89 ++++++++-----------
 .../iotdb/confignode/manager/ConfigManager.java    |  6 +-
 .../manager/consensus/ConsensusManager.java        | 20 ++---
 .../partition/DatabasePartitionTable.java          |  2 +-
 .../procedure/env/ConfigNodeProcedureEnv.java      |  2 +-
 .../procedure/env/DataNodeRemoveHandler.java       |  2 +-
 .../thrift/ConfigNodeRPCServiceProcessor.java      |  3 +-
 .../request/ConfigPhysicalPlanSerDeTest.java       |  4 +-
 consensus/README.md                                |  2 +-
 .../apache/iotdb/consensus/ratis/UtilsTest.java    |  4 +-
 .../consensus/simple/SimpleConsensusTest.java      |  4 +-
 .../iotdb/it/env/cluster/MppCommonConfig.java      |  2 +-
 .../it/env/cluster/MppSharedCommonConfig.java      |  6 +-
 .../iotdb/it/env/remote/RemoteCommonConfig.java    |  2 +-
 .../org/apache/iotdb/itbase/env/CommonConfig.java  |  2 +-
 .../confignode/it/IoTDBSnapshotTransferIT.java     |  4 +-
 .../it/load/IoTDBConfigNodeSwitchLeaderIT.java     |  2 +-
 .../it/partition/IoTDBPartitionGetterIT.java       |  6 +-
 ...ConfigNodeRegionId.java => ConfigRegionId.java} |  6 +-
 .../iotdb/commons/consensus/ConsensusGroupId.java  |  8 +-
 .../commons/utils/ThriftCommonsSerDeUtilsTest.java |  2 +-
 .../iotdb/db/auth/ClusterAuthorityFetcher.java     | 14 +--
 .../apache/iotdb/db/client/ConfigNodeClient.java   | 22 ++---
 .../iotdb/db/client/ConfigNodeClientManager.java   |  8 +-
 .../org/apache/iotdb/db/client/ConfigNodeInfo.java |  8 +-
 .../iotdb/db/client/DataNodeClientPoolFactory.java | 14 +--
 .../metadata/template/ClusterTemplateManager.java  | 16 ++--
 .../iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java  |  2 +-
 .../mpp/plan/analyze/ClusterPartitionFetcher.java  | 18 ++--
 .../db/mpp/plan/analyze/cache/PartitionCache.java  | 10 +--
 .../config/executor/ClusterConfigTaskExecutor.java | 99 +++++++++++-----------
 .../java/org/apache/iotdb/db/service/DataNode.java | 10 +--
 .../db/service/DataNodeServerCommandLine.java      |  6 +-
 .../apache/iotdb/db/service/IoTDBShutdownHook.java |  2 +-
 .../iotdb/db/service/RegionMigrateService.java     |  2 +-
 .../db/sync/common/ClusterSyncInfoFetcher.java     | 12 +--
 .../db/trigger/executor/TriggerFireVisitor.java    |  8 +-
 .../trigger/service/TriggerInformationUpdater.java |  8 +-
 thrift-commons/src/main/thrift/common.thrift       |  2 +-
 40 files changed, 213 insertions(+), 236 deletions(-)

diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java
index f92da4aea2..71cac276b7 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java
@@ -51,7 +51,7 @@ public class ConfigNodeConfig {
   private TEndPoint targetConfigNode = new TEndPoint("127.0.0.1", 10710);
 
   // TODO: Read from iotdb-confignode.properties
-  private int configNodeRegionId = 0;
+  private int configRegionId = 0;
 
   /** ConfigNodeGroup consensus protocol */
   private String configNodeConsensusProtocolClass = ConsensusFactory.RATIS_CONSENSUS;
@@ -380,12 +380,12 @@ public class ConfigNodeConfig {
     this.targetConfigNode = targetConfigNode;
   }
 
-  public int getConfigNodeRegionId() {
-    return configNodeRegionId;
+  public int getConfigRegionId() {
+    return configRegionId;
   }
 
-  public void setConfigNodeRegionId(int configNodeRegionId) {
-    this.configNodeRegionId = configNodeRegionId;
+  public void setConfigRegionId(int configRegionId) {
+    this.configRegionId = configRegionId;
   }
 
   public int getSeriesSlotNum() {
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/ConfigNodeRegionStateMachine.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/ConfigRegionStateMachine.java
similarity index 93%
rename from confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/ConfigNodeRegionStateMachine.java
rename to confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/ConfigRegionStateMachine.java
index 97014df506..b4c6dbd77d 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/ConfigNodeRegionStateMachine.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/ConfigRegionStateMachine.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.confignode.consensus.statemachine;
 
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
@@ -55,11 +56,11 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
-/** StateMachine for ConfigNodeRegion */
-public class ConfigNodeRegionStateMachine
+/** StateMachine for ConfigRegion. */
+public class ConfigRegionStateMachine
     implements IStateMachine, IStateMachine.EventApi, IStateMachine.RetryPolicy {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(ConfigNodeRegionStateMachine.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(ConfigRegionStateMachine.class);
 
   private static final ExecutorService threadPool =
       IoTDBThreadPoolFactory.newCachedThreadPool("CQ-recovery");
@@ -67,7 +68,7 @@ public class ConfigNodeRegionStateMachine
   private final ConfigPlanExecutor executor;
   private ConfigManager configManager;
 
-  /** Variables for ConfigNode Simple Consensus */
+  /** Variables for ConfigNode Simple Consensus. */
   private LogWriter simpleLogWriter;
 
   private File simpleLogFile;
@@ -83,7 +84,7 @@ public class ConfigNodeRegionStateMachine
       CONF.getConfigNodeSimpleConsensusLogSegmentSizeMax();
   private final TEndPoint currentNodeTEndPoint;
 
-  public ConfigNodeRegionStateMachine(ConfigManager configManager, ConfigPlanExecutor executor) {
+  public ConfigRegionStateMachine(ConfigManager configManager, ConfigPlanExecutor executor) {
     this.executor = executor;
     this.configManager = configManager;
     this.currentNodeTEndPoint =
@@ -107,13 +108,29 @@ public class ConfigNodeRegionStateMachine
         .orElseGet(() -> new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()));
   }
 
+  /** Transmit PhysicalPlan to confignode.service.executor.PlanExecutor */
+  protected TSStatus write(ConfigPhysicalPlan plan) {
+    TSStatus result;
+    try {
+      result = executor.executeNonQueryPlan(plan);
+    } catch (UnknownPhysicalPlanTypeException | AuthException e) {
+      LOGGER.error(e.getMessage());
+      result = new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode());
+    }
+
+    if (ConsensusFactory.SIMPLE_CONSENSUS.equals(CONF.getConfigNodeConsensusProtocolClass())) {
+      writeLogForSimpleConsensus(plan);
+    }
+    return result;
+  }
+
   @Override
   public IConsensusRequest deserializeRequest(IConsensusRequest request) {
     IConsensusRequest result;
     if (request instanceof ByteBufferConsensusRequest) {
       try {
         result = ConfigPhysicalPlan.Factory.create(request.serializeToByteBuffer());
-      } catch (Throwable e) {
+      } catch (Exception e) {
         LOGGER.error(
             "Deserialization error for write plan, request: {}, bytebuffer: {}",
             request,
@@ -133,29 +150,13 @@ public class ConfigNodeRegionStateMachine
     return result;
   }
 
-  /** Transmit PhysicalPlan to confignode.service.executor.PlanExecutor */
-  protected TSStatus write(ConfigPhysicalPlan plan) {
-    TSStatus result;
-    try {
-      result = executor.executeNonQueryPlan(plan);
-    } catch (UnknownPhysicalPlanTypeException | AuthException e) {
-      LOGGER.error(e.getMessage());
-      result = new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode());
-    }
-
-    if (ConsensusFactory.SIMPLE_CONSENSUS.equals(CONF.getConfigNodeConsensusProtocolClass())) {
-      writeLogForSimpleConsensus(plan);
-    }
-    return result;
-  }
-
   @Override
   public DataSet read(IConsensusRequest request) {
     ConfigPhysicalPlan plan;
     if (request instanceof ByteBufferConsensusRequest) {
       try {
         plan = ConfigPhysicalPlan.Factory.create(request.serializeToByteBuffer());
-      } catch (Throwable e) {
+      } catch (Exception e) {
         LOGGER.error("Deserialization error for write plan : {}", request);
         return null;
       }
@@ -168,16 +169,6 @@ public class ConfigNodeRegionStateMachine
     return read(plan);
   }
 
-  @Override
-  public boolean takeSnapshot(File snapshotDir) {
-    return executor.takeSnapshot(snapshotDir);
-  }
-
-  @Override
-  public void loadSnapshot(File latestSnapshotRootDir) {
-    executor.loadSnapshot(latestSnapshotRootDir);
-  }
-
   /** Transmit PhysicalPlan to confignode.service.executor.PlanExecutor */
   protected DataSet read(ConfigPhysicalPlan plan) {
     DataSet result;
@@ -190,10 +181,20 @@ public class ConfigNodeRegionStateMachine
     return result;
   }
 
+  @Override
+  public boolean takeSnapshot(File snapshotDir) {
+    return executor.takeSnapshot(snapshotDir);
+  }
+
+  @Override
+  public void loadSnapshot(File latestSnapshotRootDir) {
+    executor.loadSnapshot(latestSnapshotRootDir);
+  }
+
   @Override
   public void notifyLeaderChanged(ConsensusGroupId groupId, int newLeaderId) {
     // We get currentNodeId here because the currentNodeId
-    // couldn't initialize earlier than the ConfigNodeRegionStateMachine
+    // couldn't initialize earlier than the ConfigRegionStateMachine
     int currentNodeId = ConfigNodeDescriptor.getInstance().getConf().getConfigNodeId();
 
     if (currentNodeId == newLeaderId) {
@@ -254,25 +255,7 @@ public class ConfigNodeRegionStateMachine
     return CommonDescriptor.getInstance().getConfig().isReadOnly();
   }
 
-  @Override
-  public boolean shouldRetry(TSStatus writeResult) {
-    // TODO implement this
-    return RetryPolicy.super.shouldRetry(writeResult);
-  }
-
-  @Override
-  public TSStatus updateResult(TSStatus previousResult, TSStatus retryResult) {
-    // TODO implement this
-    return RetryPolicy.super.updateResult(previousResult, retryResult);
-  }
-
-  @Override
-  public long getSleepTime() {
-    // TODO implement this
-    return RetryPolicy.super.getSleepTime();
-  }
-
-  /** TODO optimize the lock usage */
+  /** TODO optimize the lock usage. */
   private synchronized void writeLogForSimpleConsensus(ConfigPhysicalPlan plan) {
     if (simpleLogFile.length() > LOG_FILE_MAX_SIZE) {
       try {
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 6b24df7bb9..1c3023588f 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
@@ -79,7 +79,7 @@ import org.apache.iotdb.confignode.consensus.response.partition.RegionInfoListRe
 import org.apache.iotdb.confignode.consensus.response.partition.SchemaNodeManagementResp;
 import org.apache.iotdb.confignode.consensus.response.partition.SchemaPartitionResp;
 import org.apache.iotdb.confignode.consensus.response.template.TemplateSetInfoResp;
-import org.apache.iotdb.confignode.consensus.statemachine.ConfigNodeRegionStateMachine;
+import org.apache.iotdb.confignode.consensus.statemachine.ConfigRegionStateMachine;
 import org.apache.iotdb.confignode.manager.consensus.ConsensusManager;
 import org.apache.iotdb.confignode.manager.cq.CQManager;
 import org.apache.iotdb.confignode.manager.load.LoadManager;
@@ -229,7 +229,7 @@ public class ConfigManager implements IManager {
   /** ML Model */
   private final ModelManager modelManager;
 
-  private final ConfigNodeRegionStateMachine stateMachine;
+  private final ConfigRegionStateMachine stateMachine;
 
   private final RetryFailedTasksThread retryFailedTasksThread;
 
@@ -259,7 +259,7 @@ public class ConfigManager implements IManager {
             syncInfo,
             cqInfo,
             modelInfo);
-    this.stateMachine = new ConfigNodeRegionStateMachine(this, executor);
+    this.stateMachine = new ConfigRegionStateMachine(this, executor);
 
     // Build the manager module
     this.nodeManager = new NodeManager(this, nodeInfo);
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java
index d31e6678a9..e3b4f948e1 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java
@@ -23,14 +23,14 @@ import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.exception.BadNodeUrlException;
 import org.apache.iotdb.confignode.conf.ConfigNodeConfig;
 import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
 import org.apache.iotdb.confignode.conf.SystemPropertiesUtils;
 import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan;
-import org.apache.iotdb.confignode.consensus.statemachine.ConfigNodeRegionStateMachine;
+import org.apache.iotdb.confignode.consensus.statemachine.ConfigRegionStateMachine;
 import org.apache.iotdb.confignode.exception.AddPeerException;
 import org.apache.iotdb.confignode.manager.IManager;
 import org.apache.iotdb.confignode.manager.node.NodeManager;
@@ -68,7 +68,7 @@ public class ConsensusManager {
   private ConsensusGroupId consensusGroupId;
   private IConsensus consensusImpl;
 
-  public ConsensusManager(IManager configManager, ConfigNodeRegionStateMachine stateMachine)
+  public ConsensusManager(IManager configManager, ConfigRegionStateMachine stateMachine)
       throws IOException {
     this.configManager = configManager;
     setConsensusLayer(stateMachine);
@@ -79,9 +79,9 @@ public class ConsensusManager {
   }
 
   /** ConsensusLayer local implementation. */
-  private void setConsensusLayer(ConfigNodeRegionStateMachine stateMachine) throws IOException {
+  private void setConsensusLayer(ConfigRegionStateMachine stateMachine) throws IOException {
     // There is only one ConfigNodeGroup
-    consensusGroupId = new ConfigNodeRegionId(CONF.getConfigNodeRegionId());
+    consensusGroupId = new ConfigRegionId(CONF.getConfigRegionId());
 
     if (SIMPLE_CONSENSUS.equals(CONF.getConfigNodeConsensusProtocolClass())) {
       consensusImpl =
@@ -91,7 +91,7 @@ public class ConsensusManager {
                       .setThisNode(
                           new TEndPoint(CONF.getInternalAddress(), CONF.getConsensusPort()))
                       .setStorageDir("target" + java.io.File.separator + "simple")
-                      .setConsensusGroupType(TConsensusGroupType.ConfigNodeRegion)
+                      .setConsensusGroupType(TConsensusGroupType.ConfigRegion)
                       .build(),
                   gid -> stateMachine)
               .orElseThrow(
@@ -180,7 +180,7 @@ public class ConsensusManager {
                                       .build())
                               .build())
                       .setStorageDir(CONF.getConsensusDir())
-                      .setConsensusGroupType(TConsensusGroupType.ConfigNodeRegion)
+                      .setConsensusGroupType(TConsensusGroupType.ConfigRegion)
                       .build(),
                   gid -> stateMachine)
               .orElseThrow(
@@ -236,7 +236,7 @@ public class ConsensusManager {
   }
 
   /**
-   * Add a new ConfigNode Peer into ConfigNodeRegion.
+   * Add a new ConfigNode Peer into ConfigRegion.
    *
    * @param configNodeLocation The new ConfigNode
    * @throws AddPeerException When addPeer doesn't success
@@ -258,11 +258,11 @@ public class ConsensusManager {
   }
 
   /**
-   * Remove a ConfigNode Peer out of ConfigNodeRegion.
+   * Remove a ConfigNode Peer out of ConfigRegion.
    *
    * @param configNodeLocation config node location
    * @return True if successfully removePeer. False if another ConfigNode is being removed to the
-   *     ConfigNodeRegion
+   *     ConfigRegion
    */
   public boolean removeConfigNodePeer(TConfigNodeLocation configNodeLocation) {
     return consensusImpl
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/DatabasePartitionTable.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/DatabasePartitionTable.java
index 5866a092c3..bb0bb153c9 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/DatabasePartitionTable.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/DatabasePartitionTable.java
@@ -424,7 +424,7 @@ public class DatabasePartitionTable {
         return dataPartitionTable.getSeriesSlotList();
       case SchemaRegion:
         return schemaPartitionTable.getSeriesSlotList();
-      case ConfigNodeRegion:
+      case ConfigRegion:
       default:
         return Stream.concat(
                 schemaPartitionTable.getSeriesSlotList().stream(),
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java
index 1771d16841..ca5377efb1 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java
@@ -234,7 +234,7 @@ public class ConfigNodeProcedureEnv {
   }
 
   /**
-   * Leader will add the new ConfigNode Peer into ConfigNodeRegion
+   * Leader will add the new ConfigNode Peer into ConfigRegion
    *
    * @param configNodeLocation The new ConfigNode
    * @throws AddPeerException When addPeer doesn't success
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/DataNodeRemoveHandler.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/DataNodeRemoveHandler.java
index 33fec29316..2d8f038152 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/DataNodeRemoveHandler.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/DataNodeRemoveHandler.java
@@ -88,7 +88,7 @@ public class DataNodeRemoveHandler {
         .filter(
             replicaSet ->
                 replicaSet.getDataNodeLocations().contains(removedDataNode)
-                    && replicaSet.regionId.getType() != TConsensusGroupType.ConfigNodeRegion)
+                    && replicaSet.regionId.getType() != TConsensusGroupType.ConfigRegion)
         .map(TRegionReplicaSet::getRegionId)
         .collect(Collectors.toList());
   }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
index 30a96f2fab..248426f1c5 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
@@ -892,8 +892,7 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
 
   @Override
   public TGetSeriesSlotListResp getSeriesSlotList(TGetSeriesSlotListReq req) {
-    TConsensusGroupType type =
-        req.isSetType() ? req.getType() : TConsensusGroupType.ConfigNodeRegion;
+    TConsensusGroupType type = req.isSetType() ? req.getType() : TConsensusGroupType.ConfigRegion;
     GetSeriesSlotListPlan plan = new GetSeriesSlotListPlan(req.getDatabase(), type);
     return configManager.getSeriesSlotList(plan);
   }
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
index b63d8e6a4b..dce99b4275 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
@@ -137,7 +137,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import static org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.ConfigNodeRegion;
+import static org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.ConfigRegion;
 import static org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.DataRegion;
 import static org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.SchemaRegion;
 import static org.junit.Assert.assertEquals;
@@ -1183,7 +1183,7 @@ public class ConfigPhysicalPlanSerDeTest {
   public void GetRegionIdPlanTest() throws IOException {
     GetRegionIdPlan getRegionIdPlan0 =
         new GetRegionIdPlan(
-            "root.test", ConfigNodeRegion, new TSeriesPartitionSlot(1), new TTimePartitionSlot(0));
+            "root.test", ConfigRegion, new TSeriesPartitionSlot(1), new TTimePartitionSlot(0));
     GetRegionIdPlan getRegionIdPlan1 =
         (GetRegionIdPlan)
             ConfigPhysicalPlan.Factory.create(getRegionIdPlan0.serializeToByteBuffer());
diff --git a/consensus/README.md b/consensus/README.md
index 99b3f37933..0402a4bc1e 100644
--- a/consensus/README.md
+++ b/consensus/README.md
@@ -72,7 +72,7 @@ IConsensus consensusImpl =
         ConsensusFactory.RatisConsensus,
         new Endpoint(conf.getRpcAddress(), conf.getInternalPort()),
         new File(conf.getConsensusDir()),
-        gid -> new ConfigNodeRegionStateMachine())
+        gid -> new ConfigRegionStateMachine())
     .orElseThrow(() ->
         new IllegalArgumentException(
         String.format(
diff --git a/consensus/src/test/java/org/apache/iotdb/consensus/ratis/UtilsTest.java b/consensus/src/test/java/org/apache/iotdb/consensus/ratis/UtilsTest.java
index 2d531bdcab..ef2287f3a5 100644
--- a/consensus/src/test/java/org/apache/iotdb/consensus/ratis/UtilsTest.java
+++ b/consensus/src/test/java/org/apache/iotdb/consensus/ratis/UtilsTest.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.consensus.ratis;
 
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 
 import org.apache.ratis.protocol.RaftGroupId;
@@ -28,7 +28,7 @@ import org.junit.Test;
 public class UtilsTest {
   @Test
   public void testEncryption() {
-    ConsensusGroupId raw = new ConfigNodeRegionId(100);
+    ConsensusGroupId raw = new ConfigRegionId(100);
     RaftGroupId id = Utils.fromConsensusGroupIdToRaftGroupId(raw);
     ConsensusGroupId cgid = Utils.fromRaftGroupIdToConsensusGroupId(id);
     Assert.assertEquals(raw.getId(), cgid.getId());
diff --git a/consensus/src/test/java/org/apache/iotdb/consensus/simple/SimpleConsensusTest.java b/consensus/src/test/java/org/apache/iotdb/consensus/simple/SimpleConsensusTest.java
index 558c8bdf4f..21e57b1b28 100644
--- a/consensus/src/test/java/org/apache/iotdb/consensus/simple/SimpleConsensusTest.java
+++ b/consensus/src/test/java/org/apache/iotdb/consensus/simple/SimpleConsensusTest.java
@@ -21,7 +21,7 @@ package org.apache.iotdb.consensus.simple;
 
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.DataRegionId;
 import org.apache.iotdb.commons.consensus.SchemaRegionId;
@@ -65,7 +65,7 @@ public class SimpleConsensusTest {
       new ByteBufferConsensusRequest(ByteBuffer.wrap(new byte[4]));
   private final ConsensusGroupId dataRegionId = new DataRegionId(0);
   private final ConsensusGroupId schemaRegionId = new SchemaRegionId(1);
-  private final ConsensusGroupId configId = new ConfigNodeRegionId(2);
+  private final ConsensusGroupId configId = new ConfigRegionId(2);
 
   private static class TestEntry implements IConsensusRequest {
 
diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java
index 4518e2531d..c1bcbde4ec 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java
@@ -141,7 +141,7 @@ public class MppCommonConfig extends MppBaseConfig implements CommonConfig {
   }
 
   @Override
-  public CommonConfig setConfigNodeRegionRatisRPCLeaderElectionTimeoutMaxMs(int maxMs) {
+  public CommonConfig setConfigRegionRatisRPCLeaderElectionTimeoutMaxMs(int maxMs) {
     setProperty("config_node_ratis_rpc_leader_election_timeout_max_ms", String.valueOf(maxMs));
     return this;
   }
diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java
index 32e92e7669..ddaba5a331 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java
@@ -73,9 +73,9 @@ public class MppSharedCommonConfig implements CommonConfig {
   }
 
   @Override
-  public CommonConfig setConfigNodeRegionRatisRPCLeaderElectionTimeoutMaxMs(int maxMs) {
-    cnConfig.setConfigNodeRegionRatisRPCLeaderElectionTimeoutMaxMs(maxMs);
-    dnConfig.setConfigNodeRegionRatisRPCLeaderElectionTimeoutMaxMs(maxMs);
+  public CommonConfig setConfigRegionRatisRPCLeaderElectionTimeoutMaxMs(int maxMs) {
+    cnConfig.setConfigRegionRatisRPCLeaderElectionTimeoutMaxMs(maxMs);
+    dnConfig.setConfigRegionRatisRPCLeaderElectionTimeoutMaxMs(maxMs);
     return this;
   }
 
diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java
index b47d0cc441..440b38253e 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java
@@ -52,7 +52,7 @@ public class RemoteCommonConfig implements CommonConfig {
   }
 
   @Override
-  public CommonConfig setConfigNodeRegionRatisRPCLeaderElectionTimeoutMaxMs(int maxMs) {
+  public CommonConfig setConfigRegionRatisRPCLeaderElectionTimeoutMaxMs(int maxMs) {
     return this;
   }
 
diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
index 416a043345..59dcd778ab 100644
--- a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
@@ -33,7 +33,7 @@ public interface CommonConfig {
 
   CommonConfig setCompressor(String compressor);
 
-  CommonConfig setConfigNodeRegionRatisRPCLeaderElectionTimeoutMaxMs(int maxMs);
+  CommonConfig setConfigRegionRatisRPCLeaderElectionTimeoutMaxMs(int maxMs);
 
   CommonConfig setUdfMemoryBudgetInMB(float udfCollectorMemoryBudgetInMB);
 
diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBSnapshotTransferIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBSnapshotTransferIT.java
index c66ce401d1..ae5676d189 100644
--- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBSnapshotTransferIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBSnapshotTransferIT.java
@@ -144,9 +144,7 @@ public class IoTDBSnapshotTransferIT {
           registerResult.getRegionInfoList().stream()
               .filter(
                   info ->
-                      info.getConsensusGroupId()
-                          .getType()
-                          .equals(TConsensusGroupType.ConfigNodeRegion))
+                      info.getConsensusGroupId().getType().equals(TConsensusGroupType.ConfigRegion))
               .count();
       Assert.assertEquals(4, configNodeGroupCount);
     }
diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/load/IoTDBConfigNodeSwitchLeaderIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/load/IoTDBConfigNodeSwitchLeaderIT.java
index 3c264f70bf..a49d1501af 100644
--- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/load/IoTDBConfigNodeSwitchLeaderIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/load/IoTDBConfigNodeSwitchLeaderIT.java
@@ -68,7 +68,7 @@ public class IoTDBConfigNodeSwitchLeaderIT {
         .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS)
         .setSchemaReplicationFactor(testReplicationFactor)
         .setDataReplicationFactor(testReplicationFactor)
-        .setConfigNodeRegionRatisRPCLeaderElectionTimeoutMaxMs(testElectionTimeoutMaxMs);
+        .setConfigRegionRatisRPCLeaderElectionTimeoutMaxMs(testElectionTimeoutMaxMs);
 
     // Init 3C3D cluster environment
     EnvFactory.getEnv().initClusterEnvironment(testConfigNodeNum, testDataNodeNum);
diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java
index f8afd8d2d4..96c7ac7715 100644
--- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java
@@ -370,7 +370,7 @@ public class IoTDBPartitionGetterIT {
           TSStatusCode.ILLEGAL_PARAMETER.getStatusCode(), getRegionIdResp.status.getCode());
 
       // Get RegionId with wrong RegionType
-      getRegionIdReq.setType(TConsensusGroupType.ConfigNodeRegion);
+      getRegionIdReq.setType(TConsensusGroupType.ConfigRegion);
       getRegionIdResp = client.getRegionId(getRegionIdReq);
       Assert.assertEquals(
           TSStatusCode.ILLEGAL_PARAMETER.getStatusCode(), getRegionIdResp.status.getCode());
@@ -415,7 +415,7 @@ public class IoTDBPartitionGetterIT {
           TSStatusCode.SUCCESS_STATUS.getStatusCode(), getRegionIdResp.status.getCode());
       Assert.assertEquals(1, getRegionIdResp.getDataRegionIdListSize());
 
-      getRegionIdReq.setType(TConsensusGroupType.ConfigNodeRegion);
+      getRegionIdReq.setType(TConsensusGroupType.ConfigRegion);
       getRegionIdResp = client.getRegionId(getRegionIdReq);
       Assert.assertEquals(
           TSStatusCode.SUCCESS_STATUS.getStatusCode(), getRegionIdResp.status.getCode());
@@ -461,7 +461,7 @@ public class IoTDBPartitionGetterIT {
       Assert.assertEquals(
           testSeriesPartitionSlotNum + 2, getSeriesSlotListResp.getSeriesSlotListSize());
 
-      getSeriesSlotListReq.setType(TConsensusGroupType.ConfigNodeRegion);
+      getSeriesSlotListReq.setType(TConsensusGroupType.ConfigRegion);
 
       getSeriesSlotListResp = client.getSeriesSlotList(getSeriesSlotListReq);
       Assert.assertEquals(
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConfigNodeRegionId.java b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConfigRegionId.java
similarity index 87%
rename from node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConfigNodeRegionId.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConfigRegionId.java
index c6850e3dd2..d057b5bba5 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConfigNodeRegionId.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConfigRegionId.java
@@ -21,14 +21,14 @@ package org.apache.iotdb.commons.consensus;
 
 import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 
-public class ConfigNodeRegionId extends ConsensusGroupId {
+public class ConfigRegionId extends ConsensusGroupId {
 
-  public ConfigNodeRegionId(int id) {
+  public ConfigRegionId(int id) {
     this.id = id;
   }
 
   @Override
   public TConsensusGroupType getType() {
-    return TConsensusGroupType.ConfigNodeRegion;
+    return TConsensusGroupType.ConfigRegion;
   }
 }
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 167c1c3b3e..f3030d6af6 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
@@ -71,8 +71,8 @@ public abstract class ConsensusGroupId {
         groupId = new DataRegionId(id);
       } else if (type == TConsensusGroupType.SchemaRegion.getValue()) {
         groupId = new SchemaRegionId(id);
-      } else if (type == TConsensusGroupType.ConfigNodeRegion.getValue()) {
-        groupId = new ConfigNodeRegionId(id);
+      } else if (type == TConsensusGroupType.ConfigRegion.getValue()) {
+        groupId = new ConfigRegionId(id);
       } else {
         throw new IllegalArgumentException(
             "Unrecognized TConsensusGroupType: " + type + " with id = " + id);
@@ -96,8 +96,8 @@ public abstract class ConsensusGroupId {
       case DataRegion:
         format.append("DataRegion");
         break;
-      case ConfigNodeRegion:
-        format.append("ConfigNodeRegion");
+      case ConfigRegion:
+        format.append("ConfigRegion");
         break;
     }
 
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
index d5c1e4f65b..ffc69800b2 100644
--- 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
@@ -138,7 +138,7 @@ public class ThriftCommonsSerDeUtilsTest {
   @Test
   public void readWriteTConsensusGroupIdTest() throws IOException {
     TConsensusGroupId consensusGroupId0 =
-        new TConsensusGroupId(TConsensusGroupType.ConfigNodeRegion, 0);
+        new TConsensusGroupId(TConsensusGroupType.ConfigRegion, 0);
     try (PublicBAOS byteArrayOutputStream = new PublicBAOS();
         DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) {
       ThriftCommonsSerDeUtils.serializeTConsensusGroupId(consensusGroupId0, outputStream);
diff --git a/server/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java b/server/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java
index 3e35f14702..6c48a86e96 100644
--- a/server/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java
@@ -28,7 +28,7 @@ import org.apache.iotdb.commons.auth.entity.Role;
 import org.apache.iotdb.commons.auth.entity.User;
 import org.apache.iotdb.commons.client.IClientManager;
 import org.apache.iotdb.commons.client.exception.ClientManagerException;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.exception.IoTDBException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.utils.AuthUtils;
@@ -65,8 +65,8 @@ public class ClusterAuthorityFetcher implements IAuthorityFetcher {
   private IAuthorCache iAuthorCache;
   private IAuthorizer authorizer;
 
-  private static final IClientManager<ConfigNodeRegionId, ConfigNodeClient>
-      CONFIG_NODE_CLIENT_MANAGER = ConfigNodeClientManager.getInstance();
+  private static final IClientManager<ConfigRegionId, ConfigNodeClient> CONFIG_NODE_CLIENT_MANAGER =
+      ConfigNodeClientManager.getInstance();
 
   public ClusterAuthorityFetcher(IAuthorCache iAuthorCache) {
     this.iAuthorCache = iAuthorCache;
@@ -122,7 +122,7 @@ public class ClusterAuthorityFetcher implements IAuthorityFetcher {
   public SettableFuture<ConfigTaskResult> operatePermission(AuthorStatement authorStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       // Construct request using statement
       TAuthorizerReq authorizerReq = statementToAuthorizerReq(authorStatement);
       // Send request to some API server
@@ -154,7 +154,7 @@ public class ClusterAuthorityFetcher implements IAuthorityFetcher {
     TAuthorizerResp authorizerResp = new TAuthorizerResp();
 
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       // Construct request using statement
       TAuthorizerReq authorizerReq = statementToAuthorizerReq(authorStatement);
       // Send request to some API server
@@ -204,7 +204,7 @@ public class ClusterAuthorityFetcher implements IAuthorityFetcher {
       TLoginReq req = new TLoginReq(username, password);
       TPermissionInfoResp status = null;
       try (ConfigNodeClient configNodeClient =
-          CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         // Send request to some API server
         status = configNodeClient.login(req);
       } catch (ClientManagerException | TException e) {
@@ -231,7 +231,7 @@ public class ClusterAuthorityFetcher implements IAuthorityFetcher {
     TCheckUserPrivilegesReq req = new TCheckUserPrivilegesReq(username, allPath, permission);
     TPermissionInfoResp permissionInfoResp;
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       // Send request to some API server
       permissionInfoResp = configNodeClient.checkUserPrivileges(req);
     } catch (ClientManagerException | TException e) {
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 c525d5cd75..6dc2f13c4a 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
@@ -30,7 +30,7 @@ import org.apache.iotdb.commons.client.ThriftClient;
 import org.apache.iotdb.commons.client.factory.ThriftClientFactory;
 import org.apache.iotdb.commons.client.property.ThriftClientProperty;
 import org.apache.iotdb.commons.client.sync.SyncThriftClientWithErrorHandler;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.confignode.rpc.thrift.IConfigNodeRPCService;
 import org.apache.iotdb.confignode.rpc.thrift.TAddConsensusGroupReq;
 import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerReq;
@@ -163,9 +163,9 @@ public class ConfigNodeClient implements IConfigNodeRPCService.Iface, ThriftClie
 
   private final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
 
-  ClientManager<ConfigNodeRegionId, ConfigNodeClient> clientManager;
+  ClientManager<ConfigRegionId, ConfigNodeClient> clientManager;
 
-  ConfigNodeRegionId configNodeRegionId = ConfigNodeInfo.configNodeRegionId;
+  ConfigRegionId configRegionId = ConfigNodeInfo.CONFIG_REGION_ID;
 
   TProtocolFactory protocolFactory;
 
@@ -173,7 +173,7 @@ public class ConfigNodeClient implements IConfigNodeRPCService.Iface, ThriftClie
       List<TEndPoint> configNodes,
       TProtocolFactory protocolFactory,
       long connectionTimeout,
-      ClientManager<ConfigNodeRegionId, ConfigNodeClient> clientManager)
+      ClientManager<ConfigRegionId, ConfigNodeClient> clientManager)
       throws TException {
     this.configNodes = configNodes;
     this.protocolFactory = protocolFactory;
@@ -270,7 +270,7 @@ public class ConfigNodeClient implements IConfigNodeRPCService.Iface, ThriftClie
 
   @Override
   public void close() {
-    clientManager.returnClient(configNodeRegionId, this);
+    clientManager.returnClient(configRegionId, this);
   }
 
   @Override
@@ -280,7 +280,7 @@ public class ConfigNodeClient implements IConfigNodeRPCService.Iface, ThriftClie
 
   @Override
   public void invalidateAll() {
-    clientManager.clear(ConfigNodeInfo.configNodeRegionId);
+    clientManager.clear(ConfigNodeInfo.CONFIG_REGION_ID);
   }
 
   private boolean updateConfigNodeLeader(TSStatus status) {
@@ -1953,22 +1953,22 @@ public class ConfigNodeClient implements IConfigNodeRPCService.Iface, ThriftClie
     throw new TException(new UnsupportedOperationException().getCause());
   }
 
-  public static class Factory extends ThriftClientFactory<ConfigNodeRegionId, ConfigNodeClient> {
+  public static class Factory extends ThriftClientFactory<ConfigRegionId, ConfigNodeClient> {
 
     public Factory(
-        ClientManager<ConfigNodeRegionId, ConfigNodeClient> clientManager,
+        ClientManager<ConfigRegionId, ConfigNodeClient> clientManager,
         ThriftClientProperty thriftClientProperty) {
       super(clientManager, thriftClientProperty);
     }
 
     @Override
     public void destroyObject(
-        ConfigNodeRegionId configNodeRegionId, PooledObject<ConfigNodeClient> pooledObject) {
+        ConfigRegionId configRegionId, PooledObject<ConfigNodeClient> pooledObject) {
       pooledObject.getObject().invalidate();
     }
 
     @Override
-    public PooledObject<ConfigNodeClient> makeObject(ConfigNodeRegionId configNodeRegionId)
+    public PooledObject<ConfigNodeClient> makeObject(ConfigRegionId configRegionId)
         throws Exception {
       return new DefaultPooledObject<>(
           SyncThriftClientWithErrorHandler.newErrorHandler(
@@ -1983,7 +1983,7 @@ public class ConfigNodeClient implements IConfigNodeRPCService.Iface, ThriftClie
 
     @Override
     public boolean validateObject(
-        ConfigNodeRegionId configNodeRegionId, PooledObject<ConfigNodeClient> pooledObject) {
+        ConfigRegionId configRegionId, PooledObject<ConfigNodeClient> pooledObject) {
       return Optional.ofNullable(pooledObject.getObject().getTransport())
           .map(TTransport::isOpen)
           .orElse(false);
diff --git a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClientManager.java b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClientManager.java
index 12df1907bb..b73ea6d4a6 100644
--- a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClientManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClientManager.java
@@ -20,16 +20,16 @@
 package org.apache.iotdb.db.client;
 
 import org.apache.iotdb.commons.client.IClientManager;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 
 public class ConfigNodeClientManager {
   private static final class ConfigNodeClientManagerHolder {
-    private static final IClientManager<ConfigNodeRegionId, ConfigNodeClient> INSTANCE =
-        new IClientManager.Factory<ConfigNodeRegionId, ConfigNodeClient>()
+    private static final IClientManager<ConfigRegionId, ConfigNodeClient> INSTANCE =
+        new IClientManager.Factory<ConfigRegionId, ConfigNodeClient>()
             .createClientManager(new DataNodeClientPoolFactory.ConfigNodeClientPoolFactory());
   }
 
-  public static IClientManager<ConfigNodeRegionId, ConfigNodeClient> getInstance() {
+  public static IClientManager<ConfigRegionId, ConfigNodeClient> getInstance() {
     return ConfigNodeClientManagerHolder.INSTANCE;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeInfo.java b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeInfo.java
index 78673bc32e..1d61c9485b 100644
--- a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeInfo.java
+++ b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeInfo.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.client;
 
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.exception.BadNodeUrlException;
 import org.apache.iotdb.commons.file.SystemFileFactory;
 import org.apache.iotdb.commons.utils.NodeUrlUtils;
@@ -50,12 +50,12 @@ public class ConfigNodeInfo {
 
   private final ReentrantReadWriteLock configNodeInfoReadWriteLock;
 
-  // latest config nodes
+  /** latest config nodes. */
   private final Set<TEndPoint> onlineConfigNodes;
 
-  public static ConfigNodeRegionId configNodeRegionId = new ConfigNodeRegionId(0);
+  public static final ConfigRegionId CONFIG_REGION_ID = new ConfigRegionId(0);
 
-  private File propertiesFile;
+  private final File propertiesFile;
 
   private ConfigNodeInfo() {
     this.configNodeInfoReadWriteLock = new ReentrantReadWriteLock();
diff --git a/server/src/main/java/org/apache/iotdb/db/client/DataNodeClientPoolFactory.java b/server/src/main/java/org/apache/iotdb/db/client/DataNodeClientPoolFactory.java
index 52114a1415..74065da70a 100644
--- a/server/src/main/java/org/apache/iotdb/db/client/DataNodeClientPoolFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/client/DataNodeClientPoolFactory.java
@@ -23,7 +23,7 @@ import org.apache.iotdb.commons.client.ClientManager;
 import org.apache.iotdb.commons.client.IClientPoolFactory;
 import org.apache.iotdb.commons.client.property.ClientPoolProperty;
 import org.apache.iotdb.commons.client.property.ThriftClientProperty;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 
@@ -37,11 +37,11 @@ public class DataNodeClientPoolFactory {
   private DataNodeClientPoolFactory() {}
 
   public static class ConfigNodeClientPoolFactory
-      implements IClientPoolFactory<ConfigNodeRegionId, ConfigNodeClient> {
+      implements IClientPoolFactory<ConfigRegionId, ConfigNodeClient> {
 
     @Override
-    public KeyedObjectPool<ConfigNodeRegionId, ConfigNodeClient> createClientPool(
-        ClientManager<ConfigNodeRegionId, ConfigNodeClient> manager) {
+    public KeyedObjectPool<ConfigRegionId, ConfigNodeClient> createClientPool(
+        ClientManager<ConfigRegionId, ConfigNodeClient> manager) {
       return new GenericKeyedObjectPool<>(
           new ConfigNodeClient.Factory(
               manager,
@@ -58,11 +58,11 @@ public class DataNodeClientPoolFactory {
   }
 
   public static class ClusterDeletionConfigNodeClientPoolFactory
-      implements IClientPoolFactory<ConfigNodeRegionId, ConfigNodeClient> {
+      implements IClientPoolFactory<ConfigRegionId, ConfigNodeClient> {
 
     @Override
-    public KeyedObjectPool<ConfigNodeRegionId, ConfigNodeClient> createClientPool(
-        ClientManager<ConfigNodeRegionId, ConfigNodeClient> manager) {
+    public KeyedObjectPool<ConfigRegionId, ConfigNodeClient> createClientPool(
+        ClientManager<ConfigRegionId, ConfigNodeClient> manager) {
       return new GenericKeyedObjectPool<>(
           new ConfigNodeClient.Factory(
               manager,
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/template/ClusterTemplateManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/template/ClusterTemplateManager.java
index 605b9b81e0..1350a07a04 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/template/ClusterTemplateManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/template/ClusterTemplateManager.java
@@ -22,7 +22,7 @@ package org.apache.iotdb.db.metadata.template;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.client.IClientManager;
 import org.apache.iotdb.commons.client.exception.ClientManagerException;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.exception.IoTDBException;
 import org.apache.iotdb.commons.path.PartialPath;
@@ -78,14 +78,14 @@ public class ClusterTemplateManager implements ITemplateManager {
     return ClusterTemplateManager.ClusterTemplateManagerHolder.INSTANCE;
   }
 
-  private static final IClientManager<ConfigNodeRegionId, ConfigNodeClient>
-      CONFIG_NODE_CLIENT_MANAGER = ConfigNodeClientManager.getInstance();
+  private static final IClientManager<ConfigRegionId, ConfigNodeClient> CONFIG_NODE_CLIENT_MANAGER =
+      ConfigNodeClientManager.getInstance();
 
   @Override
   public TSStatus createSchemaTemplate(CreateSchemaTemplateStatement statement) {
     TCreateSchemaTemplateReq req = constructTCreateSchemaTemplateReq(statement);
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       // Send request to some API server
       TSStatus tsStatus = configNodeClient.createSchemaTemplate(req);
       // Get response or throw exception
@@ -127,7 +127,7 @@ public class ClusterTemplateManager implements ITemplateManager {
   public List<Template> getAllTemplates() {
     List<Template> templatesList = new ArrayList<>();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TGetAllTemplatesResp tGetAllTemplatesResp = configNodeClient.getAllTemplates();
       // Get response or throw exception
       if (tGetAllTemplatesResp.getStatus().getCode()
@@ -156,7 +156,7 @@ public class ClusterTemplateManager implements ITemplateManager {
   @Override
   public Template getTemplate(String name) {
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TGetTemplateResp resp = configNodeClient.getTemplate(name);
       if (resp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
         byte[] templateBytes = resp.getTemplate();
@@ -177,7 +177,7 @@ public class ClusterTemplateManager implements ITemplateManager {
   @Override
   public void setSchemaTemplate(String name, PartialPath path) {
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TSetSchemaTemplateReq req = new TSetSchemaTemplateReq();
       req.setName(name);
       req.setPath(path.getFullPath());
@@ -194,7 +194,7 @@ public class ClusterTemplateManager implements ITemplateManager {
   public List<PartialPath> getPathsSetTemplate(String name) {
     List<PartialPath> listPath = new ArrayList<>();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TGetPathsSetTemplatesResp resp = configNodeClient.getPathsSetTemplate(name);
       if (resp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
         if (resp.getPathList() != null) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
index a0c4650db9..25953f8460 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
@@ -2833,7 +2833,7 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
 
   private List<TDataNodeLocation> getRunningDataNodeLocations() {
     try (ConfigNodeClient client =
-        ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TGetDataNodeLocationsResp showDataNodesResp = client.getRunningDataNodeLocations();
       if (showDataNodesResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
         throw new StatementAnalyzeException(
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ClusterPartitionFetcher.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ClusterPartitionFetcher.java
index 472dc5b695..7bf6cfecd3 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ClusterPartitionFetcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ClusterPartitionFetcher.java
@@ -24,7 +24,7 @@ import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
 import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.commons.client.IClientManager;
 import org.apache.iotdb.commons.client.exception.ClientManagerException;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.exception.IoTDBException;
 import org.apache.iotdb.commons.partition.DataPartition;
 import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
@@ -73,7 +73,7 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
 
   private final PartitionCache partitionCache;
 
-  private final IClientManager<ConfigNodeRegionId, ConfigNodeClient> configNodeClientManager =
+  private final IClientManager<ConfigRegionId, ConfigNodeClient> configNodeClientManager =
       ConfigNodeClientManager.getInstance();
 
   private static final class ClusterPartitionFetcherHolder {
@@ -97,7 +97,7 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
   @Override
   public SchemaPartition getSchemaPartition(PathPatternTree patternTree) {
     try (ConfigNodeClient client =
-        configNodeClientManager.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       patternTree.constructTree();
       List<String> devicePaths = patternTree.getAllDevicePatterns();
       Map<String, List<String>> storageGroupToDeviceMap =
@@ -129,7 +129,7 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
   @Override
   public SchemaPartition getOrCreateSchemaPartition(PathPatternTree patternTree) {
     try (ConfigNodeClient client =
-        configNodeClientManager.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       patternTree.constructTree();
       List<String> devicePaths = patternTree.getAllDevicePatterns();
       Map<String, List<String>> storageGroupToDeviceMap =
@@ -161,7 +161,7 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
   public SchemaNodeManagementPartition getSchemaNodeManagementPartitionWithLevel(
       PathPatternTree patternTree, Integer level) {
     try (ConfigNodeClient client =
-        configNodeClientManager.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       patternTree.constructTree();
       TSchemaNodeManagementResp schemaNodeManagementResp =
           client.getSchemaNodeManagementPartition(
@@ -180,7 +180,7 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
     DataPartition dataPartition = partitionCache.getDataPartition(sgNameToQueryParamsMap);
     if (null == dataPartition) {
       try (ConfigNodeClient client =
-          configNodeClientManager.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         TDataPartitionTableResp dataPartitionTableResp =
             client.getDataPartitionTable(constructDataPartitionReqForQuery(sgNameToQueryParamsMap));
         if (dataPartitionTableResp.getStatus().getCode()
@@ -207,7 +207,7 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
     // and there is no need to update cache because since we will never fetch it from cache, the
     // update operation will be only time waste
     try (ConfigNodeClient client =
-        configNodeClientManager.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TDataPartitionTableResp dataPartitionTableResp =
           client.getDataPartitionTable(constructDataPartitionReqForQuery(sgNameToQueryParamsMap));
       if (dataPartitionTableResp.getStatus().getCode()
@@ -231,7 +231,7 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
     if (null == dataPartition) {
       // Do not use data partition cache
       try (ConfigNodeClient client =
-          configNodeClientManager.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         TDataPartitionTableResp dataPartitionTableResp =
             client.getOrCreateDataPartitionTable(constructDataPartitionReq(sgNameToQueryParamsMap));
         if (dataPartitionTableResp.getStatus().getCode()
@@ -261,7 +261,7 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
 
     if (null == dataPartition) {
       try (ConfigNodeClient client =
-          configNodeClientManager.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         TDataPartitionReq req = constructDataPartitionReq(splitDataPartitionQueryParams);
         TDataPartitionTableResp dataPartitionTableResp = client.getOrCreateDataPartitionTable(req);
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/cache/PartitionCache.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/cache/PartitionCache.java
index 8ff3f587cf..8449e4fc05 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/cache/PartitionCache.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/cache/PartitionCache.java
@@ -25,7 +25,7 @@ import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
 import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
 import org.apache.iotdb.commons.client.IClientManager;
 import org.apache.iotdb.commons.client.exception.ClientManagerException;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.exception.IoTDBException;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.partition.DataPartition;
@@ -104,7 +104,7 @@ public class PartitionCache {
 
   private final ReentrantReadWriteLock regionReplicaSetLock = new ReentrantReadWriteLock();
 
-  private final IClientManager<ConfigNodeRegionId, ConfigNodeClient> configNodeClientManager =
+  private final IClientManager<ConfigRegionId, ConfigNodeClient> configNodeClientManager =
       ConfigNodeClientManager.getInstance();
 
   public PartitionCache() {
@@ -185,7 +185,7 @@ public class PartitionCache {
       StorageGroupCacheResult<?> result, List<String> devicePaths)
       throws ClientManagerException, TException {
     try (ConfigNodeClient client =
-        configNodeClientManager.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       storageGroupCacheLock.writeLock().lock();
       result.reset();
       getStorageGroupMap(result, devicePaths, true);
@@ -214,7 +214,7 @@ public class PartitionCache {
       StorageGroupCacheResult<?> result, List<String> devicePaths)
       throws ClientManagerException, MetadataException, TException {
     try (ConfigNodeClient client =
-        configNodeClientManager.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       storageGroupCacheLock.writeLock().lock();
       // try to check whether database need to be created
       result.reset();
@@ -406,7 +406,7 @@ public class PartitionCache {
         // verify that there are not hit in cache
         if (!groupIdToReplicaSetMap.containsKey(consensusGroupId)) {
           try (ConfigNodeClient client =
-              configNodeClientManager.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+              configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
             TRegionRouteMapResp resp = client.getLatestRegionRouteMap();
             if (TSStatusCode.SUCCESS_STATUS.getStatusCode() == resp.getStatus().getCode()) {
               updateGroupIdToReplicaSetMap(resp.getTimestamp(), resp.getRegionRouteMap());
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java
index 52fd828625..a079c260e4 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java
@@ -26,7 +26,7 @@ import org.apache.iotdb.commons.client.IClientManager;
 import org.apache.iotdb.commons.client.exception.ClientManagerException;
 import org.apache.iotdb.commons.cluster.NodeStatus;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.exception.IoTDBException;
 import org.apache.iotdb.commons.executable.ExecutableManager;
 import org.apache.iotdb.commons.executable.ExecutableResource;
@@ -181,13 +181,13 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
   private static final Logger LOGGER = LoggerFactory.getLogger(ClusterConfigTaskExecutor.class);
 
-  private static final IClientManager<ConfigNodeRegionId, ConfigNodeClient>
-      CONFIG_NODE_CLIENT_MANAGER = ConfigNodeClientManager.getInstance();
+  private static final IClientManager<ConfigRegionId, ConfigNodeClient> CONFIG_NODE_CLIENT_MANAGER =
+      ConfigNodeClientManager.getInstance();
 
   /** FIXME Consolidate this clientManager with the upper one. */
-  private static final IClientManager<ConfigNodeRegionId, ConfigNodeClient>
+  private static final IClientManager<ConfigRegionId, ConfigNodeClient>
       CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER =
-          new IClientManager.Factory<ConfigNodeRegionId, ConfigNodeClient>()
+          new IClientManager.Factory<ConfigRegionId, ConfigNodeClient>()
               .createClientManager(
                   new DataNodeClientPoolFactory.ClusterDeletionConfigNodeClientPoolFactory());
 
@@ -210,7 +210,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     TDatabaseSchema storageGroupSchema =
         DatabaseSchemaTask.constructStorageGroupSchema(databaseSchemaStatement);
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       // Send request to some API server
       TSStatus tsStatus = configNodeClient.setDatabase(storageGroupSchema);
       // Get response or throw exception
@@ -237,7 +237,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     TDatabaseSchema storageGroupSchema =
         DatabaseSchemaTask.constructStorageGroupSchema(databaseSchemaStatement);
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       // Send request to some API server
       TSStatus tsStatus = configNodeClient.alterDatabase(storageGroupSchema);
       // Get response or throw exception
@@ -264,7 +264,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     List<String> storageGroupPathPattern =
         Arrays.asList(showStorageGroupStatement.getPathPattern().getNodes());
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       // Send request to some API server
       TShowDatabaseResp resp = client.showDatabase(storageGroupPathPattern);
       // build TSBlock
@@ -283,7 +283,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     List<String> storageGroupPathPattern =
         Arrays.asList(countStorageGroupStatement.getPathPattern().getNodes());
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TCountDatabaseResp resp = client.countMatchedDatabases(storageGroupPathPattern);
       storageGroupNum = resp.getCount();
       // build TSBlock
@@ -300,7 +300,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     TDeleteDatabasesReq req = new TDeleteDatabasesReq(deleteStorageGroupStatement.getPrefixPath());
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TSStatus tsStatus = client.deleteDatabases(req);
       if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) {
         LOGGER.warn(
@@ -324,7 +324,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     String udfName = createFunctionStatement.getUdfName();
     String className = createFunctionStatement.getClassName();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TCreateFunctionReq tCreateFunctionReq = new TCreateFunctionReq(udfName, className, false);
       String libRoot = UDFExecutableManager.getInstance().getLibRoot();
       String jarFileName;
@@ -443,7 +443,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   public SettableFuture<ConfigTaskResult> dropFunction(String udfName) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       final TSStatus executionStatus = client.dropFunction(new TDropFunctionReq(udfName));
 
       if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != executionStatus.getCode()) {
@@ -462,7 +462,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   public SettableFuture<ConfigTaskResult> showFunctions() {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TGetUDFTableResp getUDFTableResp = client.getUDFTable();
       if (getUDFTableResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
         future.setException(
@@ -484,7 +484,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
       CreateTriggerStatement createTriggerStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
 
       TCreateTriggerReq tCreateTriggerReq =
           new TCreateTriggerReq(
@@ -618,7 +618,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   public SettableFuture<ConfigTaskResult> dropTrigger(String triggerName) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       final TSStatus executionStatus = client.dropTrigger(new TDropTriggerReq(triggerName));
       if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != executionStatus.getCode()) {
         LOGGER.warn("[{}] Failed to drop trigger {}.", executionStatus, triggerName);
@@ -636,7 +636,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   public SettableFuture<ConfigTaskResult> showTriggers() {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TGetTriggerTableResp getTriggerTableResp = client.getTriggerTable();
       if (getTriggerTableResp.getStatus().getCode()
           != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
@@ -661,7 +661,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
         Arrays.asList(setTTLStatement.getStorageGroupPath().getNodes());
     TSetTTLReq setTTLReq = new TSetTTLReq(storageGroupPathPattern, setTTLStatement.getTTL());
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       // Send request to some API server
       TSStatus tsStatus = configNodeClient.setTTL(setTTLReq);
       // Get response or throw exception
@@ -687,7 +687,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     TSStatus tsStatus = new TSStatus();
     if (onCluster) {
       try (ConfigNodeClient client =
-          CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         // Send request to some API server
         tsStatus = client.merge();
       } catch (ClientManagerException | TException e) {
@@ -715,7 +715,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     TSStatus tsStatus = new TSStatus();
     if (onCluster) {
       try (ConfigNodeClient client =
-          CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         // Send request to some API server
         tsStatus = client.flush(tFlushReq);
       } catch (ClientManagerException | TException e) {
@@ -743,7 +743,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     TSStatus tsStatus = new TSStatus();
     if (onCluster) {
       try (ConfigNodeClient client =
-          CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         // Send request to some API server
         tsStatus = client.clearCache();
       } catch (ClientManagerException | TException e) {
@@ -771,7 +771,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     TSStatus tsStatus = new TSStatus();
     if (onCluster) {
       try (ConfigNodeClient client =
-          CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         // Send request to some API server
         tsStatus = client.loadConfiguration();
       } catch (ClientManagerException | TException e) {
@@ -799,7 +799,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     TSStatus tsStatus = new TSStatus();
     if (onCluster) {
       try (ConfigNodeClient client =
-          CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         // Send request to some API server
         tsStatus = client.setSystemStatus(status.getStatus());
       } catch (ClientManagerException | TException e) {
@@ -839,7 +839,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     }
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       final TSStatus executionStatus = client.killQuery(queryId, dataNodeId);
       if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != executionStatus.getCode()) {
         LOGGER.warn("Failed to kill query [{}], because {}", queryId, executionStatus.message);
@@ -858,7 +858,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     TShowClusterResp showClusterResp = new TShowClusterResp();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       showClusterResp = client.showCluster();
     } catch (ClientManagerException | TException e) {
       if (showClusterResp.getConfigNodeList() == null) {
@@ -883,7 +883,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     TShowVariablesResp showVariablesResp = new TShowVariablesResp();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       showVariablesResp = client.showVariables();
     } catch (ClientManagerException | TException e) {
       future.setException(e);
@@ -901,7 +901,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     List<PartialPath> storageGroupPaths = showTTLStatement.getPaths();
     Map<String, Long> storageGroupToTTL = new HashMap<>();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       if (showTTLStatement.isAll()) {
         List<String> allStorageGroupPathPattern = Arrays.asList("root", "**");
         TDatabaseSchemaResp resp = client.getMatchedDatabaseSchemas(allStorageGroupPathPattern);
@@ -942,7 +942,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
               .collect(Collectors.toList()));
     }
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       showRegionResp = client.showRegion(showRegionReq);
       if (showRegionResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
         future.setException(
@@ -977,7 +977,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     TShowDataNodesResp showDataNodesResp = new TShowDataNodesResp();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       showDataNodesResp = client.showDataNodes();
       if (showDataNodesResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
         future.setException(
@@ -998,7 +998,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     TShowConfigNodesResp showConfigNodesResp = new TShowConfigNodesResp();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       showConfigNodesResp = client.showConfigNodes();
       if (showConfigNodesResp.getStatus().getCode()
           != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
@@ -1115,8 +1115,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     req.setPathPatternTree(
         serializePatternListToByteBuffer(deactivateTemplateStatement.getPathPatternList()));
     try (ConfigNodeClient client =
-        CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(
-            ConfigNodeInfo.configNodeRegionId)) {
+        CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TSStatus tsStatus;
       do {
         try {
@@ -1154,7 +1153,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
       DropSchemaTemplateStatement dropSchemaTemplateStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       // Send request to some API server
       TSStatus tsStatus =
           configNodeClient.dropSchemaTemplate(dropSchemaTemplateStatement.getTemplateName());
@@ -1199,8 +1198,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     req.setTemplateName(unsetSchemaTemplateStatement.getTemplateName());
     req.setPath(unsetSchemaTemplateStatement.getPath().getFullPath());
     try (ConfigNodeClient client =
-        CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(
-            ConfigNodeInfo.configNodeRegionId)) {
+        CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TSStatus tsStatus;
       do {
         try {
@@ -1238,7 +1236,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
       CreatePipeSinkStatement createPipeSinkStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TPipeSinkInfo pipeSinkInfo = new TPipeSinkInfo();
       pipeSinkInfo.setPipeSinkName(createPipeSinkStatement.getPipeSinkName());
       pipeSinkInfo.setPipeSinkType(createPipeSinkStatement.getPipeSinkType());
@@ -1265,7 +1263,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
       DropPipeSinkStatement dropPipeSinkStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TDropPipeSinkReq req = new TDropPipeSinkReq();
       req.setPipeSinkName(dropPipeSinkStatement.getPipeSinkName());
       TSStatus tsStatus = configNodeClient.dropPipeSink(req);
@@ -1289,7 +1287,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
       ShowPipeSinkStatement showPipeSinkStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TGetPipeSinkReq tGetPipeSinkReq = new TGetPipeSinkReq();
       if (!StringUtils.isEmpty(showPipeSinkStatement.getPipeSinkName())) {
         tGetPipeSinkReq.setPipeSinkName(showPipeSinkStatement.getPipeSinkName());
@@ -1306,7 +1304,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   public SettableFuture<ConfigTaskResult> createPipe(CreatePipeStatement createPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TCreatePipeReq req =
           new TCreatePipeReq()
               .setPipeName(createPipeStatement.getPipeName())
@@ -1333,7 +1331,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   public SettableFuture<ConfigTaskResult> startPipe(StartPipeStatement startPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TSStatus tsStatus = configNodeClient.startPipe(startPipeStatement.getPipeName());
       if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) {
         LOGGER.warn(
@@ -1352,7 +1350,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   public SettableFuture<ConfigTaskResult> dropPipe(DropPipeStatement dropPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TSStatus tsStatus = configNodeClient.dropPipe(dropPipeStatement.getPipeName());
       if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) {
         LOGGER.warn(
@@ -1371,7 +1369,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   public SettableFuture<ConfigTaskResult> stopPipe(StopPipeStatement stopPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TSStatus tsStatus = configNodeClient.stopPipe(stopPipeStatement.getPipeName());
       if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) {
         LOGGER.warn(
@@ -1390,7 +1388,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   public SettableFuture<ConfigTaskResult> showPipe(ShowPipeStatement showPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TShowPipeReq tShowPipeReq = new TShowPipeReq();
       if (!StringUtils.isEmpty(showPipeStatement.getPipeName())) {
         tShowPipeReq.setPipeName(showPipeStatement.getPipeName());
@@ -1415,8 +1413,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
             queryId,
             serializePatternListToByteBuffer(deleteTimeSeriesStatement.getPathPatternList()));
     try (ConfigNodeClient client =
-        CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(
-            ConfigNodeInfo.configNodeRegionId)) {
+        CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TSStatus tsStatus;
       do {
         try {
@@ -1453,7 +1450,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     TGetRegionIdResp resp = new TGetRegionIdResp();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TGetRegionIdReq tGetRegionIdReq =
           new TGetRegionIdReq(
               getRegionIdStatement.getStorageGroup(), getRegionIdStatement.getPartitionType());
@@ -1485,7 +1482,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     TGetSeriesSlotListResp resp = new TGetSeriesSlotListResp();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TGetSeriesSlotListReq tGetSeriesSlotListReq =
           new TGetSeriesSlotListReq(getSeriesSlotListStatement.getStorageGroup());
       if (getSeriesSlotListStatement.getPartitionType() != null) {
@@ -1509,7 +1506,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     TGetTimeSlotListResp resp = new TGetTimeSlotListResp();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TGetTimeSlotListReq tGetTimeSlotListReq =
           new TGetTimeSlotListReq(
               getTimeSlotListStatement.getStorageGroup(),
@@ -1537,7 +1534,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
       MigrateRegionStatement migrateRegionStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TMigrateRegionReq tMigrateRegionReq =
           new TMigrateRegionReq(
               migrateRegionStatement.getRegionId(),
@@ -1567,7 +1564,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TCreateCQReq tCreateCQReq =
           new TCreateCQReq(
               createContinuousQueryStatement.getCqId(),
@@ -1601,7 +1598,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   public SettableFuture<ConfigTaskResult> dropContinuousQuery(String cqId) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       final TSStatus executionStatus = client.dropCQ(new TDropCQReq(cqId));
       if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != executionStatus.getCode()) {
         LOGGER.warn("[{}] Failed to drop continuous query {}.", executionStatus, cqId);
@@ -1619,7 +1616,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   public SettableFuture<ConfigTaskResult> showContinuousQueries() {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TShowCQResp showCQResp = client.showCQ();
       if (showCQResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
         future.setException(
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 69a609ac09..31e89349fe 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
@@ -237,7 +237,7 @@ public class DataNode implements DataNodeMBean {
     TSystemConfigurationResp configurationResp = null;
     while (retry > 0) {
       try (ConfigNodeClient configNodeClient =
-          ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         configurationResp = configNodeClient.getSystemConfiguration();
         break;
       } catch (TException | ClientManagerException e) {
@@ -349,7 +349,7 @@ public class DataNode implements DataNodeMBean {
     TDataNodeRegisterResp dataNodeRegisterResp = null;
     while (retry > 0) {
       try (ConfigNodeClient configNodeClient =
-          ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         dataNodeRegisterResp = configNodeClient.registerDataNode(req);
         break;
       } catch (TException | ClientManagerException e) {
@@ -408,7 +408,7 @@ public class DataNode implements DataNodeMBean {
     TDataNodeRestartResp dataNodeRestartResp = null;
     while (retry > 0) {
       try (ConfigNodeClient configNodeClient =
-          ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         dataNodeRestartResp = configNodeClient.restartDataNode(req);
         break;
       } catch (TException | ClientManagerException e) {
@@ -655,7 +655,7 @@ public class DataNode implements DataNodeMBean {
 
   private void getJarOfUDFs(List<UDFInformation> udfInformationList) throws StartupException {
     try (ConfigNodeClient configNodeClient =
-        ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       List<String> jarNameList =
           udfInformationList.stream().map(UDFInformation::getJarName).collect(Collectors.toList());
       TGetJarInListResp resp = configNodeClient.getUDFJar(new TGetJarInListReq(jarNameList));
@@ -765,7 +765,7 @@ public class DataNode implements DataNodeMBean {
   private void getJarOfTriggers(List<TriggerInformation> triggerInformationList)
       throws StartupException {
     try (ConfigNodeClient configNodeClient =
-        ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       List<String> jarNameList =
           triggerInformationList.stream()
               .map(TriggerInformation::getJarName)
diff --git a/server/src/main/java/org/apache/iotdb/db/service/DataNodeServerCommandLine.java b/server/src/main/java/org/apache/iotdb/db/service/DataNodeServerCommandLine.java
index a1a6d1621a..5fd060b971 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/DataNodeServerCommandLine.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/DataNodeServerCommandLine.java
@@ -112,7 +112,7 @@ public class DataNodeServerCommandLine extends ServerCommandLine {
     LOGGER.info("Start to remove datanode, removed datanode endpoints: {}", dataNodeLocations);
     TDataNodeRemoveReq removeReq = new TDataNodeRemoveReq(dataNodeLocations);
     try (ConfigNodeClient configNodeClient =
-        ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TDataNodeRemoveResp removeResp = configNodeClient.removeDataNode(removeReq);
       LOGGER.info("Remove result {} ", removeResp);
       if (removeResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
@@ -149,7 +149,7 @@ public class DataNodeServerCommandLine extends ServerCommandLine {
     try {
       List<TEndPoint> endPoints = NodeUrlUtils.parseTEndPointUrls(args);
       try (ConfigNodeClient client =
-          ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         dataNodeLocations =
             client.getDataNodeConfiguration(-1).getDataNodeConfigurationMap().values().stream()
                 .map(TDataNodeConfiguration::getLocation)
@@ -160,7 +160,7 @@ public class DataNodeServerCommandLine extends ServerCommandLine {
       }
     } catch (BadNodeUrlException e) {
       try (ConfigNodeClient client =
-          ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+          ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
         for (String id : args.split(",")) {
           if (!isNumeric(id)) {
             LOGGER.warn("Incorrect id format {}, skipped...", id);
diff --git a/server/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java b/server/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java
index 0da3cfb483..17d4662e6c 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java
@@ -98,7 +98,7 @@ public class IoTDBShutdownHook extends Thread {
     CommonDescriptor.getInstance().getConfig().setNodeStatus(NodeStatus.Unknown);
     boolean isReportSuccess = false;
     try (ConfigNodeClient client =
-        ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       isReportSuccess =
           client.reportDataNodeShutdown(DataNode.generateDataNodeLocation()).getCode()
               == TSStatusCode.SUCCESS_STATUS.getStatusCode();
diff --git a/server/src/main/java/org/apache/iotdb/db/service/RegionMigrateService.java b/server/src/main/java/org/apache/iotdb/db/service/RegionMigrateService.java
index b027be04ce..547fb78ee0 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/RegionMigrateService.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/RegionMigrateService.java
@@ -509,7 +509,7 @@ public class RegionMigrateService implements IService {
       throws TException, ClientManagerException {
     TSStatus status;
     try (ConfigNodeClient client =
-        ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       status = client.reportRegionMigrateResult(req);
       LOGGER.info(
           "{}, Report region {} migrate result {} to Config node succeed, result: {}",
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/common/ClusterSyncInfoFetcher.java b/server/src/main/java/org/apache/iotdb/db/sync/common/ClusterSyncInfoFetcher.java
index 10d4376d2c..6c4c6282a7 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/common/ClusterSyncInfoFetcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/common/ClusterSyncInfoFetcher.java
@@ -20,7 +20,7 @@ package org.apache.iotdb.db.sync.common;
 
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.client.IClientManager;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.exception.sync.PipeSinkException;
 import org.apache.iotdb.commons.sync.pipe.PipeInfo;
 import org.apache.iotdb.commons.sync.pipe.PipeMessage;
@@ -49,8 +49,8 @@ public class ClusterSyncInfoFetcher implements ISyncInfoFetcher {
 
   private static final Logger LOGGER = LoggerFactory.getLogger(ClusterSyncInfoFetcher.class);
 
-  private static final IClientManager<ConfigNodeRegionId, ConfigNodeClient>
-      CONFIG_NODE_CLIENT_MANAGER = ConfigNodeClientManager.getInstance();
+  private static final IClientManager<ConfigRegionId, ConfigNodeClient> CONFIG_NODE_CLIENT_MANAGER =
+      ConfigNodeClientManager.getInstance();
 
   // region Interfaces of PipeSink
 
@@ -67,7 +67,7 @@ public class ClusterSyncInfoFetcher implements ISyncInfoFetcher {
   @Override
   public PipeSink getPipeSink(String name) throws PipeSinkException {
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TGetPipeSinkReq tGetPipeSinkReq = new TGetPipeSinkReq().setPipeSinkName(name);
       TGetPipeSinkResp resp = configNodeClient.getPipeSink(tGetPipeSinkReq);
       if (resp.getPipeSinkInfoList().isEmpty()) {
@@ -113,7 +113,7 @@ public class ClusterSyncInfoFetcher implements ISyncInfoFetcher {
   @Override
   public List<PipeInfo> getAllPipeInfos() {
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TGetAllPipeInfoResp resp = configNodeClient.getAllPipeInfo();
       return resp.getAllPipeInfo().stream()
           .map(PipeInfo::deserializePipeInfo)
@@ -127,7 +127,7 @@ public class ClusterSyncInfoFetcher implements ISyncInfoFetcher {
   @Override
   public TSStatus recordMsg(String pipeName, PipeMessage message) {
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TRecordPipeMessageReq req =
           new TRecordPipeMessageReq(pipeName, message.serializeToByteBuffer());
       return configNodeClient.recordPipeMessage(req);
diff --git a/server/src/main/java/org/apache/iotdb/db/trigger/executor/TriggerFireVisitor.java b/server/src/main/java/org/apache/iotdb/db/trigger/executor/TriggerFireVisitor.java
index 7feb796061..03089cba77 100644
--- a/server/src/main/java/org/apache/iotdb/db/trigger/executor/TriggerFireVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/trigger/executor/TriggerFireVisitor.java
@@ -23,7 +23,7 @@ import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
 import org.apache.iotdb.commons.client.IClientManager;
 import org.apache.iotdb.commons.client.exception.ClientManagerException;
 import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.trigger.TriggerInformation;
 import org.apache.iotdb.commons.trigger.TriggerTable;
@@ -68,8 +68,8 @@ public class TriggerFireVisitor extends PlanVisitor<TriggerFireResult, TriggerEv
 
   private static final Logger LOGGER = LoggerFactory.getLogger(TriggerFireVisitor.class);
 
-  private static final IClientManager<ConfigNodeRegionId, ConfigNodeClient>
-      CONFIG_NODE_CLIENT_MANAGER = ConfigNodeClientManager.getInstance();
+  private static final IClientManager<ConfigRegionId, ConfigNodeClient> CONFIG_NODE_CLIENT_MANAGER =
+      ConfigNodeClientManager.getInstance();
 
   /**
    * How many times should we retry when error occurred during firing a trigger on another datanode
@@ -398,7 +398,7 @@ public class TriggerFireVisitor extends PlanVisitor<TriggerFireResult, TriggerEv
   /** Return true if the config node returns a new TDataNodeLocation */
   private boolean updateLocationOfStatefulTrigger(String triggerName, int currentDataNodeId) {
     try (ConfigNodeClient configNodeClient =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TDataNodeLocation newTDataNodeLocation =
           configNodeClient.getLocationOfStatefulTrigger(triggerName).getDataNodeLocation();
       if (newTDataNodeLocation != null
diff --git a/server/src/main/java/org/apache/iotdb/db/trigger/service/TriggerInformationUpdater.java b/server/src/main/java/org/apache/iotdb/db/trigger/service/TriggerInformationUpdater.java
index 46c79aed40..39cdeb3ae1 100644
--- a/server/src/main/java/org/apache/iotdb/db/trigger/service/TriggerInformationUpdater.java
+++ b/server/src/main/java/org/apache/iotdb/db/trigger/service/TriggerInformationUpdater.java
@@ -22,7 +22,7 @@ package org.apache.iotdb.db.trigger.service;
 import org.apache.iotdb.commons.client.IClientManager;
 import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil;
-import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
+import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.exception.IoTDBException;
 import org.apache.iotdb.commons.trigger.TriggerInformation;
 import org.apache.iotdb.confignode.rpc.thrift.TGetTriggerTableResp;
@@ -44,8 +44,8 @@ public class TriggerInformationUpdater {
 
   private static final Logger LOGGER = LoggerFactory.getLogger(TriggerInformationUpdater.class);
 
-  private static final IClientManager<ConfigNodeRegionId, ConfigNodeClient>
-      CONFIG_NODE_CLIENT_MANAGER = ConfigNodeClientManager.getInstance();
+  private static final IClientManager<ConfigRegionId, ConfigNodeClient> CONFIG_NODE_CLIENT_MANAGER =
+      ConfigNodeClientManager.getInstance();
 
   private final ScheduledExecutorService triggerInformationUpdateExecutor =
       IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor(
@@ -78,7 +78,7 @@ public class TriggerInformationUpdater {
 
   public void updateTask() {
     try (ConfigNodeClient client =
-        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TGetTriggerTableResp getStatefulTriggerTableResp = client.getStatefulTriggerTable();
       if (getStatefulTriggerTableResp.getStatus().getCode()
           != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
diff --git a/thrift-commons/src/main/thrift/common.thrift b/thrift-commons/src/main/thrift/common.thrift
index 489d0cd016..994316271f 100644
--- a/thrift-commons/src/main/thrift/common.thrift
+++ b/thrift-commons/src/main/thrift/common.thrift
@@ -36,7 +36,7 @@ struct TSStatus {
 }
 
 enum TConsensusGroupType {
-  ConfigNodeRegion,
+  ConfigRegion,
   DataRegion,
   SchemaRegion
 }