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/06 13:02:29 UTC

[GitHub] [iotdb] MiniSho opened a new pull request, #7531: [IOTDB-4550] Add NodeId to Peer

MiniSho opened a new pull request, #7531:
URL: https://github.com/apache/iotdb/pull/7531

   see: https://issues.apache.org/jira/browse/IOTDB-4550
   This pr accomplishes two things:
   
   add nodeId to class Peer
   change primary key of Peer from groupId + TEndPoint to groupId + NodeId


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


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

Posted by GitBox <gi...@apache.org>.
MiniSho commented on code in PR #7531:
URL: https://github.com/apache/iotdb/pull/7531#discussion_r991763025


##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java:
##########
@@ -70,27 +70,36 @@ public static RaftPeerId fromTEndPointToRaftPeerId(TEndPoint endpoint) {
     return RaftPeerId.valueOf(fromTEndPointToString(endpoint));
   }
 
-  public static TEndPoint formRaftPeerIdToTEndPoint(RaftPeerId id) {
-    String[] items = id.toString().split("_");
+  public static RaftPeerId fromNodeIdToRaftPeerId(int nodeId) {
+    return RaftPeerId.valueOf(String.valueOf(nodeId));
+  }
+
+  public static TEndPoint formRaftPeerAddressToTEndPoint(String address) {

Review Comment:
   Sure.



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


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

Posted by GitBox <gi...@apache.org>.
Beyyes commented on code in PR #7531:
URL: https://github.com/apache/iotdb/pull/7531#discussion_r990613590


##########
consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java:
##########
@@ -24,16 +24,19 @@
 public class ConsensusConfig {
 
   private final TEndPoint thisNode;

Review Comment:
   Renaming `thisNode` to `thisNodeEndPoint` is better?



##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java:
##########
@@ -70,27 +70,36 @@ public static RaftPeerId fromTEndPointToRaftPeerId(TEndPoint endpoint) {
     return RaftPeerId.valueOf(fromTEndPointToString(endpoint));
   }
 
-  public static TEndPoint formRaftPeerIdToTEndPoint(RaftPeerId id) {
-    String[] items = id.toString().split("_");
+  public static RaftPeerId fromNodeIdToRaftPeerId(int nodeId) {
+    return RaftPeerId.valueOf(String.valueOf(nodeId));
+  }
+
+  public static TEndPoint formRaftPeerAddressToTEndPoint(String address) {
+    String[] items = address.split(":");
     return new TEndPoint(items[0], Integer.parseInt(items[1]));
   }
 
+  public static int formRaftPeerIdToNodeId(RaftPeerId id) {
+    return Integer.parseInt(id.toString());
+  }
+
   public static TEndPoint formRaftPeerProtoToTEndPoint(RaftPeerProto proto) {

Review Comment:
   `from`?



##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java:
##########
@@ -99,11 +108,21 @@ public static List<RaftPeer> fromPeersAndPriorityToRaftPeers(List<Peer> peers, i
         .collect(Collectors.toList());
   }
 
+  // TODO: RaftPeerProto的Id是否是nodeId

Review Comment:
   Do not use Chinese



##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java:
##########
@@ -99,11 +108,21 @@ public static List<RaftPeer> fromPeersAndPriorityToRaftPeers(List<Peer> peers, i
         .collect(Collectors.toList());
   }
 
+  // TODO: RaftPeerProto的Id是否是nodeId
+  public static int formRaftPeerProtoToNodeId(RaftPeerProto proto) {

Review Comment:
   `from`?



##########
confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java:
##########
@@ -765,4 +768,8 @@ private ClusterSchemaManager getClusterSchemaManager() {
   private PartitionManager getPartitionManager() {
     return configManager.getPartitionManager();
   }
+
+  public int generateNodeId() {

Review Comment:
   Maybe it's unnecessary to create a new method.



##########
confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java:
##########
@@ -197,6 +200,13 @@ public ConfigManager() throws IOException {
 
     // ConsensusManager must be initialized last, as it would load states from disk and reinitialize
     // above managers
+    // TODO:
+    // consensusManager不做初始化,因为创建时内部的RatisConsensus需要nodeId作为server的id,但是这个id只有在向seedConfignode发送注册时才会生成

Review Comment:
   Do not use chinese



##########
confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java:
##########
@@ -383,8 +383,9 @@ public List<TConfigNodeInfo> getRegisteredConfigNodeInfoList() {
    * @param configNodeLocation The new ConfigNode
    */
   public void applyConfigNode(TConfigNodeLocation configNodeLocation) {
-    // Generate new ConfigNode's index
-    configNodeLocation.setConfigNodeId(nodeInfo.generateNextNodeId());
+    if (configNodeLocation.getConfigNodeId() != 0) {

Review Comment:
   Why add this judgement? `if (configNodeLocation.getConfigNodeId() != 0) {`



##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java:
##########
@@ -99,11 +108,21 @@ public static List<RaftPeer> fromPeersAndPriorityToRaftPeers(List<Peer> peers, i
         .collect(Collectors.toList());
   }
 
+  // TODO: RaftPeerProto的Id是否是nodeId

Review Comment:
   We can add unit test to verify it



##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java:
##########
@@ -603,8 +607,8 @@ public Peer getLeader(ConsensusGroupId groupId) {
     if (leaderId == null) {
       return null;
     }
-    TEndPoint leaderEndpoint = Utils.formRaftPeerIdToTEndPoint(leaderId);
-    return new Peer(groupId, leaderEndpoint);
+    int nodeId = Utils.formRaftPeerIdToNodeId(leaderId);
+    return new Peer(groupId, nodeId, null);

Review Comment:
   This line used endpoint before, I'm not sure if there exist problem if we replace with nodeId. 



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


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

Posted by GitBox <gi...@apache.org>.
MiniSho commented on code in PR #7531:
URL: https://github.com/apache/iotdb/pull/7531#discussion_r991919089


##########
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) {

Review Comment:
   Because statemachine receive RaftGroupMemberId and RaftPeerId as parameters, and RaftPeerId was generated by Node's TEndPoint before, but now the RaftPeerId needs to be generated by Node's id in order to support the function of modifying ip:port.



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


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

Posted by GitBox <gi...@apache.org>.
MiniSho commented on code in PR #7531:
URL: https://github.com/apache/iotdb/pull/7531#discussion_r991763120


##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java:
##########
@@ -70,27 +70,36 @@ public static RaftPeerId fromTEndPointToRaftPeerId(TEndPoint endpoint) {
     return RaftPeerId.valueOf(fromTEndPointToString(endpoint));
   }
 
-  public static TEndPoint formRaftPeerIdToTEndPoint(RaftPeerId id) {
-    String[] items = id.toString().split("_");
+  public static RaftPeerId fromNodeIdToRaftPeerId(int nodeId) {
+    return RaftPeerId.valueOf(String.valueOf(nodeId));
+  }
+
+  public static TEndPoint formRaftPeerAddressToTEndPoint(String address) {
+    String[] items = address.split(":");
     return new TEndPoint(items[0], Integer.parseInt(items[1]));
   }
 
-  public static TEndPoint formRaftPeerProtoToTEndPoint(RaftPeerProto proto) {
+  public static int frommRaftPeerIdToNodeId(RaftPeerId id) {

Review Comment:
   done



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


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

Posted by GitBox <gi...@apache.org>.
SpriCoder commented on code in PR #7531:
URL: https://github.com/apache/iotdb/pull/7531#discussion_r991408543


##########
confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java:
##########
@@ -202,9 +205,12 @@ public ConfigManager() throws IOException {
     this.loadManager = new LoadManager(this);
     this.syncManager = new SyncManager(this, syncInfo);
 
-    // ConsensusManager must be initialized last, as it would load states from disk and reinitialize
-    // above managers
-    this.consensusManager = new ConsensusManager(this, stateMachine);
+    // ConsensusManager doesn't initialize until the ConfigNode knows its own nodeId
+    this.consensusManager = null;
+  }
+
+  public void initConsensusManager() throws IOException {
+    this.consensusManager = new ConsensusManager(this, this.stateMachine);

Review Comment:
   Please add check of whether consensusManager is already initialized. if this method is called be a Thread-Safe method, then you can simply judge whether consensusManager is null. Or else you need to guarantee consensusManager only be initialized one time.



##########
confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java:
##########
@@ -265,9 +266,9 @@ TDataPartitionTableResp getOrCreateDataPartition(
   /**
    * Register ConfigNode when it is first startup
    *
-   * @return TSStatus
+   * @return TConfigNodeRegisterResp
    */

Review Comment:
   From my perspective, I think if there are no explain of param, there are no need to mention in java doc.



##########
confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java:
##########
@@ -727,8 +736,18 @@ private TSStatus checkConfigNodeGlobalConfig(TConfigNodeRegisterReq req) {
 
   @Override
   public TSStatus createPeerForConsensusGroup(List<TConfigNodeLocation> configNodeLocations) {
-    consensusManager.createPeerForConsensusGroup(configNodeLocations);
-    return StatusUtils.OK;
+    for (int i = 0; i < 30; i++) {
+      try {
+        if (consensusManager == null) {
+          Thread.sleep(1000);
+        } else {
+          consensusManager.createPeerForConsensusGroup(configNodeLocations);
+          return StatusUtils.OK;
+        }
+      } catch (Exception ignored) {
+      }

Review Comment:
   This process of Exception is too simply. From my perspective, this way may ignore some useful exception caused by createPeerForConsensusGroup, so I think you can use multi catch in level.



##########
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);

Review Comment:
   Please Avoid Magic Number in Code. You use static final param.



##########
confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java:
##########
@@ -90,7 +91,10 @@ public void active() {
 
       /* Initial startup of Seed-ConfigNode */
       if (ConfigNodeDescriptor.getInstance().isSeedConfigNode()) {
+        configManager.initConsensusManager();
+
         SystemPropertiesUtils.storeSystemParameters();
+        SystemPropertiesUtils.storeConfigNodeId(0);

Review Comment:
   Also, Please avoid magic number.



##########
confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java:
##########
@@ -239,6 +239,16 @@ public static void storeConfigNodeList(List<TConfigNodeLocation> configNodes) th
     storeSystemProperties(systemProperties);
   }
 
+  public static void storeConfigNodeId(int nodeId) throws IOException {
+    if (!systemPropertiesFile.exists()) {
+      return;
+    }

Review Comment:
   Maybe there can add some error logs.



##########
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:
   Avoid Magic Number Please.



##########
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:
   Support~. If there are only index, it's hard to debug



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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
wangchao316 commented on code in PR #7531:
URL: https://github.com/apache/iotdb/pull/7531#discussion_r991892814


##########
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) {

Review Comment:
   why TEndPoint change int ?



##########
confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java:
##########
@@ -202,9 +205,12 @@ public ConfigManager() throws IOException {
     this.loadManager = new LoadManager(this);
     this.syncManager = new SyncManager(this, syncInfo);
 
-    // ConsensusManager must be initialized last, as it would load states from disk and reinitialize
-    // above managers
-    this.consensusManager = new ConsensusManager(this, stateMachine);
+    // ConsensusManager doesn't initialize until the ConfigNode knows its own nodeId
+    this.consensusManager = null;

Review Comment:
   remove "this.consensusManager = null;"



##########
consensus/src/main/java/org/apache/iotdb/consensus/IStateMachine.java:
##########
@@ -132,9 +131,9 @@ interface EventApi {
      * can possibly be this server.
      *
      * @param groupId The id of this consensus group.
-     * @param newLeader The id of the new leader.
+     * @param newLeader The id of the new leader node.
      */
-    default void notifyLeaderChanged(ConsensusGroupId groupId, TEndPoint newLeader) {}
+    default void notifyLeaderChanged(ConsensusGroupId groupId, int newLeader) {}

Review Comment:
   rename newLeader to newLeaderId



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


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

Posted by GitBox <gi...@apache.org>.
CRZbulabula commented on code in PR #7531:
URL: https://github.com/apache/iotdb/pull/7531#discussion_r993513960


##########
confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java:
##########
@@ -208,12 +216,25 @@ private void registerConfigNode() throws StartupException, IOException {
     }
 
     for (int retry = 0; retry < 3; retry++) {
-      TSStatus status =
-          (TSStatus)
-              SyncConfigNodeClientPool.getInstance()
-                  .sendSyncRequestToConfigNodeWithRetry(
-                      targetConfigNode, req, ConfigNodeRequestType.REGISTER_CONFIG_NODE);
+      TSStatus status;
+      TConfigNodeRegisterResp resp = null;
+      Object obj =
+          SyncConfigNodeClientPool.getInstance()
+              .sendSyncRequestToConfigNodeWithRetry(
+                  targetConfigNode, req, ConfigNodeRequestType.REGISTER_CONFIG_NODE);
+
+      if (obj instanceof TConfigNodeRegisterResp) {
+        resp = (TConfigNodeRegisterResp) obj;
+        status = resp.getStatus();
+      } else {
+        status = (TSStatus) obj;
+      }

Review Comment:
   The `if` statement and the `obj` variable can be removed?



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


[GitHub] [iotdb] Beyyes merged pull request #7531: [IOTDB-4550] Add NodeId to Peer

Posted by GitBox <gi...@apache.org>.
Beyyes merged PR #7531:
URL: https://github.com/apache/iotdb/pull/7531


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


[GitHub] [iotdb] MiniSho commented on pull request #7531: [IOTDB-4550] Add NodeId to Peer

Posted by GitBox <gi...@apache.org>.
MiniSho commented on PR #7531:
URL: https://github.com/apache/iotdb/pull/7531#issuecomment-1273463449

   @WilliamSong112  @SpriCoder  Please help review the 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


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

Posted by GitBox <gi...@apache.org>.
SzyWilliam commented on code in PR #7531:
URL: https://github.com/apache/iotdb/pull/7531#discussion_r991478517


##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java:
##########
@@ -70,27 +70,36 @@ public static RaftPeerId fromTEndPointToRaftPeerId(TEndPoint endpoint) {
     return RaftPeerId.valueOf(fromTEndPointToString(endpoint));
   }
 
-  public static TEndPoint formRaftPeerIdToTEndPoint(RaftPeerId id) {
-    String[] items = id.toString().split("_");
+  public static RaftPeerId fromNodeIdToRaftPeerId(int nodeId) {
+    return RaftPeerId.valueOf(String.valueOf(nodeId));
+  }
+
+  public static TEndPoint formRaftPeerAddressToTEndPoint(String address) {
+    String[] items = address.split(":");
     return new TEndPoint(items[0], Integer.parseInt(items[1]));
   }
 
-  public static TEndPoint formRaftPeerProtoToTEndPoint(RaftPeerProto proto) {
+  public static int frommRaftPeerIdToNodeId(RaftPeerId id) {
+    return Integer.parseInt(id.toString());
+  }
+
+  public static TEndPoint fromRaftPeerProtoToTEndPoint(RaftPeerProto proto) {
     String[] items = proto.getAddress().split(":");
     return new TEndPoint(items[0], Integer.parseInt(items[1]));
   }
 
   // priority is used as ordinal of leader election
-  public static RaftPeer fromTEndPointAndPriorityToRaftPeer(TEndPoint endpoint, int priority) {
+  public static RaftPeer fromTEndPointAndPriorityToRaftPeer(

Review Comment:
   we should rename this function since now it includes NodeID & TEndPoint



##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java:
##########
@@ -70,27 +70,36 @@ public static RaftPeerId fromTEndPointToRaftPeerId(TEndPoint endpoint) {
     return RaftPeerId.valueOf(fromTEndPointToString(endpoint));
   }
 
-  public static TEndPoint formRaftPeerIdToTEndPoint(RaftPeerId id) {
-    String[] items = id.toString().split("_");
+  public static RaftPeerId fromNodeIdToRaftPeerId(int nodeId) {
+    return RaftPeerId.valueOf(String.valueOf(nodeId));
+  }
+
+  public static TEndPoint formRaftPeerAddressToTEndPoint(String address) {
+    String[] items = address.split(":");
     return new TEndPoint(items[0], Integer.parseInt(items[1]));
   }
 
-  public static TEndPoint formRaftPeerProtoToTEndPoint(RaftPeerProto proto) {
+  public static int frommRaftPeerIdToNodeId(RaftPeerId id) {

Review Comment:
   typo



##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java:
##########
@@ -70,27 +70,36 @@ public static RaftPeerId fromTEndPointToRaftPeerId(TEndPoint endpoint) {
     return RaftPeerId.valueOf(fromTEndPointToString(endpoint));
   }
 
-  public static TEndPoint formRaftPeerIdToTEndPoint(RaftPeerId id) {
-    String[] items = id.toString().split("_");
+  public static RaftPeerId fromNodeIdToRaftPeerId(int nodeId) {
+    return RaftPeerId.valueOf(String.valueOf(nodeId));
+  }
+
+  public static TEndPoint formRaftPeerAddressToTEndPoint(String address) {

Review Comment:
   I think it's my fault to use `form` instead of `from`🥵, can you fix it in this PR?



##########
consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java:
##########
@@ -70,27 +70,36 @@ public static RaftPeerId fromTEndPointToRaftPeerId(TEndPoint endpoint) {
     return RaftPeerId.valueOf(fromTEndPointToString(endpoint));
   }
 
-  public static TEndPoint formRaftPeerIdToTEndPoint(RaftPeerId id) {
-    String[] items = id.toString().split("_");
+  public static RaftPeerId fromNodeIdToRaftPeerId(int nodeId) {
+    return RaftPeerId.valueOf(String.valueOf(nodeId));
+  }
+
+  public static TEndPoint formRaftPeerAddressToTEndPoint(String address) {
+    String[] items = address.split(":");
     return new TEndPoint(items[0], Integer.parseInt(items[1]));
   }
 
-  public static TEndPoint formRaftPeerProtoToTEndPoint(RaftPeerProto proto) {
+  public static int frommRaftPeerIdToNodeId(RaftPeerId id) {
+    return Integer.parseInt(id.toString());
+  }
+
+  public static TEndPoint fromRaftPeerProtoToTEndPoint(RaftPeerProto proto) {
     String[] items = proto.getAddress().split(":");
     return new TEndPoint(items[0], Integer.parseInt(items[1]));
   }
 
   // priority is used as ordinal of leader election
-  public static RaftPeer fromTEndPointAndPriorityToRaftPeer(TEndPoint endpoint, int priority) {
+  public static RaftPeer fromTEndPointAndPriorityToRaftPeer(
+      int nodeId, TEndPoint endpoint, int priority) {
     return RaftPeer.newBuilder()
-        .setId(fromTEndPointToRaftPeerId(endpoint))
+        .setId(fromNodeIdToRaftPeerId(nodeId))
         .setAddress(HostAddress(endpoint))
         .setPriority(priority)
         .build();
   }
 
   public static RaftPeer fromTEndPointAndPriorityToRaftPeer(Peer peer, int priority) {

Review Comment:
   Same as above



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


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

Posted by GitBox <gi...@apache.org>.
CRZbulabula commented on PR #7531:
URL: https://github.com/apache/iotdb/pull/7531#issuecomment-1273390785

   Don't forget to update the ConfigNode startup process doc~


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