You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ca...@apache.org on 2023/01/04 14:07:05 UTC

[iotdb] branch rel/1.0 updated: [To rel/1.0] Cherry pick cluster name related commits and SQL SHOW VARIABLES (#8745)

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

caogaofei pushed a commit to branch rel/1.0
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/rel/1.0 by this push:
     new 6c25f43337 [To rel/1.0] Cherry pick cluster name related commits and SQL SHOW VARIABLES (#8745)
6c25f43337 is described below

commit 6c25f4333796747f6ba117ac19295f9a3fa640e5
Author: YongzaoDan <33...@users.noreply.github.com>
AuthorDate: Wed Jan 4 22:06:57 2023 +0800

    [To rel/1.0] Cherry pick cluster name related commits and SQL SHOW VARIABLES (#8745)
---
 .../org/apache/iotdb/db/qp/sql/IdentifierParser.g4 |  1 -
 .../org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4   |  9 +++-
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4  |  4 +-
 .../confignode/conf/ConfigNodeDescriptor.java      |  3 ++
 .../confignode/conf/SystemPropertiesUtils.java     |  9 +++-
 .../consensus/response/DataNodeRegisterResp.java   |  7 ---
 .../iotdb/confignode/manager/ConfigManager.java    | 24 +++++----
 .../apache/iotdb/confignode/manager/IManager.java  | 12 ++---
 .../manager/node/ClusterNodeStartUtils.java        | 25 ++++++++-
 .../iotdb/confignode/manager/node/NodeManager.java |  2 -
 .../iotdb/confignode/service/ConfigNode.java       | 26 +++++-----
 .../thrift/ConfigNodeRPCServiceProcessor.java      | 15 +++---
 docs/UserGuide/Cluster/Cluster-Maintenance.md      | 10 ++--
 docs/UserGuide/Cluster/Cluster-Setup.md            | 60 +++++++++++++++++-----
 docs/zh/UserGuide/Cluster/Cluster-Maintenance.md   |  8 +--
 docs/zh/UserGuide/Cluster/Cluster-Setup.md         | 42 ++++++++++++---
 .../apache/iotdb/it/env/AbstractNodeWrapper.java   |  2 +-
 .../it/cluster/IoTDBClusterNodeErrorStartUpIT.java | 35 +++++++++++++
 .../it/cluster/IoTDBClusterNodeGetterIT.java       |  9 ++--
 .../it/partition/IoTDBPartitionGetterIT.java       |  2 +-
 .../confignode/it/utils/ConfigNodeTestUtils.java   |  3 +-
 .../resources/conf/iotdb-common.properties         |  8 +++
 .../apache/iotdb/commons/conf/IoTDBConstant.java   | 25 ++-------
 .../apache/iotdb/db/client/ConfigNodeClient.java   |  6 +--
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |  3 ++
 .../org/apache/iotdb/db/conf/IoTDBStartCheck.java  |  8 ++-
 .../db/mpp/common/header/ColumnHeaderConstant.java | 26 ++++++++--
 .../db/mpp/common/header/DatasetHeaderFactory.java |  2 +-
 .../plan/execution/config/ConfigTaskVisitor.java   | 11 ++--
 .../config/executor/ClusterConfigTaskExecutor.java | 10 ++--
 .../config/metadata/ShowClusterDetailsTask.java    |  4 +-
 .../execution/config/metadata/ShowClusterTask.java |  4 +-
 ...rParametersTask.java => ShowVariablesTask.java} | 44 ++++++++--------
 .../iotdb/db/mpp/plan/parser/ASTVisitor.java       |  8 ++-
 .../db/mpp/plan/statement/StatementVisitor.java    |  5 ++
 .../statement/metadata/ShowClusterStatement.java   |  9 ----
 ...rStatement.java => ShowVariablesStatement.java} | 23 +--------
 .../java/org/apache/iotdb/db/service/DataNode.java | 12 ++---
 .../src/main/thrift/confignode.thrift              | 37 ++++++++-----
 39 files changed, 337 insertions(+), 216 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4
index db906047c5..3aea385462 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4
@@ -194,5 +194,4 @@ keyWords
     | WITHOUT
     | WRITABLE
     | PRIVILEGE_VALUE
-    | PARAMETERS
     ;
\ No newline at end of file
diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
index 6c0f6959b8..f0687178a3 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
@@ -42,7 +42,7 @@ ddlStatement
     | dropFunction | dropTrigger | dropContinuousQuery | dropSchemaTemplate
     | setTTL | unsetTTL | startTrigger | stopTrigger | setSchemaTemplate | unsetSchemaTemplate
     | showStorageGroup | showDevices | showTimeseries | showChildPaths | showChildNodes
-    | showFunctions | showTriggers | showContinuousQueries | showTTL | showAllTTL | showCluster | showRegion | showDataNodes | showConfigNodes
+    | showFunctions | showTriggers | showContinuousQueries | showTTL | showAllTTL | showCluster | showVariables | showRegion | showDataNodes | showConfigNodes
     | showSchemaTemplates | showNodesInSchemaTemplate
     | showPathsUsingSchemaTemplate | showPathsSetSchemaTemplate
     | countStorageGroup | countDevices | countTimeseries | countNodes
@@ -334,9 +334,14 @@ showAllTTL
     : SHOW ALL TTL
     ;
 
+// Show Variables
+showVariables
+    : SHOW VARIABLES
+    ;
+
 // Show Cluster
 showCluster
-    : SHOW CLUSTER (DETAILS | PARAMETERS)?
+    : SHOW CLUSTER (DETAILS)?
     ;
 
 // Show Region
diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
index 629f7e0c39..8f66867ff9 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
@@ -121,8 +121,8 @@ CLUSTER
     : C L U S T E R
     ;
 
-PARAMETERS
-    : P A R A M E T E R S
+VARIABLES
+    : V A R I A B L E S
     ;
 
 CONCAT
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
index c78f69cb39..63d30fc6cd 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
@@ -143,6 +143,9 @@ public class ConfigNodeDescriptor {
   }
 
   private void loadProperties(Properties properties) throws BadNodeUrlException, IOException {
+    conf.setClusterName(
+        properties.getProperty(IoTDBConstant.CLUSTER_NAME, conf.getClusterName()).trim());
+
     conf.setInternalAddress(
         properties
             .getProperty(IoTDBConstant.CN_INTERNAL_ADDRESS, conf.getInternalAddress())
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java b/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java
index a01c018d86..3fcc24f5c4 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java
@@ -35,8 +35,8 @@ import java.util.List;
 import java.util.Objects;
 import java.util.Properties;
 
-import static org.apache.iotdb.db.conf.IoTDBStartCheck.CLUSTER_NAME;
-import static org.apache.iotdb.db.conf.IoTDBStartCheck.DEFAULT_CLUSTER_NAME;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.CLUSTER_NAME;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.DEFAULT_CLUSTER_NAME;
 
 public class SystemPropertiesUtils {
 
@@ -71,6 +71,11 @@ public class SystemPropertiesUtils {
     boolean needReWrite = false;
 
     // Startup configuration
+    String clusterName = systemProperties.getProperty(CLUSTER_NAME, null);
+    if (clusterName != null && !clusterName.equals(conf.getClusterName())) {
+      throw new ConfigurationException(CLUSTER_NAME, conf.getClusterName(), clusterName);
+    }
+
     String internalAddress = systemProperties.getProperty("cn_internal_address", null);
     if (internalAddress == null) {
       needReWrite = true;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeRegisterResp.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeRegisterResp.java
index e3ab67070c..a6e8d9d713 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeRegisterResp.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeRegisterResp.java
@@ -35,8 +35,6 @@ public class DataNodeRegisterResp implements DataSet {
 
   private TSStatus status;
   private List<TConfigNodeLocation> configNodeList;
-
-  private String clusterName;
   private Integer dataNodeId;
 
   private TRuntimeConfiguration runtimeConfiguration;
@@ -57,10 +55,6 @@ public class DataNodeRegisterResp implements DataSet {
     this.configNodeList = configNodeList;
   }
 
-  public void setClusterName(String clusterName) {
-    this.clusterName = clusterName;
-  }
-
   public void setDataNodeId(Integer dataNodeId) {
     this.dataNodeId = dataNodeId;
   }
@@ -89,7 +83,6 @@ public class DataNodeRegisterResp implements DataSet {
     resp.setConfigNodeList(configNodeList);
 
     if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
-      resp.setClusterName(clusterName);
       resp.setDataNodeId(dataNodeId);
       resp.setRuntimeConfiguration(runtimeConfiguration);
     }
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 b765b81675..ff80e81a5c 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
@@ -105,6 +105,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TCreateFunctionReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCreateSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCreateTriggerReq;
+import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRestartReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRestartResp;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionTableResp;
@@ -135,13 +136,13 @@ import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionTableResp;
 import org.apache.iotdb.confignode.rpc.thrift.TSetDataNodeStatusReq;
 import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TShowCQResp;
-import org.apache.iotdb.confignode.rpc.thrift.TShowClusterParametersResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowConfigNodesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowPipeReq;
 import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowStorageGroupResp;
+import org.apache.iotdb.confignode.rpc.thrift.TShowVariablesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.confignode.rpc.thrift.TTimeSlotList;
 import org.apache.iotdb.confignode.rpc.thrift.TUnsetSchemaTemplateReq;
@@ -285,16 +286,18 @@ public class ConfigManager implements IManager {
   }
 
   @Override
-  public DataSet registerDataNode(RegisterDataNodePlan registerDataNodePlan) {
+  public DataSet registerDataNode(TDataNodeRegisterReq req) {
     TSStatus status = confirmLeader();
     if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
       status =
           ClusterNodeStartUtils.confirmNodeRegistration(
               NodeType.DataNode,
-              registerDataNodePlan.getDataNodeConfiguration().getLocation(),
+              req.getClusterName(),
+              req.getDataNodeConfiguration().getLocation(),
               this);
       if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
-        return nodeManager.registerDataNode(registerDataNodePlan);
+        return nodeManager.registerDataNode(
+            new RegisterDataNodePlan(req.getDataNodeConfiguration()));
       }
     }
 
@@ -401,9 +404,9 @@ public class ConfigManager implements IManager {
   }
 
   @Override
-  public TShowClusterParametersResp showClusterParameters() {
+  public TShowVariablesResp showVariables() {
     TSStatus status = confirmLeader();
-    TShowClusterParametersResp resp = new TShowClusterParametersResp();
+    TShowVariablesResp resp = new TShowVariablesResp();
     resp.setStatus(status);
     if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
       resp.setClusterParameters(getClusterParameters());
@@ -827,7 +830,10 @@ public class ConfigManager implements IManager {
       if (status == null) {
         status =
             ClusterNodeStartUtils.confirmNodeRegistration(
-                NodeType.ConfigNode, req.getConfigNodeLocation(), this);
+                NodeType.ConfigNode,
+                req.getClusterParameters().getClusterName(),
+                req.getConfigNodeLocation(),
+                this);
         if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
           return nodeManager.registerConfigNode(req);
         }
@@ -865,10 +871,6 @@ public class ConfigManager implements IManager {
     TSStatus errorStatus = new TSStatus(TSStatusCode.CONFIGURATION_ERROR.getStatusCode());
     TClusterParameters clusterParameters = req.getClusterParameters();
 
-    if (!clusterParameters.getClusterName().equals(CONF.getClusterName())) {
-      return errorStatus.setMessage(errorPrefix + "cluster_name" + errorSuffix);
-    }
-
     if (!clusterParameters
         .getConfigNodeConsensusProtocolClass()
         .equals(CONF.getConfigNodeConsensusProtocolClass())) {
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
index 490b512091..954da3a43f 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
@@ -35,7 +35,6 @@ import org.apache.iotdb.confignode.consensus.request.read.region.GetRegionInfoLi
 import org.apache.iotdb.confignode.consensus.request.read.storagegroup.CountStorageGroupPlan;
 import org.apache.iotdb.confignode.consensus.request.read.storagegroup.GetStorageGroupPlan;
 import org.apache.iotdb.confignode.consensus.request.write.confignode.RemoveConfigNodePlan;
-import org.apache.iotdb.confignode.consensus.request.write.datanode.RegisterDataNodePlan;
 import org.apache.iotdb.confignode.consensus.request.write.datanode.RemoveDataNodePlan;
 import org.apache.iotdb.confignode.consensus.request.write.datanode.UpdateDataNodePlan;
 import org.apache.iotdb.confignode.consensus.request.write.storagegroup.SetDataReplicationFactorPlan;
@@ -57,6 +56,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TCreateFunctionReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCreateSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCreateTriggerReq;
+import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRestartReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRestartResp;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionTableResp;
@@ -87,13 +87,13 @@ import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionTableResp;
 import org.apache.iotdb.confignode.rpc.thrift.TSetDataNodeStatusReq;
 import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TShowCQResp;
-import org.apache.iotdb.confignode.rpc.thrift.TShowClusterParametersResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowConfigNodesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowPipeReq;
 import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowStorageGroupResp;
+import org.apache.iotdb.confignode.rpc.thrift.TShowVariablesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TUnsetSchemaTemplateReq;
 import org.apache.iotdb.consensus.common.DataSet;
 
@@ -187,7 +187,7 @@ public interface IManager {
    *
    * @return DataNodeConfigurationDataSet
    */
-  DataSet registerDataNode(RegisterDataNodePlan registerDataNodePlan);
+  DataSet registerDataNode(TDataNodeRegisterReq req);
 
   /**
    * Restart DataNode
@@ -236,11 +236,11 @@ public interface IManager {
   TShowClusterResp showCluster();
 
   /**
-   * Get Cluster parameters
+   * Get variables
    *
-   * @return TShowClusterParametersResp
+   * @return TShowVariablesResp
    */
-  TShowClusterParametersResp showClusterParameters();
+  TShowVariablesResp showVariables();
 
   TSStatus setTTL(SetTTLPlan configRequest);
 
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/ClusterNodeStartUtils.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/ClusterNodeStartUtils.java
index fde4479c4b..d1a118a9f2 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/ClusterNodeStartUtils.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/ClusterNodeStartUtils.java
@@ -52,7 +52,7 @@ public class ClusterNodeStartUtils {
       new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()).setMessage("Accept Node restart.");
 
   public static TSStatus confirmNodeRegistration(
-      NodeType nodeType, Object nodeLocation, ConfigManager configManager) {
+      NodeType nodeType, String clusterName, Object nodeLocation, ConfigManager configManager) {
 
     final String CONF_FILE_NAME =
         NodeType.ConfigNode.equals(nodeType)
@@ -60,6 +60,25 @@ public class ClusterNodeStartUtils {
             : IoTDBConstant.DATA_NODE_CONF_FILE_NAME;
     TSStatus status = new TSStatus();
 
+    /* Reject start if the cluster name is error */
+    if (!CLUSTER_NAME.equals(clusterName)) {
+      status.setCode(TSStatusCode.REJECT_NODE_START.getStatusCode());
+      status.setMessage(
+          String.format(
+              "Reject %s start. Because the ClusterName of the current %s and the target cluster are inconsistent. "
+                  + "ClusterName of the current Node: %s, ClusterName of the target cluster: %s."
+                  + POSSIBLE_SOLUTIONS
+                  + "\t1. Change the target_config_node_list parameter in %s to join the correct cluster."
+                  + "\n\t2. Change the cluster_name parameter in %s to match the target cluster",
+              nodeType.getNodeType(),
+              nodeType.getNodeType(),
+              clusterName,
+              CLUSTER_NAME,
+              CONF_FILE_NAME,
+              CONF_FILE_NAME));
+      return status;
+    }
+
     /* Check if there exist conflict TEndPoints */
     List<TEndPoint> conflictEndPoints;
     switch (nodeType) {
@@ -119,11 +138,13 @@ public class ClusterNodeStartUtils {
               "Reject %s restart. Because the ClusterName of the current %s and the target cluster are inconsistent. "
                   + "ClusterName of the current Node: %s, ClusterName of the target cluster: %s."
                   + POSSIBLE_SOLUTIONS
-                  + "\t1. Change the target_config_node_list parameter in %s to join the correct cluster.",
+                  + "\t1. Change the target_config_node_list parameter in %s to join the correct cluster."
+                  + "\n\t2. Change the cluster_name parameter in %s to match the target cluster",
               nodeType.getNodeType(),
               nodeType.getNodeType(),
               clusterName,
               CLUSTER_NAME,
+              CONF_FILE_NAME,
               CONF_FILE_NAME));
       return status;
     }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
index c64e374863..26b11b5320 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
@@ -281,7 +281,6 @@ public class NodeManager {
 
     resp.setStatus(ClusterNodeStartUtils.ACCEPT_NODE_REGISTRATION);
     resp.setConfigNodeList(getRegisteredConfigNodes());
-    resp.setClusterName(CONF.getClusterName());
     resp.setDataNodeId(
         registerDataNodePlan.getDataNodeConfiguration().getLocation().getDataNodeId());
     resp.setRuntimeConfiguration(getRuntimeConfiguration());
@@ -396,7 +395,6 @@ public class NodeManager {
     configManager.getProcedureManager().addConfigNode(req);
     return new TConfigNodeRegisterResp()
         .setStatus(ClusterNodeStartUtils.ACCEPT_NODE_REGISTRATION)
-        .setClusterName(CONF.getClusterName())
         .setConfigNodeId(nodeId);
   }
 
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java
index 397ede2a01..f7318c9969 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java
@@ -95,7 +95,7 @@ public class ConfigNode implements ConfigNodeMBean {
       if (SystemPropertiesUtils.isRestarted()) {
         LOGGER.info("{} is in restarting process...", ConfigNodeConstant.GLOBAL_NAME);
 
-        /* Always restore ClusterId and ConfigNodeId first */
+        /* Always restore ClusterName and ConfigNodeId first */
         CONF.setClusterName(SystemPropertiesUtils.loadClusterNameWhenRestarted());
         CONF.setConfigNodeId(SystemPropertiesUtils.loadConfigNodeIdWhenRestarted());
 
@@ -107,8 +107,9 @@ public class ConfigNode implements ConfigNodeMBean {
         configManager.initConsensusManager();
         setUpRPCService();
         LOGGER.info(
-            "{} has successfully restarted and joined the cluster.",
-            ConfigNodeConstant.GLOBAL_NAME);
+            "{} has successfully restarted and joined the cluster: {}.",
+            ConfigNodeConstant.GLOBAL_NAME,
+            CONF.getClusterName());
         return;
       }
 
@@ -140,7 +141,9 @@ public class ConfigNode implements ConfigNodeMBean {
         // The initial startup of Seed-ConfigNode finished
 
         LOGGER.info(
-            "{} has successfully started and joined the cluster.", ConfigNodeConstant.GLOBAL_NAME);
+            "{} has successfully started and joined the cluster: {}.",
+            ConfigNodeConstant.GLOBAL_NAME,
+            CONF.getClusterName());
         return;
       }
 
@@ -152,9 +155,10 @@ public class ConfigNode implements ConfigNodeMBean {
       // The initial startup of Non-Seed-ConfigNode is not yet finished,
       // we should wait for leader's scheduling
       LOGGER.info(
-          "{} {} has registered successfully. Waiting for the leader's scheduling to join the cluster.",
+          "{} {} has registered successfully. Waiting for the leader's scheduling to join the cluster: {}.",
           ConfigNodeConstant.GLOBAL_NAME,
-          CONF.getConfigNodeId());
+          CONF.getConfigNodeId(),
+          CONF.getClusterName());
 
       boolean isJoinedCluster = false;
       for (int retry = 0; retry < SCHEDULE_WAITING_RETRY_NUM; retry++) {
@@ -271,14 +275,8 @@ public class ConfigNode implements ConfigNodeMBean {
       } else if (status.getCode() == TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) {
         targetConfigNode = status.getRedirectNode();
         LOGGER.info("ConfigNode need redirect to  {}.", targetConfigNode);
-      } else if (status.getCode() == TSStatusCode.CONFIGURATION_ERROR.getStatusCode()) {
-        LOGGER.error(status.getMessage());
-        throw new StartupException("Configuration are not consistent!");
-      } else if (status.getCode() == TSStatusCode.CONSENSUS_NOT_INITIALIZED.getStatusCode()) {
-        LOGGER.error(status.getMessage());
-        throw new StartupException(
-            "The target ConfigNode is not started successfully, "
-                + "please check the cn_target_config_node_list config!");
+      } else {
+        throw new StartupException(status.getMessage());
       }
 
       try {
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 7daf3d6eda..9673660879 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
@@ -46,7 +46,6 @@ import org.apache.iotdb.confignode.consensus.request.read.region.GetRegionInfoLi
 import org.apache.iotdb.confignode.consensus.request.read.storagegroup.CountStorageGroupPlan;
 import org.apache.iotdb.confignode.consensus.request.read.storagegroup.GetStorageGroupPlan;
 import org.apache.iotdb.confignode.consensus.request.write.confignode.RemoveConfigNodePlan;
-import org.apache.iotdb.confignode.consensus.request.write.datanode.RegisterDataNodePlan;
 import org.apache.iotdb.confignode.consensus.request.write.datanode.RemoveDataNodePlan;
 import org.apache.iotdb.confignode.consensus.request.write.datanode.UpdateDataNodePlan;
 import org.apache.iotdb.confignode.consensus.request.write.storagegroup.SetDataReplicationFactorPlan;
@@ -132,7 +131,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TSetStorageGroupReq;
 import org.apache.iotdb.confignode.rpc.thrift.TSetTimePartitionIntervalReq;
 import org.apache.iotdb.confignode.rpc.thrift.TShowCQResp;
-import org.apache.iotdb.confignode.rpc.thrift.TShowClusterParametersResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowConfigNodesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodesResp;
@@ -141,6 +139,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowRegionReq;
 import org.apache.iotdb.confignode.rpc.thrift.TShowRegionResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowStorageGroupResp;
+import org.apache.iotdb.confignode.rpc.thrift.TShowVariablesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchemaResp;
 import org.apache.iotdb.confignode.rpc.thrift.TSystemConfigurationResp;
@@ -195,9 +194,7 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
   @Override
   public TDataNodeRegisterResp registerDataNode(TDataNodeRegisterReq req) {
     TDataNodeRegisterResp resp =
-        ((DataNodeRegisterResp)
-                configManager.registerDataNode(
-                    new RegisterDataNodePlan(req.getDataNodeConfiguration())))
+        ((DataNodeRegisterResp) configManager.registerDataNode(req))
             .convertToRpcDataNodeRegisterResp();
 
     // Print log to record the ConfigNode that performs the RegisterDatanodeRequest
@@ -262,8 +259,8 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
   }
 
   @Override
-  public TShowClusterParametersResp showClusterParameters() throws TException {
-    return configManager.showClusterParameters();
+  public TShowVariablesResp showVariables() throws TException {
+    return configManager.showVariables();
   }
 
   @Override
@@ -483,7 +480,9 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
 
     // The initial startup of Non-Seed-ConfigNode finished
     LOGGER.info(
-        "{} has successfully started and joined the cluster.", ConfigNodeConstant.GLOBAL_NAME);
+        "{} has successfully started and joined the cluster: {}.",
+        ConfigNodeConstant.GLOBAL_NAME,
+        ConfigNodeDescriptor.getInstance().getConf().getClusterName());
     return StatusUtils.OK;
   }
 
diff --git a/docs/UserGuide/Cluster/Cluster-Maintenance.md b/docs/UserGuide/Cluster/Cluster-Maintenance.md
index 38911edc20..dcd8999f78 100644
--- a/docs/UserGuide/Cluster/Cluster-Maintenance.md
+++ b/docs/UserGuide/Cluster/Cluster-Maintenance.md
@@ -21,18 +21,18 @@
 
 # Cluster Information Query Command
 
-## Show Cluster Parameters
+## Show Variables
 
 Currently, IoTDB supports showing key parameters of the cluster:
 ```
-SHOW CLUSTER PARAMETERS
+SHOW VARIABLES
 ```
 
 Eg:
 ```
-IoTDB> show cluster parameters
+IoTDB> show variables
 +----------------------------------+-----------------------------------------------------------------+
-|                 ClusterParameters|                                                            Value|
+|                          Variable|                                                            Value|
 +----------------------------------+-----------------------------------------------------------------+
 |                       ClusterName|                                                   defaultCluster|
 |             DataReplicationFactor|                                                                1|
@@ -51,7 +51,7 @@ IoTDB> show cluster parameters
 |         DiskSpaceWarningThreshold|                                                             0.05|
 +----------------------------------+-----------------------------------------------------------------+
 Total line number = 15
-It costs 0.242s
+It costs 0.225s
 ```
 
 **Notice:** Ensure that all key parameters displayed in this SQL are consist on each node in the same cluster
diff --git a/docs/UserGuide/Cluster/Cluster-Setup.md b/docs/UserGuide/Cluster/Cluster-Setup.md
index b68838eab7..1bc33b56f5 100644
--- a/docs/UserGuide/Cluster/Cluster-Setup.md
+++ b/docs/UserGuide/Cluster/Cluster-Setup.md
@@ -113,6 +113,7 @@ and set the following parameters base on the
 
 | **Configuration**                          | **Description**                                                                                                    | **Default**                                     |
 |--------------------------------------------|--------------------------------------------------------------------------------------------------------------------|-------------------------------------------------|
+| cluster\_name                              | Cluster name for which the Node to join in                                                                         | defaultCluster                                  |
 | config\_node\_consensus\_protocol\_class   | Consensus protocol of ConfigNode                                                                                   | org.apache.iotdb.consensus.ratis.RatisConsensus |
 | schema\_replication\_factor                | Schema replication factor, no more than DataNode number                                                            | 1                                               |
 | schema\_region\_consensus\_protocol\_class | Consensus protocol of schema replicas                                                                              | org.apache.iotdb.consensus.ratis.RatisConsensus |
@@ -140,16 +141,16 @@ and set the following parameters based on the IP address and available port of t
 Open the DataNode configuration file ./conf/iotdb-datanode.properties,
 and set the following parameters based on the IP address and available port of the server or VM:
 
-| **Configuration**                   | **Description**                                  | **Default**     | **Usage**                                                                                                                             |
-|-------------------------------------|--------------------------------------------------|-----------------|---------------------------------------------------------------------------------------------------------------------------------------|
-| dn\_rpc\_address                    | Client RPC Service address                       | 127.0.0.1       | Set to the IPV4 address or domain name of the server                                                                                  |
-| dn\_rpc\_port                       | Client RPC Service port                          | 6667            | Set to any unoccupied port                                                                                                            |
-| dn\_internal\_address               | Control flow address of DataNode inside cluster  | 127.0.0.1       | Set to the IPV4 address or domain name of the server                                                                                  |
-| dn\_internal\_port                  | Control flow port of DataNode inside cluster     | 10730            | Set to any unoccupied port                                                                                                            |
-| dn\_mpp\_data\_exchange\_port       | Data flow port of DataNode inside cluster        | 10740            | Set to any unoccupied port                                                                                                            |
-| dn\_data\_region\_consensus\_port   | Data replicas communication port for consensus   | 10750           | Set to any unoccupied port                                                                                                            |
-| dn\_schema\_region\_consensus\_port | Schema replicas communication port for consensus | 10760           | Set to any unoccupied port                                                                                                            |
-| dn\_target\_config\_node\_list      | Running ConfigNode of the Cluster                | 127.0.0.1:10710 | Set to any running ConfigNode's cn\_internal\_address:cn\_internal\_port. You can set multiple values, separate them with commas(",") |
+| **Configuration**                   | **Description**                                        | **Default**     | **Usage**                                                                                                                             |
+|-------------------------------------|--------------------------------------------------------|-----------------|---------------------------------------------------------------------------------------------------------------------------------------|
+| dn\_rpc\_address                    | Client RPC Service address                             | 127.0.0.1       | Set to the IPV4 address or domain name of the server                                                                                  |
+| dn\_rpc\_port                       | Client RPC Service port                                | 6667            | Set to any unoccupied port                                                                                                            |
+| dn\_internal\_address               | Control flow address of DataNode inside cluster        | 127.0.0.1       | Set to the IPV4 address or domain name of the server                                                                                  |
+| dn\_internal\_port                  | Control flow port of DataNode inside cluster           | 10730           | Set to any unoccupied port                                                                                                            |
+| dn\_mpp\_data\_exchange\_port       | Data flow port of DataNode inside cluster              | 10740           | Set to any unoccupied port                                                                                                            |
+| dn\_data\_region\_consensus\_port   | Data replicas communication port for consensus         | 10750           | Set to any unoccupied port                                                                                                            |
+| dn\_schema\_region\_consensus\_port | Schema replicas communication port for consensus       | 10760           | Set to any unoccupied port                                                                                                            |
+| dn\_target\_config\_node\_list      | Running ConfigNode of the Cluster                      | 127.0.0.1:10710 | Set to any running ConfigNode's cn\_internal\_address:cn\_internal\_port. You can set multiple values, separate them with commas(",") |
 
 **Notice: The preceding configuration parameters cannot be changed after the node is started. Ensure that all ports are not occupied. Otherwise, the Node cannot be started.**
 
@@ -172,7 +173,21 @@ The total process are three steps:
 **The first Node started in the cluster must be ConfigNode. The first started ConfigNode must follow the tutorial in this section.**
 
 The first ConfigNode to start is the Seed-ConfigNode, which marks the creation of the new cluster.
-Before start the Seed-ConfigNode, please open its configuration file ./conf/iotdb-confignode.properties and check the following parameters:
+Before start the Seed-ConfigNode, please open the common configuration file ./conf/iotdb-common.properties and check the following parameters:
+
+| **Configuration**                          | **Check**                                       |
+|--------------------------------------------|-------------------------------------------------|
+| cluster\_name                              | Is set to the expected name                     |
+| config\_node\_consensus\_protocol\_class   | Is set to the expected consensus protocol       |
+| schema\_replication\_factor                | Is set to the expected schema replication count |
+| schema\_region\_consensus\_protocol\_class | Is set to the expected consensus protocol       |
+| data\_replication\_factor                  | Is set to the expected data replication count   |
+| data\_region\_consensus\_protocol\_class   | Is set to the expected consensus protocol       |
+
+**Notice:** Please set these parameters carefully based on the [Deployment Recommendation](https://iotdb.apache.org/UserGuide/Master/Cluster/Deployment-Recommendation.html).
+These parameters are not modifiable after the Node first startup.
+
+Then open its configuration file ./conf/iotdb-confignode.properties and check the following parameters:
 
 | **Configuration**              | **Check**                                                                                           |
 |--------------------------------|-----------------------------------------------------------------------------------------------------|
@@ -206,8 +221,18 @@ A common configuration is to add extra two ConfigNodes to make the cluster has t
 
 Ensure that all configuration parameters in the ./conf/iotdb-common.properites are the same as those in the Seed-ConfigNode;
 otherwise, it may fail to start or generate runtime errors.
+Therefore, please check the following parameters in common configuration file:
+
+| **Configuration**                          | **Check**                              |
+|--------------------------------------------|----------------------------------------|
+| cluster\_name                              | Is consistent with the Seed-ConfigNode |
+| config\_node\_consensus\_protocol\_class   | Is consistent with the Seed-ConfigNode |
+| schema\_replication\_factor                | Is consistent with the Seed-ConfigNode |
+| schema\_region\_consensus\_protocol\_class | Is consistent with the Seed-ConfigNode |
+| data\_replication\_factor                  | Is consistent with the Seed-ConfigNode |
+| data\_region\_consensus\_protocol\_class   | Is consistent with the Seed-ConfigNode |
 
-Before start the new ConfigNode, please open its configuration file ./conf/iotdb-confignode.properties and check the following parameters:
+Then, please open its configuration file ./conf/iotdb-confignode.properties and check the following parameters:
 
 | **Configuration**              | **Check**                                                                                                                                              |
 |--------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------|
@@ -237,8 +262,15 @@ For more details about other configuration parameters of ConfigNode, see the
 **Before adding DataNodes, ensure that there exists at least one ConfigNode is running in the cluster.**
 
 You can add any number of DataNodes to the cluster.
-Before adding a new DataNode, please
-open its configuration file ./conf/iotdb-datanode.properties and check the following parameters:
+Before adding a new DataNode, 
+
+please open its common configuration file ./conf/iotdb-common.properties and check the following parameters:
+
+| **Configuration**                          | **Check**                              |
+|--------------------------------------------|----------------------------------------|
+| cluster\_name                              | Is consistent with the Seed-ConfigNode |
+
+Then open its configuration file ./conf/iotdb-datanode.properties and check the following parameters:
 
 | **Configuration**                   | **Check**                                                                                                                                            |
 |-------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------|
diff --git a/docs/zh/UserGuide/Cluster/Cluster-Maintenance.md b/docs/zh/UserGuide/Cluster/Cluster-Maintenance.md
index a55a94ac75..a69785d586 100644
--- a/docs/zh/UserGuide/Cluster/Cluster-Maintenance.md
+++ b/docs/zh/UserGuide/Cluster/Cluster-Maintenance.md
@@ -25,14 +25,14 @@
 
 当前 IoTDB 支持使用如下 SQL 展示集群的关键参数:
 ```
-SHOW CLUSTER PARAMETERS
+SHOW VARIABLES
 ```
 
 示例:
 ```
-IoTDB> show cluster parameters
+IoTDB> show variables
 +----------------------------------+-----------------------------------------------------------------+
-|                 ClusterParameters|                                                            Value|
+|                          Variable|                                                            Value|
 +----------------------------------+-----------------------------------------------------------------+
 |                       ClusterName|                                                   defaultCluster|
 |             DataReplicationFactor|                                                                1|
@@ -51,7 +51,7 @@ IoTDB> show cluster parameters
 |         DiskSpaceWarningThreshold|                                                             0.05|
 +----------------------------------+-----------------------------------------------------------------+
 Total line number = 15
-It costs 0.242s
+It costs 0.225s
 ```
 
 **注意:** 必须保证该 SQL 展示的所有配置参数在同一集群各个节点完全一致
diff --git a/docs/zh/UserGuide/Cluster/Cluster-Setup.md b/docs/zh/UserGuide/Cluster/Cluster-Setup.md
index 3800b0ecf2..949ade85e4 100644
--- a/docs/zh/UserGuide/Cluster/Cluster-Setup.md
+++ b/docs/zh/UserGuide/Cluster/Cluster-Setup.md
@@ -111,6 +111,7 @@ mvn clean package -pl distribution -am -DskipTests
 
 | **配置项**                                    | **说明**                                 | **默认**                                          |
 |--------------------------------------------|----------------------------------------|-------------------------------------------------|
+| cluster\_name                              | 节点希望加入的集群的名称                           | defaultCluster                                  |
 | config\_node\_consensus\_protocol\_class   | ConfigNode 使用的共识协议                     | org.apache.iotdb.consensus.ratis.RatisConsensus |
 | schema\_replication\_factor                | 元数据副本数,DataNode 数量不应少于此数目              | 1                                               |
 | schema\_region\_consensus\_protocol\_class | 元数据副本组的共识协议                            | org.apache.iotdb.consensus.ratis.RatisConsensus |
@@ -141,8 +142,8 @@ mvn clean package -pl distribution -am -DskipTests
 | dn\_rpc\_address                    | 客户端 RPC 服务的地址             | 127.0.0.1       | 设置为服务器的 IPV4 地址或域名                                                                |
 | dn\_rpc\_port                       | 客户端 RPC 服务的端口             | 6667            | 设置为任意未占用端口                                                                        |
 | dn\_internal\_address               | DataNode 在集群内部接收控制流使用的地址  | 127.0.0.1       | 设置为服务器的 IPV4 地址或域名                                                                |
-| dn\_internal\_port                  | DataNode 在集群内部接收控制流使用的端口  | 10730            | 设置为任意未占用端口                                                                        |
-| dn\_mpp\_data\_exchange\_port       | DataNode 在集群内部接收数据流使用的端口  | 10740            | 设置为任意未占用端口                                                                        |
+| dn\_internal\_port                  | DataNode 在集群内部接收控制流使用的端口  | 10730           | 设置为任意未占用端口                                                                        |
+| dn\_mpp\_data\_exchange\_port       | DataNode 在集群内部接收数据流使用的端口  | 10740           | 设置为任意未占用端口                                                                        |
 | dn\_data\_region\_consensus\_port   | DataNode 的数据副本间共识协议通信的端口  | 10750           | 设置为任意未占用端口                                                                        |
 | dn\_schema\_region\_consensus\_port | DataNode 的元数据副本间共识协议通信的端口 | 10760           | 设置为任意未占用端口                                                                        |
 | dn\_target\_config\_node\_list      | 集群中正在运行的 ConfigNode 地址    | 127.0.0.1:10710 | 设置为任意正在运行的 ConfigNode 的 cn\_internal\_address:cn\_internal\_port,可设置多个,用逗号(",")隔开 |
@@ -167,7 +168,20 @@ mvn clean package -pl distribution -am -DskipTests
 **集群第一个启动的节点必须是 ConfigNode,第一个启动的 ConfigNode 必须遵循本小节教程。**
 
 第一个启动的 ConfigNode 是 Seed-ConfigNode,标志着新集群的创建。
-在启动 Seed-ConfigNode 前,请打开它的配置文件 ./conf/iotdb-confignode.properties,并检查如下参数:
+在启动 Seed-ConfigNode 前,请打开通用配置文件 ./conf/iotdb-common.properties,并检查如下参数:
+
+| **配置项**                                    | **检查**        |
+|--------------------------------------------|---------------|
+| cluster\_name                              | 已设置为期望的集群名称   |
+| config\_node\_consensus\_protocol\_class   | 已设置为期望的共识协议   |
+| schema\_replication\_factor                | 已设置为期望的元数据副本数 |
+| schema\_region\_consensus\_protocol\_class | 已设置为期望的共识协议   |
+| data\_replication\_factor                  | 已设置为期望的数据副本数  |
+| data\_region\_consensus\_protocol\_class   | 已设置为期望的共识协议   |
+
+**注意:** 请根据[部署推荐](https://iotdb.apache.org/zh/UserGuide/Master/Cluster/Deployment-Recommendation.html)配置合适的通用参数,这些参数在首次配置后即不可修改。
+
+接着请打开它的配置文件 ./conf/iotdb-confignode.properties,并检查如下参数:
 
 | **配置项**                        | **检查**                                                   |
 |--------------------------------|----------------------------------------------------------|
@@ -198,9 +212,19 @@ ConfigNode 的其它配置参数可参考
 
 可向集群添加更多 ConfigNode,以保证 ConfigNode 的高可用。常用的配置为额外增加两个 ConfigNode,使集群共有三个 ConfigNode。
 
-新增的 ConfigNode 需要保证 ./conf/iotdb-common.properites 中的所有配置参数与种子 ConfigNode 完全一致,否则可能启动失败或产生运行时错误。
+新增的 ConfigNode 需要保证 ./conf/iotdb-common.properites 中的所有配置参数与 Seed-ConfigNode 完全一致,否则可能启动失败或产生运行时错误。
+因此,请着重检查通用配置文件中的以下参数:
 
-在增加一个新的 ConfigNode 之前,请打开它的配置文件 ./conf/iotdb-confignode.properties,并检查以下参数:
+| **配置项**                                    | **检查**                 |
+|--------------------------------------------|------------------------|
+| cluster\_name                              | 与 Seed-ConfigNode 保持一致 |
+| config\_node\_consensus\_protocol\_class   | 与 Seed-ConfigNode 保持一致 |
+| schema\_replication\_factor                | 与 Seed-ConfigNode 保持一致 |
+| schema\_region\_consensus\_protocol\_class | 与 Seed-ConfigNode 保持一致 |
+| data\_replication\_factor                  | 与 Seed-ConfigNode 保持一致 |
+| data\_region\_consensus\_protocol\_class   | 与 Seed-ConfigNode 保持一致 |
+
+接着请打开它的配置文件 ./conf/iotdb-confignode.properties,并检查以下参数:
 
 | **配置项**                        | **检查**                                                       |
 |--------------------------------|--------------------------------------------------------------|
@@ -230,7 +254,13 @@ ConfigNode 的其它配置参数可参考
 **确保集群已有正在运行的 ConfigNode 后,才能开始增加 DataNode。**
 
 可以向集群中添加任意个 DataNode。
-在添加新的 DataNode 前,请打开它的配置文件 ./conf/iotdb-datanode.properties 并检查以下参数:
+在添加新的 DataNode 前,请先打开通用配置文件 ./conf/iotdb-common.properties 并检查以下参数:
+
+| **配置项**                                    | **检查**                 |
+|--------------------------------------------|------------------------|
+| cluster\_name                              | 与 Seed-ConfigNode 保持一致 |
+
+接着打开它的配置文件 ./conf/iotdb-datanode.properties 并检查以下参数:
 
 | **配置项**                             | **检查**                                                    |
 |-------------------------------------|-----------------------------------------------------------|
diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/AbstractNodeWrapper.java b/integration-test/src/main/java/org/apache/iotdb/it/env/AbstractNodeWrapper.java
index c27f9bdcfe..438d0ba20d 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/AbstractNodeWrapper.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/AbstractNodeWrapper.java
@@ -208,7 +208,7 @@ public abstract class AbstractNodeWrapper implements BaseNodeWrapper {
 
   @Override
   public void waitingToShutDown() {
-    while (this.instance.isAlive()) {
+    while (this.instance != null && this.instance.isAlive()) {
       try {
         Thread.sleep(100);
       } catch (InterruptedException e) {
diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/cluster/IoTDBClusterNodeErrorStartUpIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/cluster/IoTDBClusterNodeErrorStartUpIT.java
index d331af6226..369f676d8e 100644
--- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/cluster/IoTDBClusterNodeErrorStartUpIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/cluster/IoTDBClusterNodeErrorStartUpIT.java
@@ -92,6 +92,41 @@ public class IoTDBClusterNodeErrorStartUpIT {
     CONF.setConfigNodeConsesusProtocolClass(originalConfigNodeConsensusProtocolClass);
   }
 
+  @Test
+  public void testIllegalNodeRegistration()
+      throws ClientManagerException, IOException, InterruptedException, TException {
+    ConfigNodeWrapper configNodeWrapper = EnvFactory.getEnv().generateRandomConfigNodeWrapper();
+    DataNodeWrapper dataNodeWrapper = EnvFactory.getEnv().generateRandomDataNodeWrapper();
+
+    try (SyncConfigNodeIServiceClient client =
+        (SyncConfigNodeIServiceClient) EnvFactory.getEnv().getLeaderConfigNodeConnection()) {
+      /* Register with error cluster name */
+      TConfigNodeRegisterReq configNodeRegisterReq =
+          ConfigNodeTestUtils.generateTConfigNodeRegisterReq(configNodeWrapper);
+      configNodeRegisterReq.getClusterParameters().setClusterName(ERROR_CLUSTER_NAME);
+      configNodeRegisterReq
+          .getClusterParameters()
+          .setConfigNodeConsensusProtocolClass(testConsensusProtocolClass);
+      TConfigNodeRegisterResp configNodeRegisterResp =
+          client.registerConfigNode(configNodeRegisterReq);
+      Assert.assertEquals(
+          TSStatusCode.REJECT_NODE_START.getStatusCode(),
+          configNodeRegisterResp.getStatus().getCode());
+      Assert.assertTrue(
+          configNodeRegisterResp.getStatus().getMessage().contains("cluster are inconsistent"));
+
+      TDataNodeRegisterReq dataNodeRegisterReq =
+          ConfigNodeTestUtils.generateTDataNodeRegisterReq(dataNodeWrapper);
+      dataNodeRegisterReq.setClusterName(ERROR_CLUSTER_NAME);
+      TDataNodeRegisterResp dataNodeRegisterResp = client.registerDataNode(dataNodeRegisterReq);
+      Assert.assertEquals(
+          TSStatusCode.REJECT_NODE_START.getStatusCode(),
+          dataNodeRegisterResp.getStatus().getCode());
+      Assert.assertTrue(
+          dataNodeRegisterResp.getStatus().getMessage().contains("cluster are inconsistent"));
+    }
+  }
+
   @Test
   public void testConflictNodeRegistration()
       throws ClientManagerException, InterruptedException, TException, IOException {
diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/cluster/IoTDBClusterNodeGetterIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/cluster/IoTDBClusterNodeGetterIT.java
index 1462f579cd..1066e48c46 100644
--- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/cluster/IoTDBClusterNodeGetterIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/cluster/IoTDBClusterNodeGetterIT.java
@@ -30,10 +30,10 @@ import org.apache.iotdb.confignode.rpc.thrift.TDataNodeConfigurationResp;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeInfo;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRemoveReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRemoveResp;
-import org.apache.iotdb.confignode.rpc.thrift.TShowClusterParametersResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowConfigNodesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodesResp;
+import org.apache.iotdb.confignode.rpc.thrift.TShowVariablesResp;
 import org.apache.iotdb.consensus.ConsensusFactory;
 import org.apache.iotdb.it.env.ConfigFactory;
 import org.apache.iotdb.it.env.ConfigNodeWrapper;
@@ -143,11 +143,10 @@ public class IoTDBClusterNodeGetterIT {
       }
 
       /* Tests showClusterParameters */
-      TShowClusterParametersResp showClusterParametersResp = client.showClusterParameters();
+      TShowVariablesResp showVariablesResp = client.showVariables();
       Assert.assertEquals(
-          TSStatusCode.SUCCESS_STATUS.getStatusCode(),
-          showClusterParametersResp.getStatus().getCode());
-      TClusterParameters clusterParameters = showClusterParametersResp.getClusterParameters();
+          TSStatusCode.SUCCESS_STATUS.getStatusCode(), showVariablesResp.getStatus().getCode());
+      TClusterParameters clusterParameters = showVariablesResp.getClusterParameters();
       Assert.assertEquals(
           testConsensusProtocolClass, clusterParameters.getConfigNodeConsensusProtocolClass());
       Assert.assertEquals(
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 da6e76e3e4..e8f19aed24 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
@@ -94,7 +94,7 @@ public class IoTDBPartitionGetterIT {
   private static final long testTimePartitionInterval = 604800000;
 
   protected static int originalLeastDataRegionGroupNum;
-  private static final int testLeastDataRegionGroupNum = 3;
+  private static final int testLeastDataRegionGroupNum = 5;
 
   private static final String sg = "root.sg";
   private static final int storageGroupNum = 2;
diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java
index 66376ae959..31ad85822b 100644
--- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java
+++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java
@@ -276,7 +276,8 @@ public class ConfigNodeTestUtils {
   }
 
   public static TDataNodeRegisterReq generateTDataNodeRegisterReq(DataNodeWrapper dataNodeWrapper) {
-    return new TDataNodeRegisterReq(generateTDataNodeConfiguration(-1, dataNodeWrapper));
+    return new TDataNodeRegisterReq(
+        generateTDataNodeConfiguration(-1, dataNodeWrapper), CONF.getClusterName());
   }
 
   public static TDataNodeRestartReq generateTDataNodeRestartReq(
diff --git a/node-commons/src/assembly/resources/conf/iotdb-common.properties b/node-commons/src/assembly/resources/conf/iotdb-common.properties
index b8982f0c31..92c4eb73f0 100644
--- a/node-commons/src/assembly/resources/conf/iotdb-common.properties
+++ b/node-commons/src/assembly/resources/conf/iotdb-common.properties
@@ -16,6 +16,14 @@
 # specific language governing permissions and limitations
 # under the License.
 
+####################
+### Cluster Configuration
+####################
+
+# Used for indicate cluster name and distinguish different cluster.
+# Datatype: string
+cluster_name=defaultCluster
+
 ####################
 ### Replication configuration
 ####################
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java b/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java
index b380716089..3edf0ffb16 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java
@@ -69,6 +69,9 @@ public class IoTDBConstant {
   public static final String CN_TARGET_CONFIG_NODE_LIST = "cn_target_config_node_list";
   public static final String DN_TARGET_CONFIG_NODE_LIST = "dn_target_config_node_list";
 
+  public static final String CLUSTER_NAME = "cluster_name";
+  public static final String DEFAULT_CLUSTER_NAME = "defaultCluster";
+
   // when running the program in IDE, we can not get the version info using
   // getImplementationVersion()
   public static final String VERSION =
@@ -265,28 +268,6 @@ public class IoTDBConstant {
   public static final String WAL_START_SEARCH_INDEX = "startSearchIndex";
   public static final String WAL_STATUS_CODE = "statusCode";
 
-  // show cluster status
-  public static final String NODE_TYPE_CONFIG_NODE = "ConfigNode";
-  public static final String NODE_TYPE_DATA_NODE = "DataNode";
-  public static final String CLUSTER_NAME = "ClusterName";
-  public static final String CONFIG_NODE_CONSENSUS_PROTOCOL_CLASS =
-      "ConfigNodeConsensusProtocolClass";
-  public static final String DATA_REGION_CONSENSUS_PROTOCOL_CLASS =
-      "DataRegionConsensusProtocolClass";
-  public static final String SCHEMA_REGION_CONSENSUS_PROTOCOL_CLASS =
-      "SchemaRegionConsensusProtocolClass";
-  public static final String SERIES_SLOT_NUM = "SeriesSlotNum";
-  public static final String SERIES_SLOT_EXECUTOR_CLASS = "SeriesSlotExecutorClass";
-  public static final String DEFAULT_TTL = "DefaultTTL(ms)";
-  public static final String TIME_PARTITION_INTERVAL = "TimePartitionInterval";
-  public static final String DATA_REPLICATION_FACTOR = "DataReplicationFactor";
-  public static final String SCHEMA_REPLICATION_FACTOR = "SchemaReplicationFactor";
-  public static final String SCHEMA_REGION_PER_DATA_NODE = "SchemaRegionPerDataNode";
-  public static final String DATA_REGION_PER_PROCESSOR = "DataRegionPerProcessor";
-  public static final String READ_CONSISTENCY_LEVEL = "ReadConsistencyLevel";
-  public static final String DISK_SPACE_WARNING_THRESHOLD = "DiskSpaceWarningThreshold";
-  public static final String LEAST_DATA_REGION_GROUP_NUM = "LeastDataRegionGroupNum";
-
   public static final String IOTDB_FOREGROUND = "iotdb-foreground";
   public static final String IOTDB_PIDFILE = "iotdb-pidfile";
 
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 ac2e8b3d82..ac6f4d0a49 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
@@ -98,7 +98,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TSetStorageGroupReq;
 import org.apache.iotdb.confignode.rpc.thrift.TSetTimePartitionIntervalReq;
 import org.apache.iotdb.confignode.rpc.thrift.TShowCQResp;
-import org.apache.iotdb.confignode.rpc.thrift.TShowClusterParametersResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowConfigNodesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodesResp;
@@ -107,6 +106,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowRegionReq;
 import org.apache.iotdb.confignode.rpc.thrift.TShowRegionResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowStorageGroupResp;
+import org.apache.iotdb.confignode.rpc.thrift.TShowVariablesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchemaResp;
 import org.apache.iotdb.confignode.rpc.thrift.TSystemConfigurationResp;
 import org.apache.iotdb.confignode.rpc.thrift.TUnsetSchemaTemplateReq;
@@ -480,10 +480,10 @@ public class ConfigNodeClient
   }
 
   @Override
-  public TShowClusterParametersResp showClusterParameters() throws TException {
+  public TShowVariablesResp showVariables() throws TException {
     for (int i = 0; i < RETRY_NUM; i++) {
       try {
-        TShowClusterParametersResp resp = client.showClusterParameters();
+        TShowVariablesResp resp = client.showVariables();
         if (!updateConfigNodeLeader(resp.status)) {
           return resp;
         }
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 e3915805a8..377af01153 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
@@ -199,6 +199,9 @@ public class IoTDBDescriptor {
 
   public void loadProperties(Properties properties) {
 
+    conf.setClusterName(
+        properties.getProperty(IoTDBConstant.CLUSTER_NAME, conf.getClusterName()).trim());
+
     conf.setRpcAddress(
         properties.getProperty(IoTDBConstant.DN_RPC_ADDRESS, conf.getRpcAddress()).trim());
 
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBStartCheck.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBStartCheck.java
index baefd08f12..908d6864c1 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBStartCheck.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBStartCheck.java
@@ -111,8 +111,6 @@ public class IoTDBStartCheck {
   private static String timeEncoderValue =
       String.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder());
 
-  public static final String CLUSTER_NAME = "cluster_name";
-  public static final String DEFAULT_CLUSTER_NAME = "defaultCluster";
   private static final String DATA_NODE_ID = "data_node_id";
 
   private static final String SCHEMA_REGION_CONSENSUS_PROTOCOL = "schema_region_consensus_protocol";
@@ -437,8 +435,8 @@ public class IoTDBStartCheck {
     }
 
     // load configuration from system properties only when start as Data node
-    if (properties.containsKey(CLUSTER_NAME)) {
-      config.setClusterName(properties.getProperty(CLUSTER_NAME));
+    if (properties.containsKey(IoTDBConstant.CLUSTER_NAME)) {
+      config.setClusterName(properties.getProperty(IoTDBConstant.CLUSTER_NAME));
     }
     if (properties.containsKey(DATA_NODE_ID)) {
       config.setDataNodeId(Integer.parseInt(properties.getProperty(DATA_NODE_ID)));
@@ -483,7 +481,7 @@ public class IoTDBStartCheck {
     reloadProperties();
 
     try (FileOutputStream tmpFOS = new FileOutputStream(tmpPropertiesFile.toString())) {
-      properties.setProperty(CLUSTER_NAME, clusterName);
+      properties.setProperty(IoTDBConstant.CLUSTER_NAME, clusterName);
       properties.setProperty(DATA_NODE_ID, String.valueOf(dataNodeId));
       properties.store(tmpFOS, SYSTEM_PROPERTIES_STRING);
       // serialize finished, delete old system.properties file
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeaderConstant.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeaderConstant.java
index ba405166f7..986d9be6c2 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeaderConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeaderConstant.java
@@ -58,7 +58,7 @@ public class ColumnHeaderConstant {
   public static final String VERSION = "Version";
   public static final String BUILD_INFO = "BuildInfo";
   public static final String PATHS = "Paths";
-  public static final String CLUSTER_PARAMETERS = "ClusterParameters";
+  public static final String VARIABLE = "Variable";
 
   // column names for count statement
   public static final String COLUMN = "Column";
@@ -93,6 +93,25 @@ public class ColumnHeaderConstant {
   public static final String PATH_PATTERN = "PathPattern";
   public static final String CLASS_NAME = "ClassName";
 
+  // show cluster status
+  public static final String NODE_TYPE_CONFIG_NODE = "ConfigNode";
+  public static final String NODE_TYPE_DATA_NODE = "DataNode";
+  public static final String COLUMN_CLUSTER_NAME = "ClusterName";
+  public static final String CONFIG_NODE_CONSENSUS_PROTOCOL_CLASS =
+      "ConfigNodeConsensusProtocolClass";
+  public static final String DATA_REGION_CONSENSUS_PROTOCOL_CLASS =
+      "DataRegionConsensusProtocolClass";
+  public static final String SCHEMA_REGION_CONSENSUS_PROTOCOL_CLASS =
+      "SchemaRegionConsensusProtocolClass";
+  public static final String SERIES_SLOT_NUM = "SeriesSlotNum";
+  public static final String SERIES_SLOT_EXECUTOR_CLASS = "SeriesSlotExecutorClass";
+  public static final String DEFAULT_TTL = "DefaultTTL(ms)";
+  public static final String SCHEMA_REGION_PER_DATA_NODE = "SchemaRegionPerDataNode";
+  public static final String DATA_REGION_PER_PROCESSOR = "DataRegionPerProcessor";
+  public static final String READ_CONSISTENCY_LEVEL = "ReadConsistencyLevel";
+  public static final String DISK_SPACE_WARNING_THRESHOLD = "DiskSpaceWarningThreshold";
+  public static final String LEAST_DATA_REGION_GROUP_NUM = "LeastDataRegionGroupNum";
+
   // column names for show region statement
   public static final String REGION_ID = "RegionId";
   public static final String TYPE = "Type";
@@ -267,10 +286,9 @@ public class ColumnHeaderConstant {
           new ColumnHeader(SCHEMA_CONSENSUS_PORT, TSDataType.TEXT),
           new ColumnHeader(DATA_CONSENSUS_PORT, TSDataType.TEXT));
 
-  public static final List<ColumnHeader> showClusterParametersColumnHeaders =
+  public static final List<ColumnHeader> showVariablesColumnHeaders =
       ImmutableList.of(
-          new ColumnHeader(CLUSTER_PARAMETERS, TSDataType.TEXT),
-          new ColumnHeader(VALUE, TSDataType.TEXT));
+          new ColumnHeader(VARIABLE, TSDataType.TEXT), new ColumnHeader(VALUE, TSDataType.TEXT));
 
   public static final List<ColumnHeader> showFunctionsColumnHeaders =
       ImmutableList.of(
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeaderFactory.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeaderFactory.java
index 76a0038e85..03aecf7df0 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeaderFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeaderFactory.java
@@ -82,7 +82,7 @@ public class DatasetHeaderFactory {
   }
 
   public static DatasetHeader getShowClusterParametersHeader() {
-    return new DatasetHeader(ColumnHeaderConstant.showClusterParametersColumnHeaders, true);
+    return new DatasetHeader(ColumnHeaderConstant.showVariablesColumnHeaders, true);
   }
 
   public static DatasetHeader getShowClusterDetailsHeader() {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java
index 6f678e8ba0..6aad1837f0 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java
@@ -34,7 +34,6 @@ import org.apache.iotdb.db.mpp.plan.execution.config.metadata.GetTimeSlotListTas
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.SetStorageGroupTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.SetTTLTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowClusterDetailsTask;
-import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowClusterParametersTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowClusterTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowConfigNodesTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowContinuousQueriesTask;
@@ -44,6 +43,7 @@ import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowRegionTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowStorageGroupTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowTTLTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowTriggersTask;
+import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowVariablesTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.UnSetTTLTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.CreateSchemaTemplateTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.DeactivateSchemaTemplateTask;
@@ -93,6 +93,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowRegionStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTriggersStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowVariablesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.UnSetTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.CreateSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.DeactivateTemplateStatement;
@@ -170,13 +171,17 @@ public class ConfigTaskVisitor
     return new ShowTTLTask(showTTLStatement);
   }
 
+  @Override
+  public IConfigTask visitShowVariables(
+      ShowVariablesStatement showVariablesStatement, TaskContext context) {
+    return new ShowVariablesTask();
+  }
+
   @Override
   public IConfigTask visitShowCluster(
       ShowClusterStatement showClusterStatement, TaskContext context) {
     if (showClusterStatement.isDetails()) {
       return new ShowClusterDetailsTask(showClusterStatement);
-    } else if (showClusterStatement.isParameters()) {
-      return new ShowClusterParametersTask();
     } else {
       return new ShowClusterTask(showClusterStatement);
     }
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 5518645d35..11f6ad0494 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
@@ -60,7 +60,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TGetUDFTableResp;
 import org.apache.iotdb.confignode.rpc.thrift.TPipeSinkInfo;
 import org.apache.iotdb.confignode.rpc.thrift.TSetStorageGroupReq;
 import org.apache.iotdb.confignode.rpc.thrift.TShowCQResp;
-import org.apache.iotdb.confignode.rpc.thrift.TShowClusterParametersResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowConfigNodesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodesResp;
@@ -70,6 +69,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowRegionReq;
 import org.apache.iotdb.confignode.rpc.thrift.TShowRegionResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowStorageGroupResp;
+import org.apache.iotdb.confignode.rpc.thrift.TShowVariablesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchemaResp;
 import org.apache.iotdb.confignode.rpc.thrift.TUnsetSchemaTemplateReq;
@@ -87,7 +87,6 @@ import org.apache.iotdb.db.mpp.plan.execution.config.metadata.GetSeriesSlotListT
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.GetTimeSlotListTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.SetStorageGroupTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowClusterDetailsTask;
-import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowClusterParametersTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowClusterTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowConfigNodesTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowContinuousQueriesTask;
@@ -97,6 +96,7 @@ import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowRegionTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowStorageGroupTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowTTLTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowTriggersTask;
+import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowVariablesTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.ShowNodesInSchemaTemplateTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.ShowPathSetTemplateTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.ShowSchemaTemplateTask;
@@ -793,16 +793,16 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   @Override
   public SettableFuture<ConfigTaskResult> showClusterParameters() {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
-    TShowClusterParametersResp showClusterParametersResp = new TShowClusterParametersResp();
+    TShowVariablesResp showVariablesResp = new TShowVariablesResp();
     try (ConfigNodeClient client =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.configNodeRegionId)) {
-      showClusterParametersResp = client.showClusterParameters();
+      showVariablesResp = client.showVariables();
     } catch (ClientManagerException | TException e) {
       future.setException(e);
     }
 
     // build TSBlock
-    ShowClusterParametersTask.buildTSBlock(showClusterParametersResp, future);
+    ShowVariablesTask.buildTSBlock(showVariablesResp, future);
 
     return future;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowClusterDetailsTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowClusterDetailsTask.java
index 01e77c71bb..6549ee3ffc 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowClusterDetailsTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowClusterDetailsTask.java
@@ -39,8 +39,8 @@ import com.google.common.util.concurrent.SettableFuture;
 import java.util.List;
 import java.util.stream.Collectors;
 
-import static org.apache.iotdb.commons.conf.IoTDBConstant.NODE_TYPE_CONFIG_NODE;
-import static org.apache.iotdb.commons.conf.IoTDBConstant.NODE_TYPE_DATA_NODE;
+import static org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant.NODE_TYPE_CONFIG_NODE;
+import static org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant.NODE_TYPE_DATA_NODE;
 
 public class ShowClusterDetailsTask implements IConfigTask {
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowClusterTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowClusterTask.java
index 2b5817523e..b1cf6a2e95 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowClusterTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowClusterTask.java
@@ -39,8 +39,8 @@ import com.google.common.util.concurrent.SettableFuture;
 import java.util.List;
 import java.util.stream.Collectors;
 
-import static org.apache.iotdb.commons.conf.IoTDBConstant.NODE_TYPE_CONFIG_NODE;
-import static org.apache.iotdb.commons.conf.IoTDBConstant.NODE_TYPE_DATA_NODE;
+import static org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant.NODE_TYPE_CONFIG_NODE;
+import static org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant.NODE_TYPE_DATA_NODE;
 
 public class ShowClusterTask implements IConfigTask {
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowClusterParametersTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowVariablesTask.java
similarity index 75%
rename from server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowClusterParametersTask.java
rename to server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowVariablesTask.java
index fab89048a1..794d3aa84c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowClusterParametersTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/ShowVariablesTask.java
@@ -19,9 +19,8 @@
 
 package org.apache.iotdb.db.mpp.plan.execution.config.metadata;
 
-import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.confignode.rpc.thrift.TClusterParameters;
-import org.apache.iotdb.confignode.rpc.thrift.TShowClusterParametersResp;
+import org.apache.iotdb.confignode.rpc.thrift.TShowVariablesResp;
 import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
 import org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant;
 import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
@@ -40,9 +39,9 @@ import com.google.common.util.concurrent.SettableFuture;
 import java.util.List;
 import java.util.stream.Collectors;
 
-public class ShowClusterParametersTask implements IConfigTask {
+public class ShowVariablesTask implements IConfigTask {
 
-  public ShowClusterParametersTask() {
+  public ShowVariablesTask() {
     // Empty constructor
   }
 
@@ -54,74 +53,73 @@ public class ShowClusterParametersTask implements IConfigTask {
   }
 
   public static void buildTSBlock(
-      TShowClusterParametersResp showClusterParametersResp,
-      SettableFuture<ConfigTaskResult> future) {
+      TShowVariablesResp showVariablesResp, SettableFuture<ConfigTaskResult> future) {
     List<TSDataType> outputDataTypes =
-        ColumnHeaderConstant.showClusterParametersColumnHeaders.stream()
+        ColumnHeaderConstant.showVariablesColumnHeaders.stream()
             .map(ColumnHeader::getColumnType)
             .collect(Collectors.toList());
     TsBlockBuilder builder = new TsBlockBuilder(outputDataTypes);
 
-    TClusterParameters clusterParameters = showClusterParametersResp.getClusterParameters();
+    TClusterParameters clusterParameters = showVariablesResp.getClusterParameters();
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.CLUSTER_NAME),
+        new Binary(ColumnHeaderConstant.COLUMN_CLUSTER_NAME),
         new Binary(clusterParameters.getClusterName()));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.DATA_REPLICATION_FACTOR),
+        new Binary(ColumnHeaderConstant.DATA_REPLICATION_FACTOR),
         new Binary(String.valueOf(clusterParameters.getDataReplicationFactor())));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.SCHEMA_REPLICATION_FACTOR),
+        new Binary(ColumnHeaderConstant.SCHEMA_REPLICATION_FACTOR),
         new Binary(String.valueOf(clusterParameters.getSchemaReplicationFactor())));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.DATA_REGION_CONSENSUS_PROTOCOL_CLASS),
+        new Binary(ColumnHeaderConstant.DATA_REGION_CONSENSUS_PROTOCOL_CLASS),
         new Binary(clusterParameters.getDataRegionConsensusProtocolClass()));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.SCHEMA_REGION_CONSENSUS_PROTOCOL_CLASS),
+        new Binary(ColumnHeaderConstant.SCHEMA_REGION_CONSENSUS_PROTOCOL_CLASS),
         new Binary(clusterParameters.getSchemaRegionConsensusProtocolClass()));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.CONFIG_NODE_CONSENSUS_PROTOCOL_CLASS),
+        new Binary(ColumnHeaderConstant.CONFIG_NODE_CONSENSUS_PROTOCOL_CLASS),
         new Binary(clusterParameters.getConfigNodeConsensusProtocolClass()));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.TIME_PARTITION_INTERVAL),
+        new Binary(ColumnHeaderConstant.TIME_PARTITION_INTERVAL),
         new Binary(String.valueOf(clusterParameters.getTimePartitionInterval())));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.DEFAULT_TTL),
+        new Binary(ColumnHeaderConstant.DEFAULT_TTL),
         new Binary(String.valueOf(clusterParameters.getDefaultTTL())));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.READ_CONSISTENCY_LEVEL),
+        new Binary(ColumnHeaderConstant.READ_CONSISTENCY_LEVEL),
         new Binary(String.valueOf(clusterParameters.getReadConsistencyLevel())));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.SCHEMA_REGION_PER_DATA_NODE),
+        new Binary(ColumnHeaderConstant.SCHEMA_REGION_PER_DATA_NODE),
         new Binary(String.valueOf(clusterParameters.getSchemaRegionPerDataNode())));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.DATA_REGION_PER_PROCESSOR),
+        new Binary(ColumnHeaderConstant.DATA_REGION_PER_PROCESSOR),
         new Binary(String.valueOf(clusterParameters.getDataRegionPerProcessor())));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.LEAST_DATA_REGION_GROUP_NUM),
+        new Binary(ColumnHeaderConstant.LEAST_DATA_REGION_GROUP_NUM),
         new Binary(String.valueOf(clusterParameters.getLeastDataRegionGroupNum())));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.SERIES_SLOT_NUM),
+        new Binary(ColumnHeaderConstant.SERIES_SLOT_NUM),
         new Binary(String.valueOf(clusterParameters.getSeriesPartitionSlotNum())));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.SERIES_SLOT_EXECUTOR_CLASS),
+        new Binary(ColumnHeaderConstant.SERIES_SLOT_EXECUTOR_CLASS),
         new Binary(clusterParameters.getSeriesPartitionExecutorClass()));
     buildTSBlock(
         builder,
-        new Binary(IoTDBConstant.DISK_SPACE_WARNING_THRESHOLD),
+        new Binary(ColumnHeaderConstant.DISK_SPACE_WARNING_THRESHOLD),
         new Binary(String.valueOf(clusterParameters.getDiskSpaceWarningThreshold())));
 
     DatasetHeader datasetHeader = DatasetHeaderFactory.getShowClusterParametersHeader();
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
index d06d7c3c9d..3732959758 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
@@ -126,6 +126,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowStorageGroupStatement
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTriggersStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowVariablesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.UnSetTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ActivateTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.CreateSchemaTemplateStatement;
@@ -1999,13 +2000,16 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     return showTTLStatement;
   }
 
+  @Override
+  public Statement visitShowVariables(IoTDBSqlParser.ShowVariablesContext ctx) {
+    return new ShowVariablesStatement();
+  }
+
   @Override
   public Statement visitShowCluster(IoTDBSqlParser.ShowClusterContext ctx) {
     ShowClusterStatement showClusterStatement = new ShowClusterStatement();
     if (ctx.DETAILS() != null) {
       showClusterStatement.setDetails(true);
-    } else if (ctx.PARAMETERS() != null) {
-      showClusterStatement.setParameters(true);
     }
     return showClusterStatement;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
index bd1d02330a..4d55216dfe 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
@@ -65,6 +65,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowStorageGroupStatement
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTriggersStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowVariablesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.UnSetTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ActivateTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.CreateSchemaTemplateStatement;
@@ -170,6 +171,10 @@ public abstract class StatementVisitor<R, C> {
     return visitStatement(showTTLStatement, context);
   }
 
+  public R visitShowVariables(ShowVariablesStatement showVariablesStatement, C context) {
+    return visitStatement(showVariablesStatement, context);
+  }
+
   public R visitShowCluster(ShowClusterStatement showClusterStatement, C context) {
     return visitStatement(showClusterStatement, context);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/ShowClusterStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/ShowClusterStatement.java
index db716b0f0c..be001b932f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/ShowClusterStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/ShowClusterStatement.java
@@ -25,7 +25,6 @@ import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
 public class ShowClusterStatement extends ShowStatement implements IConfigStatement {
 
   private boolean isDetails = false;
-  private boolean isParameters = false;
 
   @Override
   public QueryType getQueryType() {
@@ -44,12 +43,4 @@ public class ShowClusterStatement extends ShowStatement implements IConfigStatem
   public void setDetails(boolean details) {
     isDetails = details;
   }
-
-  public boolean isParameters() {
-    return isParameters;
-  }
-
-  public void setParameters(boolean parameters) {
-    isParameters = parameters;
-  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/ShowClusterStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/ShowVariablesStatement.java
similarity index 71%
copy from server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/ShowClusterStatement.java
copy to server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/ShowVariablesStatement.java
index db716b0f0c..40b028012f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/ShowClusterStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/ShowVariablesStatement.java
@@ -22,10 +22,7 @@ import org.apache.iotdb.db.mpp.plan.analyze.QueryType;
 import org.apache.iotdb.db.mpp.plan.statement.IConfigStatement;
 import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
 
-public class ShowClusterStatement extends ShowStatement implements IConfigStatement {
-
-  private boolean isDetails = false;
-  private boolean isParameters = false;
+public class ShowVariablesStatement extends ShowStatement implements IConfigStatement {
 
   @Override
   public QueryType getQueryType() {
@@ -34,22 +31,6 @@ public class ShowClusterStatement extends ShowStatement implements IConfigStatem
 
   @Override
   public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
-    return visitor.visitShowCluster(this, context);
-  }
-
-  public boolean isDetails() {
-    return isDetails;
-  }
-
-  public void setDetails(boolean details) {
-    isDetails = details;
-  }
-
-  public boolean isParameters() {
-    return isParameters;
-  }
-
-  public void setParameters(boolean parameters) {
-    isParameters = parameters;
+    return visitor.visitShowVariables(this, context);
   }
 }
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 4e82030534..b8be4a0fc0 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
@@ -97,7 +97,7 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
-import static org.apache.iotdb.db.conf.IoTDBStartCheck.DEFAULT_CLUSTER_NAME;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.DEFAULT_CLUSTER_NAME;
 
 public class DataNode implements DataNodeMBean {
   private static final Logger logger = LoggerFactory.getLogger(DataNode.class);
@@ -343,6 +343,7 @@ public class DataNode implements DataNodeMBean {
     int retry = DEFAULT_RETRY;
     TDataNodeRegisterReq req = new TDataNodeRegisterReq();
     req.setDataNodeConfiguration(generateDataNodeConfiguration());
+    req.setClusterName(config.getClusterName());
     TDataNodeRegisterResp dataNodeRegisterResp = null;
     while (retry > 0) {
       try (ConfigNodeClient configNodeClient = new ConfigNodeClient()) {
@@ -377,16 +378,15 @@ public class DataNode implements DataNodeMBean {
     if (dataNodeRegisterResp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
 
       /* Store runtime configurations when register success */
-      String clusterName = dataNodeRegisterResp.getClusterName();
-      config.setClusterName(dataNodeRegisterResp.getClusterName());
       int dataNodeID = dataNodeRegisterResp.getDataNodeId();
       config.setDataNodeId(dataNodeID);
-      IoTDBStartCheck.getInstance().serializeClusterNameAndDataNodeId(clusterName, dataNodeID);
+      IoTDBStartCheck.getInstance()
+          .serializeClusterNameAndDataNodeId(config.getClusterName(), dataNodeID);
 
       storeRuntimeConfigurations(
           dataNodeRegisterResp.getConfigNodeList(), dataNodeRegisterResp.getRuntimeConfiguration());
 
-      logger.info("Successfully register to the cluster");
+      logger.info("Successfully register to the cluster: {}", config.getClusterName());
     } else {
       /* Throw exception when register failed */
       logger.error(dataNodeRegisterResp.getStatus().getMessage());
@@ -437,7 +437,7 @@ public class DataNode implements DataNodeMBean {
       /* Store runtime configurations when restart request is accepted */
       storeRuntimeConfigurations(
           dataNodeRestartResp.getConfigNodeList(), dataNodeRestartResp.getRuntimeConfiguration());
-      logger.info("Restart request is accepted.");
+      logger.info("Restart request to cluster: {} is accepted.", config.getClusterName());
     } else {
       /* Throw exception when restart is rejected */
       throw new StartupException(dataNodeRestartResp.getStatus().getMessage());
diff --git a/thrift-confignode/src/main/thrift/confignode.thrift b/thrift-confignode/src/main/thrift/confignode.thrift
index ae6766b947..48af2acf0b 100644
--- a/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/thrift-confignode/src/main/thrift/confignode.thrift
@@ -94,14 +94,14 @@ struct TRuntimeConfiguration {
 
 struct TDataNodeRegisterReq {
   1: required common.TDataNodeConfiguration dataNodeConfiguration
+  2: required string clusterName
 }
 
 struct TDataNodeRegisterResp {
   1: required common.TSStatus status
   2: required list<common.TConfigNodeLocation> configNodeList
-  3: optional string clusterName
-  4: optional i32 dataNodeId
-  5: optional TRuntimeConfiguration runtimeConfiguration
+  3: optional i32 dataNodeId
+  4: optional TRuntimeConfiguration runtimeConfiguration
 }
 
 struct TDataNodeRestartReq {
@@ -346,8 +346,7 @@ struct TConfigNodeRegisterReq {
 
 struct TConfigNodeRegisterResp {
   1: required common.TSStatus status
-  2: optional string clusterName
-  3: optional i32 configNodeId
+  2: optional i32 configNodeId
 }
 
 struct TConfigNodeRestartReq {
@@ -438,7 +437,7 @@ struct TShowClusterResp {
   4: required map<i32, string> nodeStatus
 }
 
-struct TShowClusterParametersResp {
+struct TShowVariablesResp {
   1: required common.TSStatus status
   2: optional TClusterParameters clusterParameters
 }
@@ -664,14 +663,17 @@ service IConfigNodeRPCService {
    * Register a new DataNode into the cluster
    *
    * @return SUCCESS_STATUS if the new DataNode registered successfully
-   *         DATANODE_ALREADY_REGISTERED if the DataNode already registered
+   *         REJECT_NODE_START if the configuration chek of the DataNode to be registered fails,
+   *                           and a detailed error message will be returned.
    */
   TDataNodeRegisterResp registerDataNode(TDataNodeRegisterReq req)
 
   /**
-   * Restart a existed DataNode
+   * Restart an existed DataNode
    *
-   * @return SUCCESS_STATUS if restart DataNode success
+   * @return SUCCESS_STATUS if DataNode restart request is accepted
+   *         REJECT_NODE_START if the configuration chek of the DataNode to be restarted fails,
+   *                           and a detailed error message will be returned.
    */
   TDataNodeRestartResp restartDataNode(TDataNodeRestartReq req)
 
@@ -863,9 +865,9 @@ service IConfigNodeRPCService {
   /**
    * The Non-Seed-ConfigNode submit a registration request to the ConfigNode-leader when first startup
    *
-   * @return SUCCESS_STATUS if the AddConfigNodeProcedure submitted successfully
-   *         ERROR_GLOBAL_CONFIG if some global configurations in the Non-Seed-ConfigNode
-   *                             are inconsist with the ConfigNode-leader
+   * @return SUCCESS_STATUS if the AddConfigNodeProcedure submitted successfully.
+   *         REJECT_NODE_START if the configuration chek of the ConfigNode to be registered fails,
+   *                           and a detailed error message will be returned.
    */
   TConfigNodeRegisterResp registerConfigNode(TConfigNodeRegisterReq req)
 
@@ -875,6 +877,13 @@ service IConfigNodeRPCService {
   /** The ConfigNode-leader will notify the Non-Seed-ConfigNode that the registration success */
   common.TSStatus notifyRegisterSuccess()
 
+  /**
+   * Restart an existed ConfigNode
+   *
+   * @return SUCCESS_STATUS if ConfigNode restart request is accepted
+   *         REJECT_NODE_START if the configuration chek of the ConfigNode to be restarted fails,
+   *                           and a detailed error message will be returned.
+   */
   common.TSStatus restartConfigNode(TConfigNodeRestartReq req)
 
   /**
@@ -1000,8 +1009,8 @@ service IConfigNodeRPCService {
   /** Show cluster ConfigNodes' and DataNodes' information */
   TShowClusterResp showCluster()
 
-  /** Show cluster parameters who should be consist in the same cluster */
-  TShowClusterParametersResp showClusterParameters()
+  /** Show variables who should be consist in the same cluster */
+  TShowVariablesResp showVariables()
 
   /** Show cluster DataNodes' information */
   TShowDataNodesResp showDataNodes()