You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2021/03/02 05:42:11 UTC

[GitHub] [iotdb] LebronAl commented on a change in pull request #2740: [IOTDB-1043] Change rpc port &use internal ip for communication between nodes in cluster

LebronAl commented on a change in pull request #2740:
URL: https://github.com/apache/iotdb/pull/2740#discussion_r585241963



##########
File path: cluster/src/assembly/resources/conf/iotdb-cluster.properties
##########
@@ -17,36 +17,40 @@
 # under the License.
 #
 
-cluster_rpc_ip=127.0.0.1
-
 #-------------------------------------------IMPORTANT---------------------------------------------#
-# Note that the system will automatically create a heartbeat port for each metadata service       #
+# 1. Note that the system will automatically create a heartbeat port for each metadata service    #
 # and data service. The default metadata heartbeat port is internal_meta_port + 1,                #
 # The default data heartbeat port is internal_data_port + 1.                                      #
 # So when you configure these two items and seed_nodes, pay attention to reserve a port for       #

Review comment:
       remove outdated comments for `seed_nodes`

##########
File path: cluster/src/assembly/resources/conf/iotdb-cluster.properties
##########
@@ -17,36 +17,40 @@
 # under the License.
 #
 
-cluster_rpc_ip=127.0.0.1
-
 #-------------------------------------------IMPORTANT---------------------------------------------#
-# Note that the system will automatically create a heartbeat port for each metadata service       #
+# 1. Note that the system will automatically create a heartbeat port for each metadata service    #
 # and data service. The default metadata heartbeat port is internal_meta_port + 1,                #
 # The default data heartbeat port is internal_data_port + 1.                                      #
 # So when you configure these two items and seed_nodes, pay attention to reserve a port for       #
 # heartbeat service.                                                                              #
+# 2. If open_server_rpc_port is set to true, the server module's RPC port will be turned on,      #
+# and the server module's RPC port will be set to rpc_port (in iotdb-engines.properties) + 1,     #
+# so this port should also be reserved.                                                           #
 #-------------------------------------------IMPORTANT---------------------------------------------#
 
+internal_ip=127.0.0.1
+
 # port for metadata service
 internal_meta_port=9003
 
 # port for data service
 internal_data_port=40010
 
-# port for client service
-cluster_rpc_port=55560
+# whether open port for server module (for debug purpose)
+# if true, the single's server rpc_port will be changed to rpc_port (in iotdb-engines.properties) + 1
+open_server_rpc_port=false
 
-# comma-separated {IP/DOMAIN}:meta_port:data_port:client_port pairs
-# when used by start-node.sh(.bat), this configuration means the nodes that will form the initial
-# cluster, some every node that use start-node.sh(.bat) should have the SAME SEED_NODES, or the
+# comma-separated {IP/DOMAIN}:meta_port pairs, when used by start-node.sh(.bat),
+# this configuration means the nodes that will form the initial cluster,
+# some every node that use start-node.sh(.bat) should have the SAME SEED_NODES, or the

Review comment:
       suggested changes
   
   ```
   every node that use start-node.sh(.bat) should have the same SEED_NODES, or the
   ```

##########
File path: cluster/src/assembly/resources/conf/iotdb-cluster.properties
##########
@@ -17,36 +17,40 @@
 # under the License.
 #
 
-cluster_rpc_ip=127.0.0.1
-
 #-------------------------------------------IMPORTANT---------------------------------------------#
-# Note that the system will automatically create a heartbeat port for each metadata service       #
+# 1. Note that the system will automatically create a heartbeat port for each metadata service    #
 # and data service. The default metadata heartbeat port is internal_meta_port + 1,                #
 # The default data heartbeat port is internal_data_port + 1.                                      #
 # So when you configure these two items and seed_nodes, pay attention to reserve a port for       #
 # heartbeat service.                                                                              #
+# 2. If open_server_rpc_port is set to true, the server module's RPC port will be turned on,      #
+# and the server module's RPC port will be set to rpc_port (in iotdb-engines.properties) + 1,     #
+# so this port should also be reserved.                                                           #
 #-------------------------------------------IMPORTANT---------------------------------------------#
 
+internal_ip=127.0.0.1
+
 # port for metadata service
 internal_meta_port=9003
 
 # port for data service
 internal_data_port=40010
 
-# port for client service
-cluster_rpc_port=55560
+# whether open port for server module (for debug purpose)
+# if true, the single's server rpc_port will be changed to rpc_port (in iotdb-engines.properties) + 1

Review comment:
       typo `single's`?

##########
File path: docs/UserGuide/Server/Cluster Setup.md
##########
@@ -35,45 +35,27 @@ To start the service of one of the nodes, you need to execute the following comm
 
 ```bash
 # Unix/OS X
-> nohup sbin/start-node.sh >/dev/null 2>&1 &
-or
-> nohup sbin/start-node.sh -c <conf_path> -internal_meta_port 9003 >/dev/null 2>&1 &
+> nohup sbin/start-node.sh [printgc] [<conf_path>] >/dev/null 2>&1 &
 
 # Windows
 > sbin\start-node.bat
 or
-> sbin\start-node.bat -c <conf_path> -internal_meta_port 9003
+> sbin\start-node.bat <conf_path>
 ```
+`printgc` means whether enable the gc and print gc logs when start the node,  
+`<conf_path>` use the configuration file in the `conf_path` folder to override the default configuration file.
 
-`-c <conf_path>` use the configuration file in the `conf_path` folder to override the default configuration file; 
-`-internal_meta_port 9003` overrides the specific configuration item `internal_meta_port`.
-The currently supported items to overwrite the original configurations when starting IoTDB are the followings :
-`internal_meta_port, internal_data_port, cluster_rpc_port, seed_nodes`. 
-When both exist, the specified configuration item will overwrite the configurations in the configuration file.
-
-## Example of pseudo-distributed scaffolding for 3 nodes and 2 replicas
+## Example of pseudo-distributed scaffolding for 3 nodes and 3 replicas
 ```bash
-# First step (Note that this path is not suitable for Windows MinGW)
 > mvn clean package -pl cluster -am -Dmaven.test.skip=true
-> cp -rf ./cluster/target/cluster-0.11.0-SNAPSHOT ./cluster/target/cluster-0.11.0-SNAPSHOT1
-> cp -rf ./cluster/target/cluster-0.11.0-SNAPSHOT ./cluster/target/cluster-0.11.0-SNAPSHOT2
-> sed -i -e 's/6667/6668/g' ./cluster/target/cluster-0.11.0-SNAPSHOT1/conf/iotdb-engine.properties
-> sed -i -e 's/6667/6669/g' ./cluster/target/cluster-0.11.0-SNAPSHOT2/conf/iotdb-engine.properties
-
-# Second step: Unix/OS X/Windows (git bash or WSL)
-> sed -i -e 's/31999/32000/g' ./cluster/target/cluster-0.11.0-SNAPSHOT1/conf/iotdb-env.sh
-> sed -i -e 's/31999/32001/g' ./cluster/target/cluster-0.11.0-SNAPSHOT2/conf/iotdb-env.sh
+> cp -rf ./cluster/target/cluster-0.12.0-SNAPSHOT ./cluster/target/cluster-0.12.0-SNAPSHOT1
+> cp -rf ./cluster/target/cluster-0.12.0-SNAPSHOT ./cluster/target/cluster-0.12.0-SNAPSHOT2
+
 > chmod -R 777 ./cluster/target/
-> nohup ./cluster/target/cluster-0.11.0-SNAPSHOT/sbin/start-node.sh >/dev/null 2>&1 &
-> nohup ./cluster/target/cluster-0.11.0-SNAPSHOT1/sbin/start-node.sh -internal_meta_port 9005 -internal_data_port 40012 -cluster_rpc_port 55561 >/dev/null 2>&1 &
-> nohup ./cluster/target/cluster-0.11.0-SNAPSHOT2/sbin/start-node.sh -internal_meta_port 9007 -internal_data_port 40014 -cluster_rpc_port 55562 >/dev/null 2>&1 &
-
-# Second step: Windows (MinGW)
-> sed -i -e 's/31999/32000/g'  cluster\target\cluster-0.11.0-SNAPSHOT\conf\iotdb-env.bat
-> sed -i -e 's/31999/32001/g'  cluster\target\cluster-0.11.0-SNAPSHOT\conf\iotdb-env.bat
-> nohup cluster\target\cluster-0.11.0-SNAPSHOT\sbin\start-node.bat 
-> nohup cluster\target\cluster-0.11.0-SNAPSHOT1\sbin\start-node.bat  -internal_meta_port 9005 -internal_data_port 40012 -cluster_rpc_port 55561
-> nohup cluster\target\cluster-0.11.0-SNAPSHOT2\sbin\start-node.bat  -internal_meta_port 9007 -internal_data_port 40014 -cluster_rpc_port 55562
+
+> nohup ./cluster/target/cluster-0.12.0-SNAPSHOT/sbin/start-node.sh  ./cluster/target/test-classes/node1conf/ >/dev/null 2>&1 &
+> nohup ./cluster/target/cluster-0.12.0-SNAPSHOT1/sbin/start-node.sh ./cluster/target/test-classes/node2conf/ >/dev/null 2>&1 &

Review comment:
       it seems there is no need to cp snapshot as long as you specific different conf?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/ClientMain.java
##########
@@ -203,7 +203,7 @@ private static void doQuery(boolean noOption, CommandLine commandLine)
         queryPorts = parseIntArray(commandLine.getOptionValue(PARAM_QUERY_PORTS));
       }
       if (queryPorts == null) {
-        queryPorts = new int[] {55560, 55561, 55562};
+        queryPorts = new int[] {port, port + 2, port + 4};

Review comment:
       It seems that queryPorts should be `{port, port + 1, port + 2}`?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
##########
@@ -28,20 +28,19 @@
 
   static final String CONFIG_NAME = "iotdb-cluster.properties";
 
-  private String clusterRpcIp = "127.0.0.1";
+  private String internalIp = "127.0.0.1";
   private int internalMetaPort = 9003;
   private int internalDataPort = 40010;
-  private int clusterRpcPort = 55560;
+  private int clusterRpcPort = 6667;

Review comment:
       why not just initialized this port with`IoTDBDescriptor.getInstance().getConfig().getRpcPort()` directly?

##########
File path: cluster/src/assembly/resources/conf/iotdb-cluster.properties
##########
@@ -17,36 +17,40 @@
 # under the License.
 #
 
-cluster_rpc_ip=127.0.0.1
-
 #-------------------------------------------IMPORTANT---------------------------------------------#
-# Note that the system will automatically create a heartbeat port for each metadata service       #
+# 1. Note that the system will automatically create a heartbeat port for each metadata service    #
 # and data service. The default metadata heartbeat port is internal_meta_port + 1,                #
 # The default data heartbeat port is internal_data_port + 1.                                      #
 # So when you configure these two items and seed_nodes, pay attention to reserve a port for       #
 # heartbeat service.                                                                              #
+# 2. If open_server_rpc_port is set to true, the server module's RPC port will be turned on,      #
+# and the server module's RPC port will be set to rpc_port (in iotdb-engines.properties) + 1,     #
+# so this port should also be reserved.                                                           #
 #-------------------------------------------IMPORTANT---------------------------------------------#
 

Review comment:
       maybe some comments for internal_ip?

##########
File path: cluster/src/assembly/resources/sbin/start-node.sh
##########
@@ -88,7 +85,7 @@ launch_service()
 	iotdb_parms="$iotdb_parms -DIOTDB_HOME=${IOTDB_HOME}"
 	iotdb_parms="$iotdb_parms -DTSFILE_HOME=${IOTDB_HOME}"
 	iotdb_parms="$iotdb_parms -DIOTDB_CONF=${IOTDB_CONF}"
-	iotdb_parms="$iotdb_parms -DCLUSTER_CONF=${IOTDB_CONF}"
+	iotdb_parms="$iotdb_parms -DTSFILE_CONF=${IOTDB_CONF}"

Review comment:
       why replace `DCLUSTER_CONF ` to `DTSFILE_CONF ` here while just deleteing `DCLUSTER_CONF` in `start-node.bat`?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java
##########
@@ -179,10 +180,11 @@ private static void startServerCheck() throws StartupException {
     // assert this node is in seed nodes list
     Node localNode = new Node();
     localNode
-        .setIp(config.getClusterRpcIp())
+        .setIp(config.getInternalIp())
         .setMetaPort(config.getInternalMetaPort())
         .setDataPort(config.getInternalDataPort())
-        .setClientPort(config.getClusterRpcPort());
+        .setClientPort(config.getClusterRpcPort())

Review comment:
       should be `IoTDBDescriptor.getInstance().getConfig().getRpcPort()`? 

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java
##########
@@ -48,17 +43,26 @@
 
   private static final Logger logger = LoggerFactory.getLogger(ClusterDescriptor.class);
   private static final ClusterDescriptor INSTANCE = new ClusterDescriptor();
-
-  private static final String OPTION_INTERVAL_META_PORT = "internal_meta_port";
-  private static final String OPTION_INTERVAL_DATA_PORT = "internal_data_port";
-  private static final String OPTION_CLUSTER_RPC_PORT = "cluster_rpc_port";
-  private static final String OPTION_SEED_NODES = "seed_nodes";
-
-  private ClusterConfig config = new ClusterConfig();
-  private static CommandLine commandLine;
+  private final ClusterConfig config = new ClusterConfig();
 
   private ClusterDescriptor() {
+    // copy needed configurations from the server's config to the cluster.
+    config.setClusterRpcPort(IoTDBDescriptor.getInstance().getConfig().getRpcPort());
+
+    // then load settings from cluster's file.
+    // so, iotdb-cluster.properties can overwrite iotdb-properties.

Review comment:
       typo `iotdb-properties`?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java
##########
@@ -60,36 +59,38 @@
   // send a request to remove a node, more arguments: ip-of-removed-node
   // metaport-of-removed-node
   private static final String MODE_REMOVE = "-r";
-  // the separator between the cluster configuration and the single-server configuration
-  private static final String SERVER_CONF_SEPARATOR = "-sc";
+
   private static MetaClusterServer metaServer;
 
   public static void main(String[] args) {
     if (args.length < 1) {
       logger.error(
-          "Usage: <-s|-a|-r> [-internal_meta_port <internal meta port>] "
-              + "[-internal_data_port <internal data port>] "
-              + "[-cluster_rpc_port <cluster rpc port>] "
-              + "[-seed_nodes <node1:meta_port:data_port:cluster_rpc_port,"
-              + "node2:meta_port:data_port:cluster_rpc_port,"
-              + "...,noden:meta_port:data_port:cluster_rpc_port>] "
-              + "[-sc] "
-              + "[-rpc_port <rpc port>]");
+          "Usage: <-s|-a|-r> "

Review comment:
       where are the remaining commands? such as `-internal_data_port`?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
##########
@@ -28,20 +28,19 @@
 
   static final String CONFIG_NAME = "iotdb-cluster.properties";
 
-  private String clusterRpcIp = "127.0.0.1";
+  private String internalIp = "127.0.0.1";
   private int internalMetaPort = 9003;
   private int internalDataPort = 40010;
-  private int clusterRpcPort = 55560;
+  private int clusterRpcPort = 6667;
 
-  /** each one is a "<IP | domain name>:<meta port>:<data port>:<client port></>" string tuple */
+  /** each one is a {internalIp | domain name}:{meta port} string tuple */
   private List<String> seedNodeUrls =
-      Arrays.asList(
-          "127.0.0.1:9003:40010:55560", "127.0.0.1:9005:40012:55561", "127.0.0.1:9007:40014:55562");
+      Arrays.asList(String.format("%s:%d", internalIp, internalMetaPort));

Review comment:
       why not use seed_nodes?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java
##########
@@ -89,112 +93,36 @@ public String getPropsUrl() {
     return url;
   }
 
-  public void replaceProps(String[] params) {

Review comment:
       so we will not support specify port by command line?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/utils/ClusterNode.java
##########
@@ -73,6 +75,9 @@ public String toString() {
         + dataPort
         + ", clientPort="
         + clientPort
+        + ", clientIp='"
+        + clientIp
+        + '\''

Review comment:
       what's this?

##########
File path: cluster/src/test/java/org/apache/iotdb/cluster/utils/Constants.java
##########
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.cluster.utils;
+
+public class Constants {

Review comment:
       It seems that this class only be used in test? if so, why not remove it to cluster test module?

##########
File path: docs/zh/UserGuide/Server/Cluster Setup.md
##########
@@ -31,43 +31,25 @@ __集群模式目前是测试版!请谨慎在生产环境中使用。__
 
 ```bash
 # Unix/OS X
-> nohup sbin/start-node.sh >/dev/null 2>&1 &
-or
-> nohup sbin/start-node.sh -c <conf_path> -internal_meta_port 9003 >/dev/null 2>&1 &
+> nohup sbin/start-node.sh [printgc] [<conf_path>] >/dev/null 2>&1 &
 
 # Windows
 > sbin\start-node.bat
-or
-> sbin\start-node.bat -c <conf_path> -internal_meta_port 9003
 ```
+`printgc`表示在启动的时候,会开启GC日志。
+`<conf_path>`使用`conf_path`文件夹里面的配置文件覆盖默认配置文件。
 
-`-c <conf_path>`使用`conf_path`文件夹里面的配置文件覆盖默认配置文件; `-internal_meta_port 9003`覆盖特定配置项`internal_meta_port`的配置,
-目前支持的启动覆盖原有配置的配置项有:
-`internal_meta_port、internal_data_port、cluster_rpc_port、seed_nodes`。当配置文件和配置项都被指定的时候,指定配置项的配置会覆盖配置文件中的配置。
-
-## 3节点2副本伪分布式搭建示例
+## 3节点3副本伪分布式搭建示例
 ```bash
-# 第一步 (注意以下路径在 Windows MinGW 中并不适用)
 > mvn clean package -pl cluster -am -Dmaven.test.skip=true
-> cp -rf ./cluster/target/cluster-0.11.0-SNAPSHOT ./cluster/target/cluster-0.11.0-SNAPSHOT1
-> cp -rf ./cluster/target/cluster-0.11.0-SNAPSHOT ./cluster/target/cluster-0.11.0-SNAPSHOT2
-> sed -i -e 's/6667/6668/g' ./cluster/target/cluster-0.11.0-SNAPSHOT1/conf/iotdb-engine.properties
-> sed -i -e 's/6667/6669/g' ./cluster/target/cluster-0.11.0-SNAPSHOT2/conf/iotdb-engine.properties
-
-# 第二步: Unix/OS X/Windows (git bash or WSL)
-> sed -i -e 's/31999/32000/g' ./cluster/target/cluster-0.11.0-SNAPSHOT1/conf/iotdb-env.sh
-> sed -i -e 's/31999/32001/g' ./cluster/target/cluster-0.11.0-SNAPSHOT2/conf/iotdb-env.sh
+> cp -rf ./cluster/target/cluster-0.12.0-SNAPSHOT ./cluster/target/cluster-0.12.0-SNAPSHOT1
+> cp -rf ./cluster/target/cluster-0.12.0-SNAPSHOT ./cluster/target/cluster-0.12.0-SNAPSHOT2
+
 > chmod -R 777 ./cluster/target/
-> nohup ./cluster/target/cluster-0.11.0-SNAPSHOT/sbin/start-node.sh >/dev/null 2>&1 &
-> nohup ./cluster/target/cluster-0.11.0-SNAPSHOT1/sbin/start-node.sh -internal_meta_port 9005 -internal_data_port 40012 -cluster_rpc_port 55561 >/dev/null 2>&1 &
-> nohup ./cluster/target/cluster-0.11.0-SNAPSHOT2/sbin/start-node.sh -internal_meta_port 9007 -internal_data_port 40014 -cluster_rpc_port 55562 >/dev/null 2>&1 &
-
-# 第二步: Windows (MinGW)
-> sed -i -e 's/31999/32000/g'  cluster\target\cluster-0.11.0-SNAPSHOT\conf\iotdb-env.bat
-> sed -i -e 's/31999/32001/g'  cluster\target\cluster-0.11.0-SNAPSHOT\conf\iotdb-env.bat
-> nohup cluster\target\cluster-0.11.0-SNAPSHOT\sbin\start-node.bat 
-> nohup cluster\target\cluster-0.11.0-SNAPSHOT1\sbin\start-node.bat  -internal_meta_port 9005 -internal_data_port 40012 -cluster_rpc_port 55561
-> nohup cluster\target\cluster-0.11.0-SNAPSHOT2\sbin\start-node.bat  -internal_meta_port 9007 -internal_data_port 40014 -cluster_rpc_port 55562
+
+> nohup ./cluster/target/cluster-0.12.0-SNAPSHOT/sbin/start-node.sh  ./cluster/target/test-classes/node1conf/ >/dev/null 2>&1 &

Review comment:
       same as above

##########
File path: thrift/src/main/thrift/cluster.thrift
##########
@@ -114,6 +114,8 @@ struct Node {
   3: required int nodeIdentifier
   4: required int dataPort
   5: required int clientPort
+  // When the cluster is set up for the first time, the clientIp of other nodes is unknown
+  6: required string clientIp

Review comment:
       As clientIP is created, how about change `ip` to another name just like 'internal_ip'? 

##########
File path: cluster/src/test/resources/node2conf/iotdb-engine.properties
##########
@@ -0,0 +1,29 @@
+#Licensed to the Apache Software Foundation (ASF) under one
+#or more contributor license agreements.  See the NOTICE file
+#distributed with this work for additional information
+#regarding copyright ownership.  The ASF licenses this file
+#to you under the Apache License, Version 2.0 (the
+#"License"); you may not use this file except in compliance
+#with the License.  You may obtain a copy of the License at
+#
+#http://www.apache.org/licenses/LICENSE-2.0
+#
+#Unless required by applicable law or agreed to in writing,
+#software distributed under the License is distributed on an
+#"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#KIND, either express or implied.  See the License for the
+#specific language governing permissions and limitations
+#under the License.
+
+
+base_dir=target/tmp2

Review comment:
       how about manage these directories just like `target/node1, target/node2, target/node3`?,and then the parameters just like `base_dir` 
    in `node1conf/iotdb-engine.properties` can be `base_dir=target/node1/tmp`.I think it's better so that I can enter `target/node1` and all sub directories in this directory will belongs to node1? 

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java
##########
@@ -48,17 +43,26 @@
 
   private static final Logger logger = LoggerFactory.getLogger(ClusterDescriptor.class);
   private static final ClusterDescriptor INSTANCE = new ClusterDescriptor();
-
-  private static final String OPTION_INTERVAL_META_PORT = "internal_meta_port";
-  private static final String OPTION_INTERVAL_DATA_PORT = "internal_data_port";
-  private static final String OPTION_CLUSTER_RPC_PORT = "cluster_rpc_port";
-  private static final String OPTION_SEED_NODES = "seed_nodes";
-
-  private ClusterConfig config = new ClusterConfig();
-  private static CommandLine commandLine;
+  private final ClusterConfig config = new ClusterConfig();
 
   private ClusterDescriptor() {
+    // copy needed configurations from the server's config to the cluster.
+    config.setClusterRpcPort(IoTDBDescriptor.getInstance().getConfig().getRpcPort());

Review comment:
       it seems the port in clusterConfig can be initialized with `IoTDBDescriptor.getInstance().getConfig().getRpcPort()` directly?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org