You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ta...@apache.org on 2023/08/31 06:32:24 UTC

[iotdb] branch rel/1.2 updated: [To rel/1.2][IOTDB-6129] ConfigNode restarts without relying on Seed-ConfigNode (#11003)

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

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


The following commit(s) were added to refs/heads/rel/1.2 by this push:
     new e1ab0540d37 [To rel/1.2][IOTDB-6129] ConfigNode restarts without relying on Seed-ConfigNode (#11003)
e1ab0540d37 is described below

commit e1ab0540d37c22427c6deb29d0a815f7c906ac57
Author: Li Yu Heng <li...@126.com>
AuthorDate: Thu Aug 31 14:32:17 2023 +0800

    [To rel/1.2][IOTDB-6129] ConfigNode restarts without relying on Seed-ConfigNode (#11003)
---
 .../apache/iotdb/it/env/cluster/AbstractEnv.java   | 40 +++++++++++-----
 .../it/cluster/IoTDBClusterNodeErrorStartUpIT.java | 27 -----------
 .../it/cluster/IoTDBClusterRestartIT.java          | 28 +++++++++--
 .../confignode/it/utils/ConfigNodeTestUtils.java   |  7 ---
 .../client/sync/SyncConfigNodeClientPool.java      |  3 --
 .../confignode/conf/SystemPropertiesUtils.java     |  4 +-
 .../iotdb/confignode/manager/ConfigManager.java    | 23 ---------
 .../apache/iotdb/confignode/manager/IManager.java  |  3 --
 .../iotdb/confignode/service/ConfigNode.java       | 54 ++++------------------
 .../thrift/ConfigNodeRPCServiceProcessor.java      | 11 -----
 .../iotdb/db/protocol/client/ConfigNodeClient.java |  6 ---
 .../src/main/thrift/confignode.thrift              | 14 ------
 12 files changed, 65 insertions(+), 155 deletions(-)

diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/AbstractEnv.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/AbstractEnv.java
index af610a784b4..62387f8524f 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/AbstractEnv.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/AbstractEnv.java
@@ -65,6 +65,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
@@ -191,7 +192,7 @@ public abstract class AbstractEnv implements BaseEnv {
       fail();
     }
 
-    testWorking();
+    testWorkingNoUnknown();
   }
 
   public String getTestClassName() {
@@ -205,7 +206,26 @@ public abstract class AbstractEnv implements BaseEnv {
     return "UNKNOWN-IT";
   }
 
-  public void testWorking() {
+  private Map<String, Integer> countNodeStatus(Map<Integer, String> nodeStatus) {
+    Map<String, Integer> result = new HashMap<>();
+    nodeStatus.values().forEach(status -> result.put(status, result.getOrDefault(status, 0) + 1));
+    return result;
+  }
+
+  public void testWorkingNoUnknown() {
+    testWorking(nodeStatusMap -> nodeStatusMap.values().stream().noneMatch("Unknown"::equals));
+  }
+
+  public void testWorkingOneUnknownOtherRunning() {
+    testWorking(
+        nodeStatus -> {
+          Map<String, Integer> count = countNodeStatus(nodeStatus);
+          return count.getOrDefault("Unknown", 0) == 1
+              && count.getOrDefault("Running", 0) == nodeStatus.size() - 1;
+        });
+  }
+
+  public void testWorking(Predicate<Map<Integer, String>> statusCheck) {
     logger.info("Testing DataNode connection...");
     List<String> endpoints =
         dataNodeWrapperList.stream()
@@ -234,7 +254,7 @@ public abstract class AbstractEnv implements BaseEnv {
       long startTime = System.currentTimeMillis();
       testDelegate.requestAll();
       if (!configNodeWrapperList.isEmpty()) {
-        checkNodeHeartbeat();
+        checkNodeHeartbeat(statusCheck);
       }
       logger.info("Start cluster costs: {}s", (System.currentTimeMillis() - startTime) / 1000.0);
     } catch (Exception e) {
@@ -243,7 +263,7 @@ public abstract class AbstractEnv implements BaseEnv {
     }
   }
 
-  private void checkNodeHeartbeat() throws Exception {
+  private void checkNodeHeartbeat(Predicate<Map<Integer, String>> statusCheck) throws Exception {
     logger.info("Testing cluster environment...");
     TShowClusterResp showClusterResp;
     Exception lastException = null;
@@ -268,12 +288,7 @@ public abstract class AbstractEnv implements BaseEnv {
         // Check the status of nodes
         if (flag) {
           Map<Integer, String> nodeStatus = showClusterResp.getNodeStatus();
-          for (String status : nodeStatus.values()) {
-            if (NodeStatus.Unknown.getStatus().equals(status)) {
-              flag = false;
-              break;
-            }
-          }
+          flag = statusCheck.test(nodeStatus);
         }
 
         if (flag) {
@@ -289,6 +304,7 @@ public abstract class AbstractEnv implements BaseEnv {
     if (lastException != null) {
       throw lastException;
     }
+    throw new Exception("Check not pass");
   }
 
   @Override
@@ -659,7 +675,7 @@ public abstract class AbstractEnv implements BaseEnv {
 
     if (isNeedVerify) {
       // Test whether register success
-      testWorking();
+      testWorkingNoUnknown();
     }
   }
 
@@ -684,7 +700,7 @@ public abstract class AbstractEnv implements BaseEnv {
 
     if (isNeedVerify) {
       // Test whether register success
-      testWorking();
+      testWorkingNoUnknown();
     }
   }
 
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 5a92658f901..ae92a2cd81e 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
@@ -21,14 +21,12 @@ package org.apache.iotdb.confignode.it.cluster;
 
 import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
-import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.client.exception.ClientManagerException;
 import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient;
 import org.apache.iotdb.commons.cluster.NodeStatus;
 import org.apache.iotdb.confignode.it.utils.ConfigNodeTestUtils;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterReq;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterResp;
-import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRestartReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterResp;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRestartReq;
@@ -194,15 +192,6 @@ public class IoTDBClusterNodeErrorStartUpIT {
         (SyncConfigNodeIServiceClient) EnvFactory.getEnv().getLeaderConfigNodeConnection()) {
 
       /* Restart with error cluster name */
-
-      TConfigNodeRestartReq configNodeRestartReq =
-          ConfigNodeTestUtils.generateTConfigNodeRestartReq(
-              ERROR_CLUSTER_NAME, 1, registeredConfigNodeWrapper);
-      TSStatus configNodeRestartStatus = client.restartConfigNode(configNodeRestartReq);
-      Assert.assertEquals(
-          TSStatusCode.REJECT_NODE_START.getStatusCode(), configNodeRestartStatus.getCode());
-      Assert.assertTrue(configNodeRestartStatus.getMessage().contains("cluster are inconsistent"));
-
       TDataNodeRestartReq dataNodeRestartReq =
           ConfigNodeTestUtils.generateTDataNodeRestartReq(
               ERROR_CLUSTER_NAME, 2, registeredDataNodeWrapper);
@@ -214,15 +203,6 @@ public class IoTDBClusterNodeErrorStartUpIT {
           dataNodeRestartResp.getStatus().getMessage().contains("cluster are inconsistent"));
 
       /* Restart with error NodeId */
-
-      configNodeRestartReq =
-          ConfigNodeTestUtils.generateTConfigNodeRestartReq(
-              TEST_CLUSTER_NAME, 100, registeredConfigNodeWrapper);
-      configNodeRestartStatus = client.restartConfigNode(configNodeRestartReq);
-      Assert.assertEquals(
-          TSStatusCode.REJECT_NODE_START.getStatusCode(), configNodeRestartStatus.getCode());
-      Assert.assertTrue(configNodeRestartStatus.getMessage().contains("whose nodeId="));
-
       dataNodeRestartReq =
           ConfigNodeTestUtils.generateTDataNodeRestartReq(
               TEST_CLUSTER_NAME, 200, registeredDataNodeWrapper);
@@ -256,13 +236,6 @@ public class IoTDBClusterNodeErrorStartUpIT {
       Assert.assertNotEquals(-1, registeredConfigNodeId);
       int originPort = registeredConfigNodeWrapper.getConsensusPort();
       registeredConfigNodeWrapper.setConsensusPort(-12345);
-      configNodeRestartReq =
-          ConfigNodeTestUtils.generateTConfigNodeRestartReq(
-              TEST_CLUSTER_NAME, registeredConfigNodeId, registeredConfigNodeWrapper);
-      configNodeRestartStatus = client.restartConfigNode(configNodeRestartReq);
-      Assert.assertEquals(
-          TSStatusCode.REJECT_NODE_START.getStatusCode(), configNodeRestartStatus.getCode());
-      Assert.assertTrue(configNodeRestartStatus.getMessage().contains("the internal TEndPoints"));
       registeredConfigNodeWrapper.setConsensusPort(originPort);
 
       int registeredDataNodeId = -1;
diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/cluster/IoTDBClusterRestartIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/cluster/IoTDBClusterRestartIT.java
index ff75c5be945..803898f5378 100644
--- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/cluster/IoTDBClusterRestartIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/cluster/IoTDBClusterRestartIT.java
@@ -32,6 +32,7 @@ import org.apache.iotdb.it.env.cluster.EnvUtils;
 import org.apache.iotdb.it.env.cluster.MppBaseConfig;
 import org.apache.iotdb.it.env.cluster.MppCommonConfig;
 import org.apache.iotdb.it.env.cluster.MppJVMConfig;
+import org.apache.iotdb.it.framework.IoTDBTestLogger;
 import org.apache.iotdb.it.framework.IoTDBTestRunner;
 import org.apache.iotdb.itbase.category.ClusterIT;
 
@@ -41,6 +42,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
+import org.slf4j.Logger;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -50,15 +52,17 @@ import java.util.concurrent.TimeUnit;
 @RunWith(IoTDBTestRunner.class)
 @Category({ClusterIT.class})
 public class IoTDBClusterRestartIT {
+  private static final Logger logger = IoTDBTestLogger.logger;
 
   private static final String ratisConsensusProtocolClass =
       "org.apache.iotdb.consensus.ratis.RatisConsensus";
-  private static final int testConfigNodeNum = 2;
-  private static final int testDataNodeNum = 2;
+
   private static final int testReplicationFactor = 2;
 
+  private static final int testConfigNodeNum = 3, testDataNodeNum = 2;
+
   @Before
-  public void setUp() throws Exception {
+  public void setUp() {
     EnvFactory.getEnv()
         .getConfig()
         .getCommonConfig()
@@ -99,7 +103,7 @@ public class IoTDBClusterRestartIT {
       EnvFactory.getEnv().startDataNode(i);
     }
 
-    ((AbstractEnv) EnvFactory.getEnv()).testWorking();
+    ((AbstractEnv) EnvFactory.getEnv()).testWorkingNoUnknown();
   }
 
   @Test
@@ -152,4 +156,20 @@ public class IoTDBClusterRestartIT {
   }
 
   // TODO: Add persistence tests in the future
+
+  @Test
+  public void clusterRestartWithoutSeedConfigNode() {
+    // shutdown all 3 ConfigNodes
+    for (int i = testConfigNodeNum - 1; i >= 0; i--) {
+      EnvFactory.getEnv().shutdownConfigNode(i);
+    }
+    logger.info("Shutdown all ConfigNode");
+    // restart without seed ConfigNode, the cluster should still work
+    for (int i = 1; i < testConfigNodeNum; i++) {
+      EnvFactory.getEnv().startConfigNode(i);
+    }
+    logger.info("Restarted");
+    ((AbstractEnv) EnvFactory.getEnv()).testWorkingOneUnknownOtherRunning();
+    logger.info("Working without Seed-ConfigNode");
+  }
 }
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 8f5b46288fb..cd22958ceae 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
@@ -32,7 +32,6 @@ import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.path.PathPatternTree;
 import org.apache.iotdb.confignode.rpc.thrift.TClusterParameters;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterReq;
-import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRestartReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRestartReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
@@ -336,12 +335,6 @@ public class ConfigNodeTestUtils {
     return clusterParameters;
   }
 
-  public static TConfigNodeRestartReq generateTConfigNodeRestartReq(
-      String clusterName, int nodeId, ConfigNodeWrapper configNodeWrapper) {
-    return new TConfigNodeRestartReq(
-        clusterName, generateTConfigNodeLocation(nodeId, configNodeWrapper));
-  }
-
   public static TDataNodeLocation generateTDataNodeLocation(
       int nodeId, DataNodeWrapper dataNodeWrapper) {
     TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncConfigNodeClientPool.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncConfigNodeClientPool.java
index b2ec4e8d8b2..04d2c091769 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncConfigNodeClientPool.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncConfigNodeClientPool.java
@@ -29,7 +29,6 @@ import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient;
 import org.apache.iotdb.confignode.client.ConfigNodeRequestType;
 import org.apache.iotdb.confignode.rpc.thrift.TAddConsensusGroupReq;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterReq;
-import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRestartReq;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 
@@ -80,8 +79,6 @@ public class SyncConfigNodeClientPool {
           case NOTIFY_REGISTER_SUCCESS:
             client.notifyRegisterSuccess();
             return null;
-          case RESTART_CONFIG_NODE:
-            return client.restartConfigNode((TConfigNodeRestartReq) req);
           case REMOVE_CONFIG_NODE:
             return removeConfigNode((TConfigNodeLocation) req, client);
           case DELETE_CONFIG_NODE_PEER:
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java
index e577ad24da0..45d7ba0d672 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java
@@ -401,7 +401,9 @@ public class SystemPropertiesUtils {
   private static synchronized void storeSystemProperties(Properties systemProperties)
       throws IOException {
     try (FileOutputStream fileOutputStream = new FileOutputStream(systemPropertiesFile)) {
-      systemProperties.store(fileOutputStream, "");
+      systemProperties.store(
+          fileOutputStream,
+          " THIS FILE IS AUTOMATICALLY GENERATED. PLEASE DO NOT MODIFY THIS FILE !!!");
     }
   }
 }
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
index e6c5500d007..ea151cc6b12 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
@@ -105,7 +105,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TAlterSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TClusterParameters;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterReq;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterResp;
-import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRestartReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCountTimeSlotListReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCountTimeSlotListResp;
 import org.apache.iotdb.confignode.rpc.thrift.TCreateCQReq;
@@ -1011,28 +1010,6 @@ public class ConfigManager implements IManager {
     return new TConfigNodeRegisterResp().setStatus(status).setConfigNodeId(ERROR_STATUS_NODE_ID);
   }
 
-  @Override
-  public TSStatus restartConfigNode(TConfigNodeRestartReq req) {
-    TSStatus status = confirmLeader();
-    // Notice: The Seed-ConfigNode must also have the privilege to do Node restart check.
-    // Otherwise, the IoTDB-cluster will not have the ability to restart from scratch.
-    if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()
-        || ConfigNodeDescriptor.getInstance().isSeedConfigNode()
-        || SystemPropertiesUtils.isSeedConfigNode()) {
-      status =
-          ClusterNodeStartUtils.confirmNodeRestart(
-              NodeType.ConfigNode,
-              req.getClusterName(),
-              req.getConfigNodeLocation().getConfigNodeId(),
-              req.getConfigNodeLocation(),
-              this);
-      if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
-        return ClusterNodeStartUtils.ACCEPT_NODE_RESTART;
-      }
-    }
-    return status;
-  }
-
   public TSStatus checkConfigNodeGlobalConfig(TConfigNodeRegisterReq req) {
     final String errorPrefix = "Reject register, please ensure that the parameter ";
     final String errorSuffix = " is consistent with the Seed-ConfigNode.";
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
index f30957975f0..77c47b4cf7c 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
@@ -51,7 +51,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TAlterLogicalViewReq;
 import org.apache.iotdb.confignode.rpc.thrift.TAlterSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterReq;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterResp;
-import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRestartReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCountTimeSlotListReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCountTimeSlotListResp;
 import org.apache.iotdb.confignode.rpc.thrift.TCreateCQReq;
@@ -387,8 +386,6 @@ public interface IManager {
    */
   TConfigNodeRegisterResp registerConfigNode(TConfigNodeRegisterReq req);
 
-  TSStatus restartConfigNode(TConfigNodeRestartReq req);
-
   /**
    * Create peer in new node to build consensus group.
    *
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java
index 4527851c66c..56a20defbab 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java
@@ -42,7 +42,6 @@ import org.apache.iotdb.confignode.conf.SystemPropertiesUtils;
 import org.apache.iotdb.confignode.manager.ConfigManager;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterReq;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterResp;
-import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRestartReq;
 import org.apache.iotdb.confignode.rpc.thrift.TNodeVersionInfo;
 import org.apache.iotdb.confignode.service.thrift.ConfigNodeRPCService;
 import org.apache.iotdb.confignode.service.thrift.ConfigNodeRPCServiceProcessor;
@@ -124,17 +123,17 @@ public class ConfigNode implements ConfigNodeMBean {
       if (SystemPropertiesUtils.isRestarted()) {
         LOGGER.info("{} is in restarting process...", ConfigNodeConstant.GLOBAL_NAME);
 
-        int configNodeId;
-        if (!SystemPropertiesUtils.isSeedConfigNode()) {
-          // The non-seed-ConfigNodes should send restart request and be checked (ip and port) by
-          // leader before initConsensusManager
-          sendRestartConfigNodeRequest();
-          configNodeId = CONF.getConfigNodeId();
-        } else {
-          configNodeId = SEED_CONFIG_NODE_ID;
-        }
+        int configNodeId = CONF.getConfigNodeId();
         configManager.initConsensusManager();
-
+        while (configManager.getConsensusManager().getLeader() == null) {
+          LOGGER.info("Leader has not been elected yet, wait for 1 second");
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            LOGGER.warn("Unexpected interruption during waiting for leader election.");
+          }
+        }
         setUpMetricService();
         // Notice: We always set up Seed-ConfigNode's RPC service lastly to ensure
         // that the external service is not provided until ConfigNode is fully available
@@ -360,39 +359,6 @@ public class ConfigNode implements ConfigNodeMBean {
     stop();
   }
 
-  private void sendRestartConfigNodeRequest() throws StartupException {
-
-    TConfigNodeRestartReq req =
-        new TConfigNodeRestartReq(
-            CONF.getClusterName(), generateConfigNodeLocation(CONF.getConfigNodeId()));
-
-    TEndPoint targetConfigNode = CONF.getTargetConfigNode();
-    if (targetConfigNode == null) {
-      LOGGER.error(
-          "Please set the cn_target_config_node_list parameter in iotdb-confignode.properties file.");
-      throw new StartupException("The targetConfigNode setting in conf is empty");
-    }
-
-    for (int retry = 0; retry < STARTUP_RETRY_NUM; retry++) {
-      TSStatus status =
-          (TSStatus)
-              SyncConfigNodeClientPool.getInstance()
-                  .sendSyncRequestToConfigNodeWithRetry(
-                      targetConfigNode, req, ConfigNodeRequestType.RESTART_CONFIG_NODE);
-
-      if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
-        LOGGER.info("Registration request of current ConfigNode is accepted.");
-        return;
-      } else if (status.getCode() == TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) {
-        targetConfigNode = status.getRedirectNode();
-        LOGGER.info("ConfigNode need redirect to  {}.", targetConfigNode);
-      } else {
-        throw new StartupException(status.getMessage());
-      }
-      startUpSleep("Register ConfigNode failed! ");
-    }
-  }
-
   private TConfigNodeLocation generateConfigNodeLocation(int configNodeId) {
     return new TConfigNodeLocation(
         configNodeId,
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
index 0993272f8a6..732e47566d6 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
@@ -73,7 +73,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerResp;
 import org.apache.iotdb.confignode.rpc.thrift.TCheckUserPrivilegesReq;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterReq;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterResp;
-import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRestartReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCountDatabaseResp;
 import org.apache.iotdb.confignode.rpc.thrift.TCountTimeSlotListReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCountTimeSlotListResp;
@@ -594,16 +593,6 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
     return StatusUtils.OK;
   }
 
-  @Override
-  public TSStatus restartConfigNode(TConfigNodeRestartReq req) {
-    TSStatus status = configManager.restartConfigNode(req);
-
-    // Print log to record the ConfigNode that performs the RegisterConfigNodeRequest
-    LOGGER.info("Execute RestartConfigNodeRequest {} with result {}", req, status);
-
-    return status;
-  }
-
   /** For leader to remove ConfigNode configuration in consensus layer */
   @Override
   public TSStatus removeConfigNode(TConfigNodeLocation configNodeLocation) throws TException {
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java
index e731c7dfb59..543256d421f 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java
@@ -42,7 +42,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerResp;
 import org.apache.iotdb.confignode.rpc.thrift.TCheckUserPrivilegesReq;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterReq;
 import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterResp;
-import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRestartReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCountDatabaseResp;
 import org.apache.iotdb.confignode.rpc.thrift.TCountTimeSlotListReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCountTimeSlotListResp;
@@ -564,11 +563,6 @@ public class ConfigNodeClient implements IConfigNodeRPCService.Iface, ThriftClie
     throw new TException("DataNode to ConfigNode client doesn't support registerConfigNode.");
   }
 
-  @Override
-  public TSStatus restartConfigNode(TConfigNodeRestartReq req) throws TException {
-    throw new TException("DataNode to ConfigNode client doesn't support restartConfigNode.");
-  }
-
   @Override
   public TSStatus addConsensusGroup(TAddConsensusGroupReq registerResp) throws TException {
     throw new TException("DataNode to ConfigNode client doesn't support addConsensusGroup.");
diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift
index dff1a0736ed..1389c322c2e 100644
--- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift
@@ -376,11 +376,6 @@ struct TConfigNodeRegisterResp {
   2: optional i32 configNodeId
 }
 
-struct TConfigNodeRestartReq {
-  1: required string clusterName
-  2: required common.TConfigNodeLocation configNodeLocation
-}
-
 struct TAddConsensusGroupReq {
   1: required list<common.TConfigNodeLocation> configNodeList
 }
@@ -1026,15 +1021,6 @@ 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)
-
   /**
    * Remove the specific ConfigNode from the cluster
    *