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 2022/10/10 14:19:54 UTC

[GitHub] [iotdb] CRZbulabula commented on a diff in pull request #7531: [IOTDB-4550] Add NodeId to Peer

CRZbulabula commented on code in PR #7531:
URL: https://github.com/apache/iotdb/pull/7531#discussion_r991311874


##########
confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/PartitionRegionStateMachine.java:
##########
@@ -145,16 +147,18 @@ protected DataSet read(ConfigPhysicalPlan plan) {
   }
 
   @Override
-  public void notifyLeaderChanged(ConsensusGroupId groupId, TEndPoint newLeader) {
-    if (currentNode.equals(newLeader)) {
+  public void notifyLeaderChanged(ConsensusGroupId groupId, int newLeader) {
+    if (currentNodeId == newLeader) {
       LOGGER.info("Current node {} becomes Leader", newLeader);
       configManager.getProcedureManager().shiftExecutor(true);
       configManager.getLoadManager().startLoadBalancingService();
       configManager.getNodeManager().startHeartbeatService();
       configManager.getPartitionManager().startRegionCleaner();
     } else {
       LOGGER.info(
-          "Current node {} is not longer the leader, the new leader is {}", currentNode, newLeader);
+          "Current node {} is not longer the leader, the new leader is {}",
+          currentNodeId,
+          newLeader);

Review Comment:
   Here we should log the record of leadership transition. It's unconspicuous to just record the index however. Can we log the ip:port information through the ConsensusManager along the way?



##########
consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java:
##########
@@ -23,24 +23,31 @@
 
 public class ConsensusConfig {
 
-  private final TEndPoint thisNode;
+  private final TEndPoint thisNodeEndPoint;
+  private final int thisNodeId;
   private final String storageDir;
   private final RatisConfig ratisConfig;
   private final MultiLeaderConfig multiLeaderConfig;
 
   private ConsensusConfig(
       TEndPoint thisNode,
+      int thisNodeId,
       String storageDir,
       RatisConfig ratisConfig,
       MultiLeaderConfig multiLeaderConfig) {
-    this.thisNode = thisNode;
+    this.thisNodeEndPoint = thisNode;
+    this.thisNodeId = thisNodeId;
     this.storageDir = storageDir;
     this.ratisConfig = ratisConfig;
     this.multiLeaderConfig = multiLeaderConfig;
   }
 
-  public TEndPoint getThisNode() {
-    return thisNode;
+  public TEndPoint getThisNodeEndPoint() {
+    return thisNodeEndPoint;
+  }
+
+  public int getThisNodeId() {
+    return thisNodeId;

Review Comment:
   Will it be better if we merge these two interfaces into one? Because the NodeId and the NodeEndPoint are both rudimentary parameters for the ConsensusPeer.



##########
confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java:
##########
@@ -657,21 +663,24 @@ public TPermissionInfoResp checkUserPrivileges(
   }
 
   @Override
-  public TSStatus registerConfigNode(TConfigNodeRegisterReq req) {
+  public TConfigNodeRegisterResp registerConfigNode(TConfigNodeRegisterReq req) {
     // Check global configuration
     TSStatus status = confirmLeader();
 
     if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
       TSStatus errorStatus = checkConfigNodeGlobalConfig(req);
       if (errorStatus != null) {
-        return errorStatus;
+        return new TConfigNodeRegisterResp().setStatus(errorStatus).setConfigNodeId(-1);
       }
 
+      int nodeId = nodeManager.generateNodeId();
+      req.getConfigNodeLocation().setConfigNodeId(nodeId);
+
       procedureManager.addConfigNode(req);
-      return StatusUtils.OK;
+      return new TConfigNodeRegisterResp().setStatus(StatusUtils.OK).setConfigNodeId(nodeId);
     }
 
-    return status;
+    return new TConfigNodeRegisterResp().setStatus(status).setConfigNodeId(-1);
   }

Review Comment:
   The main logic of this interface should be moved into NodeManager now~



##########
consensus/src/main/java/org/apache/iotdb/consensus/common/Peer.java:
##########
@@ -46,16 +48,22 @@ public TEndPoint getEndpoint() {
     return endpoint;
   }
 
+  public int getNodeId() {
+    return nodeId;
+  }
+
   public void serialize(DataOutputStream stream) {
     ThriftCommonsSerDeUtils.serializeTConsensusGroupId(
         groupId.convertToTConsensusGroupId(), stream);
+    ThriftCommonsSerDeUtils.serializeInt(nodeId, stream);

Review Comment:
   Use `BasicStructureSerDeUtil` is enough~



##########
node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtils.java:
##########
@@ -81,6 +82,18 @@ public static TEndPoint deserializeTEndPoint(ByteBuffer buffer) {
     return endPoint;
   }
 
+  public static void serializeInt(int nodeId, DataOutputStream stream) {
+    try {
+      stream.writeInt(nodeId);
+    } catch (IOException e) {
+      throw new ThriftSerDeException("Write Int failed: ", e);
+    }
+  }
+
+  public static int deserializeInt(ByteBuffer buffer) {
+    return buffer.getInt();
+  }
+

Review Comment:
   These lines of code can be removed since using `BasicStructureSerDeUtil` is enough~



##########
confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java:
##########
@@ -109,6 +109,11 @@ private void loadProps() {
       Properties properties = new Properties();
       properties.load(inputStream);
 
+      conf.setConfigNodeId(
+          Integer.parseInt(
+              properties.getProperty(
+                  IoTDBConstant.CONFIG_NODE_ID, String.valueOf(conf.getConfigNodeId()))));

Review Comment:
   Here in ConfigNodeDescriptor we read from the `iotdb-confignode.properties` file to set the corresponding parameter in ConfigNodeConfig. However the parameter `config_node_id` is recorded in the `confignode-system.properties` file. Therefore, please remove these lines of code~



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

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