You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2022/06/09 10:50:05 UTC

[shardingsphere] branch master updated: refactor instance id generator (#18265)

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

zhaojinchao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new a8d3ecb0e90 refactor instance id generator (#18265)
a8d3ecb0e90 is described below

commit a8d3ecb0e90082a2058f7bf855fa9b218a9624cd
Author: Haoran Meng <me...@gmail.com>
AuthorDate: Thu Jun 9 18:49:58 2022 +0800

    refactor instance id generator (#18265)
---
 .../instance/definition/InstanceDefinition.java    | 29 ++----------
 .../instanceid/DefaultInstanceIdGenerator.java     | 39 ++++++++++++++++
 .../instance/instanceid/InstanceIdGenerator.java   | 36 +++++++++++++++
 ...e.infra.instance.instanceid.InstanceIdGenerator | 18 ++++++++
 .../core/datasource/ShardingSphereDataSource.java  |  3 +-
 .../algorithm/engine/TrafficEngineTest.java        |  2 +-
 .../RandomTrafficLoadBalanceAlgorithmTest.java     |  4 +-
 .../RoundRobinTrafficLoadBalanceAlgorithmTest.java |  4 +-
 .../instance/InstanceIdGeneratorFactory.java       | 53 ++++++++++++++++++++++
 .../mode/metadata/persist/node/ComputeNode.java    | 18 --------
 .../metadata/persist/node/ComputeNodeTest.java     | 22 +--------
 .../compute/service/ComputeNodeStatusService.java  |  3 +-
 .../watcher/ComputeNodeStateChangedWatcher.java    | 28 ++++++++----
 .../ClusterContextManagerCoordinatorTest.java      |  6 +--
 .../service/ComputeNodeStatusServiceTest.java      | 24 +++++-----
 .../StandaloneContextManagerBuilderTextTest.java   |  6 ++-
 .../proxy/initializer/BootstrapInitializer.java    |  3 +-
 17 files changed, 201 insertions(+), 97 deletions(-)

diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/definition/InstanceDefinition.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/definition/InstanceDefinition.java
index fc51c980848..452c95855c6 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/definition/InstanceDefinition.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/definition/InstanceDefinition.java
@@ -19,14 +19,11 @@ package org.apache.shardingsphere.infra.instance.definition;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Splitter;
-import com.google.common.base.Strings;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.instance.utils.IpUtils;
 
 import java.lang.management.ManagementFactory;
 import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
@@ -47,16 +44,16 @@ public final class InstanceDefinition {
     
     private String uniqueSign;
     
-    public InstanceDefinition(final InstanceType instanceType) {
+    public InstanceDefinition(final InstanceType instanceType, final String instanceId) {
         this.instanceType = instanceType;
-        instanceId = new UUID(ThreadLocalRandom.current().nextLong(), ThreadLocalRandom.current().nextLong()).toString();
+        this.instanceId = instanceId;
         ip = IpUtils.getIp();
         uniqueSign = String.join("", ManagementFactory.getRuntimeMXBean().getName().split(DELIMITER)[0], String.valueOf(ATOMIC_LONG.incrementAndGet()));
     }
     
-    public InstanceDefinition(final InstanceType instanceType, final Integer port) {
+    public InstanceDefinition(final InstanceType instanceType, final Integer port, final String instanceId) {
         this.instanceType = instanceType;
-        instanceId = new UUID(ThreadLocalRandom.current().nextLong(), ThreadLocalRandom.current().nextLong()).toString();
+        this.instanceId = instanceId;
         ip = IpUtils.getIp();
         uniqueSign = String.valueOf(port);
     }
@@ -69,11 +66,6 @@ public final class InstanceDefinition {
         uniqueSign = attributesList.get(1);
     }
     
-    public InstanceDefinition(final InstanceType instanceType, final String instanceId) {
-        this.instanceType = instanceType;
-        this.instanceId = instanceId;
-    }
-    
     /**
      * Get instance attributes.
      * 
@@ -82,17 +74,4 @@ public final class InstanceDefinition {
     public String getAttributes() {
         return Joiner.on(DELIMITER).join(ip, uniqueSign);
     }
-    
-    /**
-     * Set instance attributes.
-     *
-     * @param attributes attributes ip@uniqueSign
-     */
-    public void setAttributes(final String attributes) {
-        if (!Strings.isNullOrEmpty(attributes) && attributes.contains(DELIMITER)) {
-            List<String> attributesList = Splitter.on(DELIMITER).splitToList(attributes);
-            ip = attributesList.get(0);
-            uniqueSign = attributesList.get(1);
-        }
-    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/DefaultInstanceIdGenerator.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/DefaultInstanceIdGenerator.java
new file mode 100644
index 00000000000..0a29681b08d
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/DefaultInstanceIdGenerator.java
@@ -0,0 +1,39 @@
+/*
+ * 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.shardingsphere.infra.instance.instanceid;
+
+import org.apache.shardingsphere.infra.instance.definition.InstanceType;
+
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * Default instance id generator.
+ */
+public final class DefaultInstanceIdGenerator implements InstanceIdGenerator {
+    
+    @Override
+    public String generate(final InstanceType instanceType) {
+        return new UUID(ThreadLocalRandom.current().nextLong(), ThreadLocalRandom.current().nextLong()).toString();
+    }
+    
+    @Override
+    public boolean isDefault() {
+        return true;
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/InstanceIdGenerator.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/InstanceIdGenerator.java
new file mode 100644
index 00000000000..e29dcd791b4
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/InstanceIdGenerator.java
@@ -0,0 +1,36 @@
+/*
+ * 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.shardingsphere.infra.instance.instanceid;
+
+import org.apache.shardingsphere.infra.instance.definition.InstanceType;
+import org.apache.shardingsphere.spi.type.required.RequiredSPI;
+import org.apache.shardingsphere.spi.type.typed.TypedSPI;
+
+/**
+ * Instance id generator.
+ */
+public interface InstanceIdGenerator extends TypedSPI, RequiredSPI {
+    
+    /**
+     * Generate instance id.
+     * 
+     * @param instanceType instance type
+     * @return instance id
+     */
+    String generate(InstanceType instanceType);
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.instance.instanceid.InstanceIdGenerator b/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.instance.instanceid.InstanceIdGenerator
new file mode 100644
index 00000000000..bfa44d0a2b6
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.instance.instanceid.InstanceIdGenerator
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+org.apache.shardingsphere.infra.instance.instanceid.DefaultInstanceIdGenerator
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/ShardingSphereDataSource.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/ShardingSphereDataSource.java
index e0b2e0e5fbb..8af1987411d 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/ShardingSphereDataSource.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/ShardingSphereDataSource.java
@@ -30,6 +30,7 @@ import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilderFactory;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilderParameter;
+import org.apache.shardingsphere.mode.manager.instance.InstanceIdGeneratorFactory;
 
 import javax.sql.DataSource;
 import java.sql.Connection;
@@ -79,7 +80,7 @@ public final class ShardingSphereDataSource extends AbstractDataSourceAdapter im
                 .databaseConfigs(Collections.singletonMap(databaseName, new DataSourceProvidedDatabaseConfiguration(dataSourceMap, ruleConfigs)))
                 .globalRuleConfigs(ruleConfigs.stream().filter(each -> each instanceof GlobalRuleConfiguration).collect(Collectors.toList()))
                 .props(props)
-                .instanceDefinition(new InstanceDefinition(InstanceType.JDBC)).build();
+                .instanceDefinition(new InstanceDefinition(InstanceType.JDBC, InstanceIdGeneratorFactory.getInstance(modeConfig).generate(InstanceType.JDBC))).build();
         return ContextManagerBuilderFactory.getInstance(modeConfig).build(parameter);
     }
     
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/engine/TrafficEngineTest.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/engine/TrafficEngineTest.java
index ef5cf827c81..dd3a8bd80fa 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/engine/TrafficEngineTest.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/engine/TrafficEngineTest.java
@@ -92,7 +92,7 @@ public final class TrafficEngineTest {
         when(strategyRule.getLabels()).thenReturn(Arrays.asList("OLTP", "OLAP"));
         TrafficLoadBalanceAlgorithm loadBalancer = mock(TrafficLoadBalanceAlgorithm.class);
         List<InstanceDefinition> instanceIds = mockComputeNodeInstances();
-        when(loadBalancer.getInstanceId("traffic", instanceIds)).thenReturn(new InstanceDefinition(InstanceType.PROXY, "127.0.0.1@3307"));
+        when(loadBalancer.getInstanceId("traffic", instanceIds)).thenReturn(new InstanceDefinition(InstanceType.PROXY, 3307, "127.0.0.1@3307"));
         when(strategyRule.getLoadBalancer()).thenReturn(loadBalancer);
         when(strategyRule.getName()).thenReturn("traffic");
         when(instanceContext.getComputeNodeInstances(InstanceType.PROXY, Arrays.asList("OLTP", "OLAP"))).thenReturn(instanceIds);
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/loadbalance/RandomTrafficLoadBalanceAlgorithmTest.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/loadbalance/RandomTrafficLoadBalanceAlgorithmTest.java
index 87de86c229c..3975c10e733 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/loadbalance/RandomTrafficLoadBalanceAlgorithmTest.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/loadbalance/RandomTrafficLoadBalanceAlgorithmTest.java
@@ -30,8 +30,8 @@ public final class RandomTrafficLoadBalanceAlgorithmTest {
     
     @Test
     public void assertGetInstanceId() {
-        List<InstanceDefinition> instances = Arrays.asList(new InstanceDefinition(InstanceType.PROXY, "127.0.0.1@3307"), new InstanceDefinition(InstanceType.PROXY,
-                "127.0.0.1@3308"));
+        List<InstanceDefinition> instances = Arrays.asList(new InstanceDefinition(InstanceType.PROXY, "127.0.0.1@3307", "127.0.0.1@3307"), new InstanceDefinition(InstanceType.PROXY,
+                "127.0.0.1@3308", "127.0.0.1@3308"));
         RandomTrafficLoadBalanceAlgorithm randomAlgorithm = new RandomTrafficLoadBalanceAlgorithm();
         assertTrue(instances.contains(randomAlgorithm.getInstanceId("simple_traffic", instances)));
         assertTrue(instances.contains(randomAlgorithm.getInstanceId("simple_traffic", instances)));
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/loadbalance/RoundRobinTrafficLoadBalanceAlgorithmTest.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/loadbalance/RoundRobinTrafficLoadBalanceAlgorithmTest.java
index ef85f24d26c..4927d3bb53d 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/loadbalance/RoundRobinTrafficLoadBalanceAlgorithmTest.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/loadbalance/RoundRobinTrafficLoadBalanceAlgorithmTest.java
@@ -31,8 +31,8 @@ public final class RoundRobinTrafficLoadBalanceAlgorithmTest {
     
     @Test
     public void assertGetInstanceId() {
-        InstanceDefinition instance1 = new InstanceDefinition(InstanceType.PROXY, "127.0.0.1@3307");
-        InstanceDefinition instance2 = new InstanceDefinition(InstanceType.PROXY, "127.0.0.1@3308");
+        InstanceDefinition instance1 = new InstanceDefinition(InstanceType.PROXY, "127.0.0.1@3307", "127.0.0.1@3307");
+        InstanceDefinition instance2 = new InstanceDefinition(InstanceType.PROXY, "127.0.0.1@3308", "127.0.0.1@3308");
         List<InstanceDefinition> instances = Arrays.asList(instance1, instance2);
         RoundRobinTrafficLoadBalanceAlgorithm roundRobinAlgorithm = new RoundRobinTrafficLoadBalanceAlgorithm();
         assertThat(roundRobinAlgorithm.getInstanceId("simple_traffic", instances), is(instance1));
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/instance/InstanceIdGeneratorFactory.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/instance/InstanceIdGeneratorFactory.java
new file mode 100644
index 00000000000..c90e26d161f
--- /dev/null
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/instance/InstanceIdGeneratorFactory.java
@@ -0,0 +1,53 @@
+/*
+ * 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.shardingsphere.mode.manager.instance;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
+import org.apache.shardingsphere.infra.instance.instanceid.InstanceIdGenerator;
+import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.spi.type.required.RequiredSPIRegistry;
+import org.apache.shardingsphere.spi.type.typed.TypedSPIRegistry;
+
+import java.util.Optional;
+
+/**
+ * Instance id generator factory.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class InstanceIdGeneratorFactory {
+    
+    static {
+        ShardingSphereServiceLoader.register(InstanceIdGenerator.class);
+    }
+    
+    /**
+     * Get instance of instance id generator.
+     * 
+     * @param modeConfig mode configuration
+     * @return got instance
+     */
+    public static InstanceIdGenerator getInstance(final ModeConfiguration modeConfig) {
+        if (null == modeConfig) {
+            return RequiredSPIRegistry.getRegisteredService(InstanceIdGenerator.class);
+        }
+        Optional<InstanceIdGenerator> instanceIdGenerator = TypedSPIRegistry.findRegisteredService(InstanceIdGenerator.class, modeConfig.getType());
+        return instanceIdGenerator.isPresent() ? instanceIdGenerator.get() : RequiredSPIRegistry.getRegisteredService(InstanceIdGenerator.class);
+    }
+}
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/node/ComputeNode.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/node/ComputeNode.java
index 88e33bd6194..c8292d1c603 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/node/ComputeNode.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/node/ComputeNode.java
@@ -17,10 +17,8 @@
 
 package org.apache.shardingsphere.mode.metadata.persist.node;
 
-import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
 import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 
-import java.util.Optional;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -165,20 +163,4 @@ public final class ComputeNode {
     public static String getInstanceStatusNodePath(final String instanceId) {
         return String.join("/", "", ROOT_NODE, COMPUTE_NODE, STATUS_NODE, instanceId);
     }
-    
-    /**
-     * Get instance definition by instance online path.
-     *
-     * @param onlineInstancePath online instance path
-     * @return instance id
-     */
-    public static Optional<InstanceDefinition> getInstanceDefinitionByInstanceOnlinePath(final String onlineInstancePath) {
-        Pattern pattern = Pattern.compile(getOnlineInstanceNodePath() + "/" + "(proxy|jdbc)" + "/([\\S]+)$", Pattern.CASE_INSENSITIVE);
-        Matcher matcher = pattern.matcher(onlineInstancePath);
-        return matcher.find() ? Optional.of(new InstanceDefinition(getInstanceType(matcher.group(1)), matcher.group(2))) : Optional.empty();
-    }
-    
-    private static InstanceType getInstanceType(final String instanceType) {
-        return InstanceType.PROXY.name().equalsIgnoreCase(instanceType) ? InstanceType.PROXY : InstanceType.JDBC;
-    }
 }
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/persist/node/ComputeNodeTest.java b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/persist/node/ComputeNodeTest.java
index a7ef2bea468..dce16ec403e 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/persist/node/ComputeNodeTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/persist/node/ComputeNodeTest.java
@@ -17,15 +17,11 @@
 
 package org.apache.shardingsphere.mode.metadata.persist.node;
 
-import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
 import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.junit.Test;
 
-import java.util.Optional;
-
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
 
 public final class ComputeNodeTest {
     
@@ -65,7 +61,7 @@ public final class ComputeNodeTest {
     }
     
     @Test
-    public void assertGetInstanceIdByComuteNodePath() {
+    public void assertGetInstanceIdByComputeNodePath() {
         assertThat(ComputeNode.getInstanceIdByComputeNode("/nodes/compute_nodes/status/127.0.0.1@3307"), is("127.0.0.1@3307"));
         assertThat(ComputeNode.getInstanceIdByComputeNode("/nodes/compute_nodes/worker_id/127.0.0.1@3308"), is("127.0.0.1@3308"));
         assertThat(ComputeNode.getInstanceIdByComputeNode("/nodes/compute_nodes/labels/127.0.0.1@3309"), is("127.0.0.1@3309"));
@@ -86,22 +82,6 @@ public final class ComputeNodeTest {
         assertThat(ComputeNode.getXaRecoveryIdNodePath(), is("/nodes/compute_nodes/xa_recovery_id"));
     }
     
-    @Test
-    public void assertGetInstanceDefinitionByProxyOnlinePath() {
-        Optional<InstanceDefinition> actual = ComputeNode.getInstanceDefinitionByInstanceOnlinePath("/nodes/compute_nodes/online/proxy/127.0.0.1@3307");
-        assertTrue(actual.isPresent());
-        assertThat(actual.get().getInstanceId(), is("127.0.0.1@3307"));
-        assertThat(actual.get().getInstanceType(), is(InstanceType.PROXY));
-    }
-    
-    @Test
-    public void assertGetInstanceDefinitionByJdbcOnlinePath() {
-        Optional<InstanceDefinition> actual = ComputeNode.getInstanceDefinitionByInstanceOnlinePath("/nodes/compute_nodes/online/jdbc/127.0.0.1@3307");
-        assertTrue(actual.isPresent());
-        assertThat(actual.get().getInstanceId(), is("127.0.0.1@3307"));
-        assertThat(actual.get().getInstanceType(), is(InstanceType.JDBC));
-    }
-    
     @Test
     public void assertGetComputeNodePath() {
         assertThat(ComputeNode.getComputeNodePath(), is("/nodes/compute_nodes"));
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusService.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusService.java
index 22d76c70bf7..7b4caba570f 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusService.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusService.java
@@ -160,8 +160,7 @@ public final class ComputeNodeStatusService {
         Arrays.stream(InstanceType.values()).forEach(instanceType -> {
             Collection<String> onlineComputeNodes = repository.getChildrenKeys(ComputeNode.getOnlineNodePath(instanceType));
             onlineComputeNodes.forEach(each -> {
-                InstanceDefinition instanceDefinition = new InstanceDefinition(instanceType, each);
-                instanceDefinition.setAttributes(repository.get(ComputeNode.getOnlineInstanceNodePath(each, instanceType)));
+                InstanceDefinition instanceDefinition = new InstanceDefinition(instanceType, each, repository.get(ComputeNode.getOnlineInstanceNodePath(each, instanceType)));
                 result.add(loadComputeNodeInstance(instanceDefinition));
             });
         });
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/watcher/ComputeNodeStateChangedWatcher.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/watcher/ComputeNodeStateChan [...]
index 2517db75d5b..42a9ba614e5 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/watcher/ComputeNodeStateChangedWatcher.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/watcher/ComputeNodeStateChangedWatcher.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.stat
 
 import com.google.common.base.Strings;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
+import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.GovernanceEvent;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.GovernanceWatcher;
@@ -71,8 +72,7 @@ public final class ComputeNodeStateChangedWatcher implements GovernanceWatcher<G
                 return Optional.of(new LabelsEvent(instanceId, Strings.isNullOrEmpty(event.getValue()) ? new ArrayList<>() : YamlEngine.unmarshal(event.getValue(), Collection.class)));
             }
         } else if (event.getKey().startsWith(ComputeNode.getOnlineInstanceNodePath())) {
-            Optional<InstanceDefinition> instanceDefinition = ComputeNode.getInstanceDefinitionByInstanceOnlinePath(event.getKey());
-            return instanceDefinition.isPresent() ? createInstanceEvent(instanceDefinition.get(), event.getType(), event.getValue()) : Optional.empty();
+            return createInstanceEvent(event);
         } else if (event.getKey().startsWith(ComputeNode.getProcessTriggerNodePatch())) {
             return createShowProcessListTriggerEvent(event);
         } else if (event.getKey().startsWith(ComputeNode.getXaRecoveryIdNodePath())) {
@@ -99,16 +99,28 @@ public final class ComputeNodeStateChangedWatcher implements GovernanceWatcher<G
         return pattern.matcher(event.getKey());
     }
     
-    private Optional<GovernanceEvent> createInstanceEvent(final InstanceDefinition instanceDefinition, final Type type, final String value) {
-        if (Type.ADDED == type) {
-            instanceDefinition.setAttributes(value);
-            return Optional.of(new InstanceOnlineEvent(instanceDefinition));
-        } else if (Type.DELETED == type) {
-            return Optional.of(new InstanceOfflineEvent(instanceDefinition));
+    private Optional<GovernanceEvent> createInstanceEvent(final DataChangedEvent event) {
+        Matcher matcher = matchInstanceOnlinePath(event.getKey());
+        if (matcher.find()) {
+            InstanceDefinition instanceDefinition = new InstanceDefinition(getInstanceType(matcher.group(1)), matcher.group(2), event.getValue());
+            if (Type.ADDED == event.getType()) {
+                return Optional.of(new InstanceOnlineEvent(instanceDefinition));
+            } else if (Type.DELETED == event.getType()) {
+                return Optional.of(new InstanceOfflineEvent(instanceDefinition));
+            }
         }
         return Optional.empty();
     }
     
+    private Matcher matchInstanceOnlinePath(final String onlineInstancePath) {
+        Pattern pattern = Pattern.compile(ComputeNode.getOnlineInstanceNodePath() + "/" + "(proxy|jdbc)" + "/([\\S]+)$", Pattern.CASE_INSENSITIVE);
+        return pattern.matcher(onlineInstancePath);
+    }
+    
+    private InstanceType getInstanceType(final String instanceType) {
+        return InstanceType.PROXY.name().equalsIgnoreCase(instanceType) ? InstanceType.PROXY : InstanceType.JDBC;
+    }
+    
     private Optional<GovernanceEvent> createXaRecoveryIdEvent(final DataChangedEvent event) {
         Matcher matcher = Pattern.compile(ComputeNode.getXaRecoveryIdNodePath() + "/([\\S]+)/([\\S]+)$", Pattern.CASE_INSENSITIVE).matcher(event.getKey());
         if (matcher.find()) {
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
index e312ba1cf63..4d827b7a11c 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
@@ -135,7 +135,7 @@ public final class ClusterContextManagerCoordinatorTest {
     public void setUp() throws SQLException {
         ModeConfiguration modeConfig = new ModeConfiguration("Cluster", new ClusterPersistRepositoryConfiguration("FIXTURE", "", "", new Properties()), false);
         contextManager = new ClusterContextManagerBuilder().build(ContextManagerBuilderParameter.builder().modeConfig(modeConfig).databaseConfigs(Collections.emptyMap())
-                .globalRuleConfigs(Collections.emptyList()).props(new Properties()).instanceDefinition(new InstanceDefinition(InstanceType.PROXY, 3307)).build());
+                .globalRuleConfigs(Collections.emptyList()).props(new Properties()).instanceDefinition(new InstanceDefinition(InstanceType.PROXY, 3307, "foo_instance_id")).build());
         assertTrue(contextManager.getMetaDataContexts().getPersistService().isPresent());
         contextManager.renewMetaDataContexts(new MetaDataContexts(contextManager.getMetaDataContexts().getPersistService().get(),
                 new ShardingSphereMetaData(createDatabases(), contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData(),
@@ -334,12 +334,12 @@ public final class ClusterContextManagerCoordinatorTest {
     
     @Test
     public void assertRenewInstanceOnlineEvent() {
-        InstanceDefinition instanceDefinition1 = new InstanceDefinition(InstanceType.PROXY, 3307);
+        InstanceDefinition instanceDefinition1 = new InstanceDefinition(InstanceType.PROXY, 3307, "foo_instance_3307");
         InstanceOnlineEvent instanceOnlineEvent1 = new InstanceOnlineEvent(instanceDefinition1);
         coordinator.renew(instanceOnlineEvent1);
         assertThat(contextManager.getInstanceContext().getComputeNodeInstances().size(), is(1));
         assertThat(((LinkedList<ComputeNodeInstance>) contextManager.getInstanceContext().getComputeNodeInstances()).get(0).getInstanceDefinition(), is(instanceDefinition1));
-        InstanceDefinition instanceDefinition2 = new InstanceDefinition(InstanceType.PROXY, 3308);
+        InstanceDefinition instanceDefinition2 = new InstanceDefinition(InstanceType.PROXY, 3308, "foo_instance_3308");
         InstanceOnlineEvent instanceOnlineEvent2 = new InstanceOnlineEvent(instanceDefinition2);
         coordinator.renew(instanceOnlineEvent2);
         assertThat(contextManager.getInstanceContext().getComputeNodeInstances().size(), is(2));
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusServiceTest.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusServi [...]
index b4cf2ac97f5..88909d4e59b 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusServiceTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusServiceTest.java
@@ -47,7 +47,7 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertRegisterOnline() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307);
+        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307, "foo_instance_id");
         new ComputeNodeStatusService(repository).registerOnline(instanceDefinition);
         verify(repository).persistEphemeral(eq("/nodes/compute_nodes/online/proxy/" + instanceDefinition.getInstanceId()), anyString());
     }
@@ -55,7 +55,7 @@ public final class ComputeNodeStatusServiceTest {
     @Test
     public void assertPersistInstanceLabels() {
         ComputeNodeStatusService computeNodeStatusService = new ComputeNodeStatusService(repository);
-        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307);
+        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307, "foo_instance_id");
         final String instanceId = instanceDefinition.getInstanceId();
         computeNodeStatusService.persistInstanceLabels(instanceId, Collections.singletonList("test"));
         verify(repository, times(1)).persistEphemeral(ComputeNode.getInstanceLabelsNodePath(instanceId), YamlEngine.marshal(Collections.singletonList("test")));
@@ -65,7 +65,7 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertPersistInstanceWorkerId() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307);
+        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307, "foo_instance_id");
         final String instanceId = instanceDefinition.getInstanceId();
         new ComputeNodeStatusService(repository).persistInstanceWorkerId(instanceId, 100L);
         verify(repository).persistEphemeral(ComputeNode.getInstanceWorkerIdNodePath(instanceId), String.valueOf(100L));
@@ -73,7 +73,7 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertPersistInstanceXaRecoveryId() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307);
+        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307, "foo_instance_id");
         final String instanceId = instanceDefinition.getInstanceId();
         new ComputeNodeStatusService(repository).persistInstanceXaRecoveryId(instanceId, Collections.singleton(instanceId));
         verify(repository).getChildrenKeys(ComputeNode.getXaRecoveryIdNodePath());
@@ -82,7 +82,7 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertLoadInstanceLabels() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307);
+        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307, "foo_instance_id");
         final String instanceId = instanceDefinition.getInstanceId();
         new ComputeNodeStatusService(repository).loadInstanceLabels(instanceId);
         verify(repository).get(ComputeNode.getInstanceLabelsNodePath(instanceId));
@@ -90,7 +90,7 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertLoadInstanceStatus() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307);
+        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307, "foo_instance_id");
         final String instanceId = instanceDefinition.getInstanceId();
         new ComputeNodeStatusService(repository).loadInstanceStatus(instanceId);
         verify(repository).get(ComputeNode.getInstanceStatusNodePath(instanceId));
@@ -98,7 +98,7 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertLoadInstanceWorkerId() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307);
+        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307, "foo_instance_id");
         final String instanceId = instanceDefinition.getInstanceId();
         new ComputeNodeStatusService(repository).loadInstanceWorkerId(instanceId);
         verify(repository).get(ComputeNode.getInstanceWorkerIdNodePath(instanceId));
@@ -106,7 +106,7 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertLoadInstanceXaRecoveryId() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307);
+        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307, "foo_instance_id");
         final String instanceId = instanceDefinition.getInstanceId();
         new ComputeNodeStatusService(repository).loadXaRecoveryIds(instanceId);
         verify(repository).getChildrenKeys(ComputeNode.getXaRecoveryIdNodePath());
@@ -115,15 +115,17 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertLoadAllComputeNodeInstances() {
-        when(repository.getChildrenKeys("/nodes/compute_nodes/online/proxy")).thenReturn(Collections.singletonList("127.0.0.1@3307"));
-        when(repository.getChildrenKeys("/nodes/compute_nodes/online/jdbc")).thenReturn(Collections.singletonList("127.0.0.1@3308"));
+        when(repository.getChildrenKeys("/nodes/compute_nodes/online/proxy")).thenReturn(Collections.singletonList("foo_instance_3307"));
+        when(repository.getChildrenKeys("/nodes/compute_nodes/online/jdbc")).thenReturn(Collections.singletonList("foo_instance_3308"));
+        when(repository.get("/nodes/compute_nodes/online/proxy/foo_instance_3307")).thenReturn("127.0.0.1@3307");
+        when(repository.get("/nodes/compute_nodes/online/jdbc/foo_instance_3308")).thenReturn("127.0.0.1@3308");
         Collection<ComputeNodeInstance> actual = new ComputeNodeStatusService(repository).loadAllComputeNodeInstances();
         assertThat(actual.size(), is(2));
     }
     
     @Test
     public void assertLoadComputeNodeInstance() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307);
+        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, 3307, "foo_instance_id");
         ComputeNodeInstance actual = new ComputeNodeStatusService(repository).loadComputeNodeInstance(instanceDefinition);
         assertThat(actual.getInstanceDefinition(), is(instanceDefinition));
     }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilderTextTest.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilderTextTest.java
index 4e4518dcd40..9d78bcaa7dc 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilderTextTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilderTextTest.java
@@ -24,6 +24,7 @@ import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
 import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilderParameter;
+import org.apache.shardingsphere.mode.manager.instance.InstanceIdGeneratorFactory;
 import org.apache.shardingsphere.mode.metadata.persist.node.GlobalNode;
 import org.apache.shardingsphere.mode.metadata.persist.node.DatabaseMetaDataNode;
 import org.apache.shardingsphere.mode.persist.PersistRepository;
@@ -44,11 +45,12 @@ public final class StandaloneContextManagerBuilderTextTest {
     
     @Test
     public void assertBuild() throws SQLException {
-        ContextManager actual = new StandaloneContextManagerBuilder().build(ContextManagerBuilderParameter.builder().modeConfig(new ModeConfiguration("Standalone", null, false))
+        ContextManager actual = new StandaloneContextManagerBuilder().build(ContextManagerBuilderParameter.builder()
+                .modeConfig(new ModeConfiguration("Standalone", null, false))
                 .databaseConfigs(Collections.singletonMap("foo_schema",
                         new DataSourceProvidedDatabaseConfiguration(Collections.singletonMap("foo_ds", new MockedDataSource()), Collections.singleton(mock(RuleConfiguration.class)))))
                 .globalRuleConfigs(Collections.singleton(mock(RuleConfiguration.class))).props(new Properties())
-                .instanceDefinition(new InstanceDefinition(InstanceType.PROXY, 3307)).build());
+                .instanceDefinition(new InstanceDefinition(InstanceType.PROXY, 3307, InstanceIdGeneratorFactory.getInstance(null).generate(InstanceType.PROXY))).build());
         assertNotNull(actual.getMetaDataContexts().getMetaData().getDatabases().get("foo_schema"));
         assertTrue(actual.getMetaDataContexts().getPersistService().isPresent());
         PersistRepository repository = actual.getMetaDataContexts().getPersistService().get().getRepository();
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/BootstrapInitializer.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/BootstrapInitializer.java
index 6af71d5ad62..d1988164c38 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/BootstrapInitializer.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/BootstrapInitializer.java
@@ -26,6 +26,7 @@ import org.apache.shardingsphere.infra.yaml.config.swapper.mode.ModeConfiguratio
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilderFactory;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilderParameter;
+import org.apache.shardingsphere.mode.manager.instance.InstanceIdGeneratorFactory;
 import org.apache.shardingsphere.mode.manager.listener.ContextManagerLifecycleListener;
 import org.apache.shardingsphere.mode.manager.listener.ContextManagerLifecycleListenerFactory;
 import org.apache.shardingsphere.proxy.backend.config.ProxyConfiguration;
@@ -66,7 +67,7 @@ public final class BootstrapInitializer {
                 .globalRuleConfigs(proxyConfig.getGlobalConfiguration().getRules())
                 .props(proxyConfig.getGlobalConfiguration().getProperties())
                 .labels(proxyConfig.getGlobalConfiguration().getLabels())
-                .instanceDefinition(new InstanceDefinition(InstanceType.PROXY, port)).build();
+                .instanceDefinition(new InstanceDefinition(InstanceType.PROXY, port, InstanceIdGeneratorFactory.getInstance(modeConfig).generate(InstanceType.PROXY))).build();
         return ContextManagerBuilderFactory.getInstance(modeConfig).build(parameter);
     }