You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2022/07/01 02:07:38 UTC

[shardingsphere] branch master updated: Add InstanceDefinitionBuilder to decouple JDBC and Proxy InstanceDefinition (#18749)

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

duanzhengqiang 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 c41a66893b8 Add InstanceDefinitionBuilder to decouple JDBC and Proxy InstanceDefinition (#18749)
c41a66893b8 is described below

commit c41a66893b80b58c4788938c3d917926956f4624
Author: Liang Zhang <zh...@apache.org>
AuthorDate: Fri Jul 1 10:07:33 2022 +0800

    Add InstanceDefinitionBuilder to decouple JDBC and Proxy InstanceDefinition (#18749)
    
    * Add ContextManager
    
    * Refactor InstanceIdGeneratorFactory
    
    * Refactor InstanceDefinition
    
    * Add JDBCInstanceDefinition and ProxyInstanceDefinition
    
    * Refactor ProxyInstanceDefinition
    
    * Add InstanceDefinitionBuilderFactory
    
    * Add InstanceDefinitionBuilderFactory
    
    * Add InstanceDefinitionBuilderFactory
    
    * remove InstanceType
    
    * Move JDBCInstanceDefinition and ProxyInstanceDefinition to jdbc and proxy modules
    
    * Move JDBCInstanceDefinition and ProxyInstanceDefinition to jdbc and proxy modules
    
    * Move test cases
    
    * Back InstanceDefinition
    
    * Move test cases
    
    * Fix test cases
    
    * Fix test cases
    
    * Fix java doc
---
 .../infra/instance/InstanceContext.java            |  5 +-
 .../instance/definition/InstanceDefinition.java    | 30 +++++----
 ...nceType.java => InstanceDefinitionBuilder.java} | 26 +++++++-
 .../InstanceDefinitionBuilderFactory.java          | 71 ++++++++++++++++++++++
 .../instanceid/DefaultInstanceIdGenerator.java     |  4 +-
 .../instance/instanceid/InstanceIdGenerator.java   |  7 +--
 .../definition/InstanceDefinitionTest.java         | 52 ----------------
 .../instance/JDBCInstanceDefinitionBuilder.java    | 25 ++++----
 .../jdbc/core/connection/ConnectionManager.java    |  3 +-
 .../core/datasource/ShardingSphereDataSource.java  |  4 +-
 ...a.instance.definition.InstanceDefinitionBuilder | 18 ++++++
 .../JDBCInstanceDefinitionBuilderTest.java         | 35 ++++++-----
 .../core/connection/ConnectionManagerTest.java     |  5 +-
 .../traffic/engine/TrafficEngine.java              |  3 +-
 .../algorithm/engine/TrafficEngineTest.java        |  9 ++-
 .../RandomTrafficLoadBalanceAlgorithmTest.java     |  4 +-
 .../RoundRobinTrafficLoadBalanceAlgorithmTest.java |  5 +-
 .../mode/manager/ContextManager.java               | 52 ++++++++--------
 .../instance/InstanceIdGeneratorFactory.java       |  2 +-
 .../mode/metadata/persist/node/ComputeNode.java    | 10 ++-
 .../metadata/persist/node/ComputeNodeTest.java     |  9 ++-
 .../cluster/ClusterContextManagerBuilder.java      |  3 +-
 .../subscriber/ProcessRegistrySubscriber.java      | 17 +++---
 .../compute/service/ComputeNodeStatusService.java  | 10 +--
 .../watcher/ComputeNodeStateChangedWatcher.java    |  4 +-
 .../ClusterContextManagerCoordinatorTest.java      |  6 +-
 .../fixture/InstanceDefinitionBuilderFixture.java  | 26 ++++----
 .../lock/DistributedLockContextTest.java           |  8 +--
 .../subscriber/ProcessRegistrySubscriberTest.java  |  3 +-
 .../service/ComputeNodeStatusServiceTest.java      | 22 +++----
 ...a.instance.definition.InstanceDefinitionBuilder | 18 ++++++
 .../StandaloneContextManagerBuilder.java           |  3 +-
 .../StandaloneContextManagerBuilderTextTest.java   |  3 +-
 .../ral/common/queryable/ShowInstanceHandler.java  |  3 +-
 .../proxy/initializer/BootstrapInitializer.java    |  5 +-
 .../instance/ProxyInstanceDefinitionBuilder.java   | 25 ++++----
 ...a.instance.definition.InstanceDefinitionBuilder | 18 ++++++
 .../ProxyInstanceDefinitionBuilderTest.java        | 50 +++++++++++++++
 38 files changed, 367 insertions(+), 236 deletions(-)

diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/InstanceContext.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/InstanceContext.java
index b66be41676a..ab099c167ce 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/InstanceContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/InstanceContext.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.infra.instance;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.infra.instance.workerid.WorkerIdGenerator;
 import org.apache.shardingsphere.infra.lock.LockContext;
 
@@ -140,10 +139,10 @@ public final class InstanceContext {
      * @param labels collection of contained label
      * @return compute node instances
      */
-    public List<InstanceDefinition> getComputeNodeInstances(final InstanceType instanceType, final Collection<String> labels) {
+    public List<InstanceDefinition> getComputeNodeInstances(final String instanceType, final Collection<String> labels) {
         List<InstanceDefinition> result = new ArrayList<>(computeNodeInstances.size());
         for (ComputeNodeInstance each : computeNodeInstances) {
-            if (each.getInstanceDefinition().getInstanceType() == instanceType && labels.stream().anyMatch(((Collection<String>) each.getLabels())::contains)) {
+            if (each.getInstanceDefinition().getInstanceType().equals(instanceType) && labels.stream().anyMatch(((Collection<String>) each.getLabels())::contains)) {
                 result.add(each.getInstanceDefinition());
             }
         }
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 db7f5fa656e..6ea2bf73c9c 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
@@ -33,42 +33,40 @@ public final class InstanceDefinition {
     
     private static final String DELIMITER = "@";
     
-    private final InstanceType instanceType;
-    
     private final String instanceId;
     
-    private String ip;
+    private final String instanceType;
     
-    private int port;
+    private final String ip;
     
-    public InstanceDefinition(final String instanceId) {
-        instanceType = InstanceType.JDBC;
-        this.instanceId = instanceId;
-    }
+    private final int port;
     
-    public InstanceDefinition(final int port, final String instanceId) {
-        instanceType = InstanceType.PROXY;
+    public InstanceDefinition(final String instanceId, final String instanceType, final int port) {
         this.instanceId = instanceId;
+        this.instanceType = instanceType;
         ip = IpUtils.getIp();
         this.port = port;
     }
     
-    public InstanceDefinition(final InstanceType instanceType, final String instanceId, final String attributes) {
-        this.instanceType = instanceType;
+    public InstanceDefinition(final String instanceId, final String instanceType, final String attributes) {
         this.instanceId = instanceId;
+        this.instanceType = instanceType;
         if (!Strings.isNullOrEmpty(attributes) && attributes.contains(DELIMITER)) {
             List<String> attributesList = Splitter.on(DELIMITER).splitToList(attributes);
             ip = attributesList.get(0);
-            port = Integer.valueOf(attributesList.get(1));
+            port = Integer.parseInt(attributesList.get(1));
+        } else {
+            ip = IpUtils.getIp();
+            port = -1;
         }
     }
     
     /**
-     * Get instance attributes.
+     * Get attributes.
      * 
-     * @return got instance attributes, format is ip@uniqueSign
+     * @return attributes
      */
     public String getAttributes() {
-        return instanceType == InstanceType.JDBC ? "" : Joiner.on(DELIMITER).join(ip, port);
+        return -1 == port ? "" : Joiner.on(DELIMITER).join(ip, port);
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/definition/InstanceType.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/definition/InstanceDefinitionBuilder.java
similarity index 55%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/definition/InstanceType.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/definition/InstanceDefinitionBuilder.java
index cdbb7c28b4f..3d4ae13a709 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/definition/InstanceType.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/definition/InstanceDefinitionBuilder.java
@@ -17,10 +17,30 @@
 
 package org.apache.shardingsphere.infra.instance.definition;
 
+import org.apache.shardingsphere.spi.annotation.SingletonSPI;
+import org.apache.shardingsphere.spi.type.typed.TypedSPI;
+
 /**
- * Instance type.
+ * Instance definition builder.
  */
-public enum InstanceType {
+@SingletonSPI
+public interface InstanceDefinitionBuilder extends TypedSPI {
+    
+    /**
+     * Build instance definition.
+     * 
+     * @param instanceId instance ID
+     * @param port port
+     * @return built instance definition
+     */
+    InstanceDefinition build(String instanceId, int port);
     
-    PROXY, JDBC
+    /**
+     * Build instance definition.
+     * 
+     * @param instanceId instance ID
+     * @param attributes attributes
+     * @return built instance definition
+     */
+    InstanceDefinition build(String instanceId, String attributes);
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/definition/InstanceDefinitionBuilderFactory.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/definition/InstanceDefinitionBuilderFactory.java
new file mode 100644
index 00000000000..5b0cc08e3ef
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/definition/InstanceDefinitionBuilderFactory.java
@@ -0,0 +1,71 @@
+/*
+ * 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.definition;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.spi.type.typed.TypedSPI;
+import org.apache.shardingsphere.spi.type.typed.TypedSPIRegistry;
+
+import java.util.Collection;
+import java.util.stream.Collectors;
+
+/**
+ * Instance definition builder factory.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class InstanceDefinitionBuilderFactory {
+    
+    static {
+        ShardingSphereServiceLoader.register(InstanceDefinitionBuilder.class);
+    }
+    
+    /**
+     * Create instance of instance definition.
+     *
+     * @param type type
+     * @param instanceId instance ID
+     * @param port port 
+     * @return created instance of instance definition
+     */
+    public static InstanceDefinition newInstance(final String type, final String instanceId, final int port) {
+        return TypedSPIRegistry.getRegisteredService(InstanceDefinitionBuilder.class, type).build(instanceId, port);
+    }
+    
+    /**
+     * Create instance of instance definition.
+     * 
+     * @param type type
+     * @param instanceId instance ID
+     * @param attributes attributes 
+     * @return created instance of instance definition
+     */
+    public static InstanceDefinition newInstance(final String type, final String instanceId, final String attributes) {
+        return TypedSPIRegistry.getRegisteredService(InstanceDefinitionBuilder.class, type).build(instanceId, attributes);
+    }
+    
+    /**
+     * Get all builder types.
+     * 
+     * @return got all builder types
+     */
+    public static Collection<String> getAllTypes() {
+        return ShardingSphereServiceLoader.getServiceInstances(InstanceDefinitionBuilder.class).stream().map(TypedSPI::getType).collect(Collectors.toList());
+    }
+}
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
index 0a29681b08d..0f41ef3503c 100644
--- 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
@@ -17,8 +17,6 @@
 
 package org.apache.shardingsphere.infra.instance.instanceid;
 
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
-
 import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
 
@@ -28,7 +26,7 @@ import java.util.concurrent.ThreadLocalRandom;
 public final class DefaultInstanceIdGenerator implements InstanceIdGenerator {
     
     @Override
-    public String generate(final InstanceType instanceType) {
+    public String generate(final String instanceType) {
         return new UUID(ThreadLocalRandom.current().nextLong(), ThreadLocalRandom.current().nextLong()).toString();
     }
     
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
index e29dcd791b4..201c317e54e 100644
--- 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
@@ -17,7 +17,6 @@
 
 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;
 
@@ -27,10 +26,10 @@ import org.apache.shardingsphere.spi.type.typed.TypedSPI;
 public interface InstanceIdGenerator extends TypedSPI, RequiredSPI {
     
     /**
-     * Generate instance id.
+     * Generate instance ID.
      * 
      * @param instanceType instance type
-     * @return instance id
+     * @return instance ID
      */
-    String generate(InstanceType instanceType);
+    String generate(String instanceType);
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/instance/definition/InstanceDefinitionTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/instance/definition/InstanceDefinitionTest.java
deleted file mode 100644
index e5557597b96..00000000000
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/instance/definition/InstanceDefinitionTest.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.definition;
-
-import org.junit.Test;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-
-public final class InstanceDefinitionTest {
-    
-    @Test
-    public void assertJdbcType() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition("foo_instance_id");
-        assertThat(instanceDefinition.getInstanceId(), is("foo_instance_id"));
-        assertThat(instanceDefinition.getInstanceType(), is(InstanceType.JDBC));
-        assertThat(instanceDefinition.getAttributes(), is(""));
-    }
-    
-    @Test
-    public void assertProxyType() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(3307, "foo_instance_id");
-        assertThat(instanceDefinition.getInstanceId(), is("foo_instance_id"));
-        assertThat(instanceDefinition.getInstanceType(), is(InstanceType.PROXY));
-        assertThat(instanceDefinition.getPort(), is(3307));
-    }
-    
-    @Test
-    public void assertAttributes() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.PROXY, "foo_instance_id", "127.0.0.1@3307");
-        assertThat(instanceDefinition.getInstanceId(), is("foo_instance_id"));
-        assertThat(instanceDefinition.getInstanceType(), is(InstanceType.PROXY));
-        assertThat(instanceDefinition.getIp(), is("127.0.0.1"));
-        assertThat(instanceDefinition.getPort(), is(3307));
-        assertThat(instanceDefinition.getAttributes(), is("127.0.0.1@3307"));
-    }
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/DefaultInstanceIdGenerator.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/instance/JDBCInstanceDefinitionBuilder.java
similarity index 58%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/DefaultInstanceIdGenerator.java
copy to shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/instance/JDBCInstanceDefinitionBuilder.java
index 0a29681b08d..474b1b532fc 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/DefaultInstanceIdGenerator.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/instance/JDBCInstanceDefinitionBuilder.java
@@ -15,25 +15,28 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.instance.instanceid;
+package org.apache.shardingsphere.driver.instance;
 
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
-
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
+import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
+import org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilder;
 
 /**
- * Default instance id generator.
+ * JDBC instance definition builder.
  */
-public final class DefaultInstanceIdGenerator implements InstanceIdGenerator {
+public final class JDBCInstanceDefinitionBuilder implements InstanceDefinitionBuilder {
+    
+    @Override
+    public InstanceDefinition build(final String instanceId, final int port) {
+        return new InstanceDefinition(instanceId, getType(), -1);
+    }
     
     @Override
-    public String generate(final InstanceType instanceType) {
-        return new UUID(ThreadLocalRandom.current().nextLong(), ThreadLocalRandom.current().nextLong()).toString();
+    public InstanceDefinition build(final String instanceId, final String attributes) {
+        return new InstanceDefinition(instanceId, getType(), attributes);
     }
     
     @Override
-    public boolean isDefault() {
-        return true;
+    public String getType() {
+        return "JDBC";
     }
 }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java
index 67acffb0307..4fdbd9c260b 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java
@@ -30,7 +30,6 @@ import org.apache.shardingsphere.infra.datasource.props.DataSourceProperties;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.ExecutorJDBCConnectionManager;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
@@ -92,7 +91,7 @@ public final class ConnectionManager implements ExecutorJDBCConnectionManager, A
         Preconditions.checkState(!dataSourcePropsMap.isEmpty(), "Can not get data source properties from meta data.");
         DataSourceProperties dataSourcePropsSample = dataSourcePropsMap.values().iterator().next();
         Collection<ShardingSphereUser> users = metaDataPersistService.get().getGlobalRuleService().loadUsers();
-        Collection<InstanceDefinition> instances = contextManager.getInstanceContext().getComputeNodeInstances(InstanceType.PROXY, trafficRule.getLabels());
+        Collection<InstanceDefinition> instances = contextManager.getInstanceContext().getComputeNodeInstances("Proxy", trafficRule.getLabels());
         return DataSourcePoolCreator.create(createDataSourcePropertiesMap(instances, users, dataSourcePropsSample, schema));
     }
     
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 76bbbca67fd..bbd9fce812f 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
@@ -26,7 +26,7 @@ import org.apache.shardingsphere.infra.config.database.impl.DataSourceProvidedDa
 import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.config.scope.GlobalRuleConfiguration;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
+import org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilderFactory;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilderFactory;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilderParameter;
@@ -75,7 +75,7 @@ public final class ShardingSphereDataSource extends AbstractDataSourceAdapter im
     
     private ContextManager createContextManager(final String databaseName, final ModeConfiguration modeConfig, final Map<String, DataSource> dataSourceMap,
                                                 final Collection<RuleConfiguration> ruleConfigs, final Properties props) throws SQLException {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceIdGeneratorFactory.getInstance(modeConfig).generate(InstanceType.JDBC));
+        InstanceDefinition instanceDefinition = InstanceDefinitionBuilderFactory.newInstance("JDBC", InstanceIdGeneratorFactory.getInstance(modeConfig).generate("JDBC"), -1);
         Collection<RuleConfiguration> globalRuleConfigs = ruleConfigs.stream().filter(each -> each instanceof GlobalRuleConfiguration).collect(Collectors.toList());
         ContextManagerBuilderParameter parameter = new ContextManagerBuilderParameter(modeConfig, Collections.singletonMap(databaseName,
                 new DataSourceProvidedDatabaseConfiguration(dataSourceMap, ruleConfigs)), globalRuleConfigs, props, Collections.emptyList(), instanceDefinition);
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilder b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilder
new file mode 100644
index 00000000000..574eaa5cc8e
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilder
@@ -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.driver.instance.JDBCInstanceDefinitionBuilder
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/loadbalance/RoundRobinTrafficLoadBalanceAlgorithmTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/instance/JDBCInstanceDefinitionBuilderTest.java
similarity index 51%
copy from shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/loadbalance/RoundRobinTrafficLoadBalanceAlgorithmTest.java
copy to shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/instance/JDBCInstanceDefinitionBuilderTest.java
index 4927d3bb53d..64abe2006a0 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/loadbalance/RoundRobinTrafficLoadBalanceAlgorithmTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/instance/JDBCInstanceDefinitionBuilderTest.java
@@ -15,28 +15,35 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.traffic.algorithm.loadbalance;
+package org.apache.shardingsphere.driver.instance;
 
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
+import org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilderFactory;
 import org.junit.Test;
 
-import java.util.Arrays;
-import java.util.List;
-
 import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 
-public final class RoundRobinTrafficLoadBalanceAlgorithmTest {
+public final class JDBCInstanceDefinitionBuilderTest {
+    
+    @Test
+    public void assertNewInstance() {
+        InstanceDefinition actual = InstanceDefinitionBuilderFactory.newInstance("JDBC", "foo_id", -1);
+        assertThat(actual.getInstanceId(), is("foo_id"));
+        assertNotNull(actual.getIp());
+        assertThat(actual.getPort(), is(-1));
+        assertThat(actual.getAttributes(), is(""));
+        assertThat(actual.getInstanceType(), is("JDBC"));
+    }
     
     @Test
-    public void assertGetInstanceId() {
-        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));
-        assertThat(roundRobinAlgorithm.getInstanceId("simple_traffic", instances), is(instance2));
-        assertThat(roundRobinAlgorithm.getInstanceId("simple_traffic", instances), is(instance1));
+    public void assertNewInstanceWithAttributes() {
+        InstanceDefinition actual = InstanceDefinitionBuilderFactory.newInstance("JDBC", "foo_id", "");
+        assertThat(actual.getInstanceId(), is("foo_id"));
+        assertNotNull(actual.getIp());
+        assertThat(actual.getPort(), is(-1));
+        assertThat(actual.getAttributes(), is(""));
+        assertThat(actual.getInstanceType(), is("JDBC"));
     }
 }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManagerTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManagerTest.java
index 143f65b632a..1e998952e83 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManagerTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManagerTest.java
@@ -23,7 +23,6 @@ import org.apache.shardingsphere.infra.datasource.pool.creator.DataSourcePoolCre
 import org.apache.shardingsphere.infra.datasource.props.DataSourceProperties;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -95,8 +94,8 @@ public final class ConnectionManagerTest {
         when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
         when(globalRuleMetaData.getSingleRule(TransactionRule.class)).thenReturn(mock(TransactionRule.class, RETURNS_DEEP_STUBS));
         when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(mock(TrafficRule.class, RETURNS_DEEP_STUBS));
-        when(result.getInstanceContext().getComputeNodeInstances(InstanceType.PROXY, Arrays.asList("OLTP", "OLAP"))).thenReturn(
-                Collections.singletonList(new InstanceDefinition(InstanceType.PROXY, "127.0.0.1@3307", "127.0.0.1@3307")));
+        when(result.getInstanceContext().getComputeNodeInstances("Proxy", Arrays.asList("OLTP", "OLAP"))).thenReturn(
+                Collections.singletonList(new InstanceDefinition("127.0.0.1@3307", "Proxy", "127.0.0.1@3307")));
         dataSourcePoolCreator = mockStatic(DataSourcePoolCreator.class);
         Map<String, DataSource> trafficDataSourceMap = mockTrafficDataSourceMap();
         when(DataSourcePoolCreator.create((Map) any())).thenReturn(trafficDataSourceMap);
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/engine/TrafficEngine.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/engine/TrafficEngine.java
index ee34fea4281..ea7c55645a8 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/engine/TrafficEngine.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/engine/TrafficEngine.java
@@ -21,7 +21,6 @@ import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.traffic.context.TrafficContext;
 import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.apache.shardingsphere.traffic.rule.TrafficStrategyRule;
@@ -53,7 +52,7 @@ public final class TrafficEngine {
         if (!strategyRule.isPresent() || isInvalidStrategyRule(strategyRule.get())) {
             return result;
         }
-        List<InstanceDefinition> instances = instanceContext.getComputeNodeInstances(InstanceType.PROXY, strategyRule.get().getLabels());
+        List<InstanceDefinition> instances = instanceContext.getComputeNodeInstances("Proxy", strategyRule.get().getLabels());
         if (!instances.isEmpty()) {
             TrafficLoadBalanceAlgorithm loadBalancer = strategyRule.get().getLoadBalancer();
             InstanceDefinition instanceDefinition = 1 == instances.size() ? instances.iterator().next() : loadBalancer.getInstanceId(strategyRule.get().getName(), instances);
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 befd048a2d2..b6f01b286de 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
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.traffic.algorithm.engine;
 import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.traffic.context.TrafficContext;
 import org.apache.shardingsphere.traffic.engine.TrafficEngine;
 import org.apache.shardingsphere.traffic.rule.TrafficRule;
@@ -92,18 +91,18 @@ 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(3307, "127.0.0.1@3307"));
+        when(loadBalancer.getInstanceId("traffic", instanceIds)).thenReturn(new InstanceDefinition("127.0.0.1@3307", "Proxy", 3307));
         when(strategyRule.getLoadBalancer()).thenReturn(loadBalancer);
         when(strategyRule.getName()).thenReturn("traffic");
-        when(instanceContext.getComputeNodeInstances(InstanceType.PROXY, Arrays.asList("OLTP", "OLAP"))).thenReturn(instanceIds);
+        when(instanceContext.getComputeNodeInstances("Proxy", Arrays.asList("OLTP", "OLAP"))).thenReturn(instanceIds);
         TrafficContext actual = trafficEngine.dispatch(logicSQL, false);
         assertThat(actual.getInstanceId(), is("127.0.0.1@3307"));
     }
     
     private List<InstanceDefinition> mockComputeNodeInstances() {
         List<InstanceDefinition> result = new ArrayList<>();
-        result.add(new InstanceDefinition(InstanceType.PROXY, "127.0.0.1@3307", "127.0.0.1@3307"));
-        result.add(new InstanceDefinition(InstanceType.PROXY, "127.0.0.1@3308", "127.0.0.1@3308"));
+        result.add(new InstanceDefinition("127.0.0.1@3307", "Proxy", "127.0.0.1@3307"));
+        result.add(new InstanceDefinition("127.0.0.1@3308", "Proxy", "127.0.0.1@3308"));
         return result;
     }
 }
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 3975c10e733..6b22f4d15dd 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
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.traffic.algorithm.loadbalance;
 
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.junit.Test;
 
 import java.util.Arrays;
@@ -30,8 +29,7 @@ public final class RandomTrafficLoadBalanceAlgorithmTest {
     
     @Test
     public void assertGetInstanceId() {
-        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"));
+        List<InstanceDefinition> instances = Arrays.asList(new InstanceDefinition("127.0.0.1@3307", "Proxy", "127.0.0.1@3307"), new InstanceDefinition("127.0.0.1@3308", "Proxy", "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 4927d3bb53d..70fec042edb 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
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.traffic.algorithm.loadbalance;
 
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.junit.Test;
 
 import java.util.Arrays;
@@ -31,8 +30,8 @@ public final class RoundRobinTrafficLoadBalanceAlgorithmTest {
     
     @Test
     public void assertGetInstanceId() {
-        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");
+        InstanceDefinition instance1 = new InstanceDefinition("127.0.0.1@3307", "Proxy", "127.0.0.1@3307");
+        InstanceDefinition instance2 = new InstanceDefinition("127.0.0.1@3308", "Proxy", "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/ContextManager.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
index ec52ad11eea..e3c98c30833 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
@@ -197,36 +197,35 @@ public final class ContextManager implements AutoCloseable {
      */
     public void dropSchema(final String databaseName, final String schemaName) {
         ShardingSphereDatabase database = metaDataContexts.getMetaData().getDatabases().get(databaseName);
-        if (null == database || null == database.getSchemas().get(schemaName)) {
+        if (null == database || !database.getSchemas().containsKey(schemaName)) {
             return;
         }
-        FederationDatabaseMetaData federationDatabaseMetaData = metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().get(databaseName);
-        federationDatabaseMetaData.removeSchemaMetadata(schemaName);
         database.getSchemas().remove(schemaName);
+        metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().get(databaseName).removeSchemaMetadata(schemaName);
     }
     
     /**
      * Add resource.
      *
      * @param databaseName database name
-     * @param dataSourcePropsMap data source properties map
+     * @param toBeAddedDataSourcePropsMap data source properties map
      * @throws SQLException SQL exception
      */
-    public void addResource(final String databaseName, final Map<String, DataSourceProperties> dataSourcePropsMap) throws SQLException {
-        refreshMetaDataContext(databaseName, dataSourcePropsMap);
-        metaDataContexts.getPersistService().ifPresent(optional -> optional.getDataSourceService().append(databaseName, dataSourcePropsMap));
+    public void addResource(final String databaseName, final Map<String, DataSourceProperties> toBeAddedDataSourcePropsMap) throws SQLException {
+        refreshMetaDataContext(databaseName, toBeAddedDataSourcePropsMap);
+        metaDataContexts.getPersistService().ifPresent(optional -> optional.getDataSourceService().append(databaseName, toBeAddedDataSourcePropsMap));
     }
     
     /**
      * Alter resource.
      *
      * @param databaseName database name
-     * @param dataSourcePropsMap data source properties map
+     * @param toBeAlteredDataSourcePropsMap data source properties map
      * @throws SQLException SQL exception
      */
-    public void alterResource(final String databaseName, final Map<String, DataSourceProperties> dataSourcePropsMap) throws SQLException {
-        refreshMetaDataContext(databaseName, dataSourcePropsMap);
-        metaDataContexts.getPersistService().ifPresent(optional -> optional.getDataSourceService().append(databaseName, dataSourcePropsMap));
+    public void alterResource(final String databaseName, final Map<String, DataSourceProperties> toBeAlteredDataSourcePropsMap) throws SQLException {
+        refreshMetaDataContext(databaseName, toBeAlteredDataSourcePropsMap);
+        metaDataContexts.getPersistService().ifPresent(optional -> optional.getDataSourceService().append(databaseName, toBeAlteredDataSourcePropsMap));
     }
     
     /**
@@ -301,8 +300,7 @@ public final class ContextManager implements AutoCloseable {
         if (ruleConfigs.isEmpty()) {
             return;
         }
-        MetaDataContexts newMetaDataContexts = rebuildMetaDataContexts(
-                new ShardingSphereRuleMetaData(GlobalRulesBuilder.buildRules(ruleConfigs, metaDataContexts.getMetaData().getDatabases())));
+        MetaDataContexts newMetaDataContexts = rebuildMetaDataContexts(new ShardingSphereRuleMetaData(GlobalRulesBuilder.buildRules(ruleConfigs, metaDataContexts.getMetaData().getDatabases())));
         metaDataContexts.getMetaData().getGlobalRuleMetaData().findRules(ResourceHeldRule.class).forEach(ResourceHeldRule::closeStaleResources);
         renewMetaDataContexts(newMetaDataContexts);
         setInstanceContext();
@@ -489,19 +487,19 @@ public final class ContextManager implements AutoCloseable {
         return new MetaDataContexts(metaDataContexts.getPersistService().orElse(null), changedMetaData, metaDataContexts.getOptimizerContext());
     }
     
-    private void refreshMetaDataContext(final String databaseName, final Map<String, DataSourceProperties> dataSourceProps) throws SQLException {
-        MetaDataContexts changedMetaDataContext = buildChangedMetaDataContextWithAddedDataSource(metaDataContexts.getMetaData().getDatabases().get(databaseName), dataSourceProps);
-        metaDataContexts.getMetaData().getDatabases().putAll(changedMetaDataContext.getMetaData().getDatabases());
-        metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().putAll(changedMetaDataContext.getOptimizerContext().getFederationMetaData().getDatabases());
-        metaDataContexts.getOptimizerContext().getParserContexts().putAll(changedMetaDataContext.getOptimizerContext().getParserContexts());
-        metaDataContexts.getOptimizerContext().getPlannerContexts().putAll(changedMetaDataContext.getOptimizerContext().getPlannerContexts());
+    private void refreshMetaDataContext(final String databaseName, final Map<String, DataSourceProperties> dataSourcePropsMap) throws SQLException {
+        MetaDataContexts changedMetaDataContexts = buildChangedMetaDataContextWithAddedDataSource(databaseName, dataSourcePropsMap);
+        metaDataContexts.getMetaData().getDatabases().putAll(changedMetaDataContexts.getMetaData().getDatabases());
+        metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().putAll(changedMetaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases());
+        metaDataContexts.getOptimizerContext().getParserContexts().putAll(changedMetaDataContexts.getOptimizerContext().getParserContexts());
+        metaDataContexts.getOptimizerContext().getPlannerContexts().putAll(changedMetaDataContexts.getOptimizerContext().getPlannerContexts());
         metaDataContexts.getMetaData().getGlobalRuleMetaData().findRules(ResourceHeldRule.class).forEach(each -> each.addResource(metaDataContexts.getMetaData().getDatabases().get(databaseName)));
     }
     
-    private void refreshMetaDataContext(final String databaseName, final MetaDataContexts changedMetaDataContext, final Map<String, DataSourceProperties> dataSourcePropsMap) {
-        metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().putAll(changedMetaDataContext.getOptimizerContext().getFederationMetaData().getDatabases());
+    private void refreshMetaDataContext(final String databaseName, final MetaDataContexts changedMetaDataContexts, final Map<String, DataSourceProperties> dataSourcePropsMap) {
+        metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().putAll(changedMetaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases());
         Map<String, ShardingSphereDatabase> databases = new HashMap<>(metaDataContexts.getMetaData().getDatabases());
-        databases.putAll(changedMetaDataContext.getMetaData().getDatabases());
+        databases.putAll(changedMetaDataContexts.getMetaData().getDatabases());
         final Collection<DataSource> pendingClosedDataSources = getPendingClosedDataSources(databaseName, dataSourcePropsMap);
         MetaDataContexts newMetaDataContexts = rebuildMetaDataContexts(
                 new ShardingSphereMetaData(databases, metaDataContexts.getMetaData().getGlobalRuleMetaData(), metaDataContexts.getMetaData().getProps()));
@@ -511,13 +509,13 @@ public final class ContextManager implements AutoCloseable {
         pendingClosedDataSources.forEach(metaDataContexts.getMetaData().getDatabases().get(databaseName).getResource()::close);
     }
     
-    private MetaDataContexts buildChangedMetaDataContextWithAddedDataSource(final ShardingSphereDatabase originalDatabase,
-                                                                            final Map<String, DataSourceProperties> addedDataSourceProps) throws SQLException {
-        Map<String, DataSource> dataSourceMap = new HashMap<>(originalDatabase.getResource().getDataSources());
+    private MetaDataContexts buildChangedMetaDataContextWithAddedDataSource(final String databaseName, final Map<String, DataSourceProperties> addedDataSourceProps) throws SQLException {
+        ShardingSphereDatabase database = metaDataContexts.getMetaData().getDatabases().get(databaseName);
+        Map<String, DataSource> dataSourceMap = new HashMap<>(database.getResource().getDataSources());
         dataSourceMap.putAll(DataSourcePoolCreator.create(addedDataSourceProps));
         ConfigurationProperties props = metaDataContexts.getMetaData().getProps();
-        DatabaseConfiguration databaseConfig = new DataSourceProvidedDatabaseConfiguration(dataSourceMap, originalDatabase.getRuleMetaData().getConfigurations());
-        Map<String, ShardingSphereDatabase> databases = ShardingSphereDatabasesFactory.create(Collections.singletonMap(originalDatabase.getName(), databaseConfig), props);
+        DatabaseConfiguration databaseConfig = new DataSourceProvidedDatabaseConfiguration(dataSourceMap, database.getRuleMetaData().getConfigurations());
+        Map<String, ShardingSphereDatabase> databases = ShardingSphereDatabasesFactory.create(Collections.singletonMap(database.getName(), databaseConfig), props);
         ShardingSphereRuleMetaData globalMetaData = new ShardingSphereRuleMetaData(
                 GlobalRulesBuilder.buildRules(metaDataContexts.getMetaData().getGlobalRuleMetaData().getConfigurations(), databases));
         ShardingSphereMetaData metaData = new ShardingSphereMetaData(databases, globalMetaData, props);
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
index c90e26d161f..0cd6fd0bb8f 100644
--- 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
@@ -48,6 +48,6 @@ public final class InstanceIdGeneratorFactory {
             return RequiredSPIRegistry.getRegisteredService(InstanceIdGenerator.class);
         }
         Optional<InstanceIdGenerator> instanceIdGenerator = TypedSPIRegistry.findRegisteredService(InstanceIdGenerator.class, modeConfig.getType());
-        return instanceIdGenerator.isPresent() ? instanceIdGenerator.get() : RequiredSPIRegistry.getRegisteredService(InstanceIdGenerator.class);
+        return instanceIdGenerator.orElseGet(() -> 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 65a2c6ba34d..f17063e9bcb 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,8 +17,6 @@
 
 package org.apache.shardingsphere.mode.metadata.persist.node;
 
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
-
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -47,8 +45,8 @@ public final class ComputeNode {
      * @param instanceType instance type
      * @return path of online compute node
      */
-    public static String getOnlineNodePath(final InstanceType instanceType) {
-        return String.join("/", "", ROOT_NODE, COMPUTE_NODE, ONLINE_NODE, instanceType.name().toLowerCase());
+    public static String getOnlineNodePath(final String instanceType) {
+        return String.join("/", "", ROOT_NODE, COMPUTE_NODE, ONLINE_NODE, instanceType.toLowerCase());
     }
     
     /**
@@ -58,8 +56,8 @@ public final class ComputeNode {
      * @param instanceType instance type
      * @return path of online compute node instance
      */
-    public static String getOnlineInstanceNodePath(final String instanceId, final InstanceType instanceType) {
-        return String.join("/", "", ROOT_NODE, COMPUTE_NODE, ONLINE_NODE, instanceType.name().toLowerCase(), instanceId);
+    public static String getOnlineInstanceNodePath(final String instanceId, final String instanceType) {
+        return String.join("/", "", ROOT_NODE, COMPUTE_NODE, ONLINE_NODE, instanceType.toLowerCase(), instanceId);
     }
     
     /**
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 543833d4272..55835e67444 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,7 +17,6 @@
 
 package org.apache.shardingsphere.mode.metadata.persist.node;
 
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.junit.Test;
 
 import static org.hamcrest.CoreMatchers.is;
@@ -27,14 +26,14 @@ public final class ComputeNodeTest {
     
     @Test
     public void assertGetOnlineNodePath() {
-        assertThat(ComputeNode.getOnlineNodePath(InstanceType.PROXY), is("/nodes/compute_nodes/online/proxy"));
-        assertThat(ComputeNode.getOnlineNodePath(InstanceType.JDBC), is("/nodes/compute_nodes/online/jdbc"));
+        assertThat(ComputeNode.getOnlineNodePath("Proxy"), is("/nodes/compute_nodes/online/proxy"));
+        assertThat(ComputeNode.getOnlineNodePath("JDBC"), is("/nodes/compute_nodes/online/jdbc"));
     }
     
     @Test
     public void assertGetOnlineInstanceNodePath() {
-        assertThat(ComputeNode.getOnlineInstanceNodePath("foo_instance_1", InstanceType.PROXY), is("/nodes/compute_nodes/online/proxy/foo_instance_1"));
-        assertThat(ComputeNode.getOnlineInstanceNodePath("foo_instance_2", InstanceType.JDBC), is("/nodes/compute_nodes/online/jdbc/foo_instance_2"));
+        assertThat(ComputeNode.getOnlineInstanceNodePath("foo_instance_1", "Proxy"), is("/nodes/compute_nodes/online/proxy/foo_instance_1"));
+        assertThat(ComputeNode.getOnlineInstanceNodePath("foo_instance_2", "JDBC"), is("/nodes/compute_nodes/online/jdbc/foo_instance_2"));
     }
     
     @Test
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
index 2cadba9b810..939876e06b9 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
@@ -26,7 +26,6 @@ import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerCon
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabasesFactory;
@@ -79,7 +78,7 @@ public final class ClusterContextManagerBuilder implements ContextManagerBuilder
     }
     
     private MetaDataContexts createMetaDataContexts(final MetaDataPersistService persistService, final ContextManagerBuilderParameter parameter) throws SQLException {
-        Collection<String> databaseNames = InstanceType.JDBC == parameter.getInstanceDefinition().getInstanceType()
+        Collection<String> databaseNames = -1 == parameter.getInstanceDefinition().getPort()
                 ? parameter.getDatabaseConfigs().keySet()
                 : persistService.getSchemaMetaDataService().loadAllDatabaseNames();
         Collection<RuleConfiguration> globalRuleConfigs = persistService.getGlobalRuleService().load();
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/process/subscriber/ProcessRegistrySubscriber.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/process/subscriber/ProcessRegistrySubscriber.java
index 2ad0675077b..99486ba603a 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/process/subscriber/ProcessRegistrySubscriber.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/process/subscriber/ProcessRegistrySubscriber.java
@@ -19,7 +19,13 @@ package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.proc
 
 import com.google.common.eventbus.AllowConcurrentEvents;
 import com.google.common.eventbus.Subscribe;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
+import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
+import org.apache.shardingsphere.infra.executor.sql.process.model.ExecuteProcessConstants;
+import org.apache.shardingsphere.infra.executor.sql.process.model.ExecuteProcessContext;
+import org.apache.shardingsphere.infra.executor.sql.process.model.ExecuteProcessUnit;
+import org.apache.shardingsphere.infra.executor.sql.process.model.yaml.YamlExecuteProcessContext;
+import org.apache.shardingsphere.infra.executor.sql.process.model.yaml.YamlExecuteProcessUnit;
+import org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilderFactory;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.process.ShowProcessListManager;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.process.event.ExecuteProcessReportEvent;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.process.event.ExecuteProcessSummaryReportEvent;
@@ -30,14 +36,7 @@ import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.proce
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.process.node.ProcessNode;
 import org.apache.shardingsphere.mode.metadata.persist.node.ComputeNode;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
-import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
-import org.apache.shardingsphere.infra.executor.sql.process.model.ExecuteProcessConstants;
-import org.apache.shardingsphere.infra.executor.sql.process.model.ExecuteProcessContext;
-import org.apache.shardingsphere.infra.executor.sql.process.model.ExecuteProcessUnit;
-import org.apache.shardingsphere.infra.executor.sql.process.model.yaml.YamlExecuteProcessContext;
-import org.apache.shardingsphere.infra.executor.sql.process.model.yaml.YamlExecuteProcessUnit;
 
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.LinkedList;
 import java.util.List;
@@ -80,7 +79,7 @@ public final class ProcessRegistrySubscriber {
     }
     
     private Collection<String> getTriggerPaths(final String showProcessListId) {
-        return Arrays.stream(InstanceType.values())
+        return InstanceDefinitionBuilderFactory.getAllTypes().stream()
                 .flatMap(each -> repository.getChildrenKeys(ComputeNode.getOnlineNodePath(each)).stream()
                         .map(onlinePath -> ComputeNode.getProcessTriggerInstanceIdNodePath(onlinePath, showProcessListId)))
                 .collect(Collectors.toList());
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 1f9ceb12c8f..b6ad2464695 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
@@ -22,7 +22,7 @@ import lombok.RequiredArgsConstructor;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
+import org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilderFactory;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.mode.metadata.persist.node.ComputeNode;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
@@ -120,16 +120,16 @@ public final class ComputeNodeStatusService {
      */
     public Collection<ComputeNodeInstance> loadAllComputeNodeInstances() {
         Collection<ComputeNodeInstance> result = new LinkedList<>();
-        for (InstanceType each : InstanceType.values()) {
+        for (String each : InstanceDefinitionBuilderFactory.getAllTypes()) {
             result.addAll(loadComputeNodeInstances(each));
         }
         return result;
     }
     
-    private Collection<ComputeNodeInstance> loadComputeNodeInstances(final InstanceType type) {
+    private Collection<ComputeNodeInstance> loadComputeNodeInstances(final String type) {
         Collection<String> onlineComputeNodes = repository.getChildrenKeys(ComputeNode.getOnlineNodePath(type));
-        return onlineComputeNodes.stream()
-                .map(each -> loadComputeNodeInstance(new InstanceDefinition(type, each, repository.get(ComputeNode.getOnlineInstanceNodePath(each, type))))).collect(Collectors.toList());
+        return onlineComputeNodes.stream().map(each -> loadComputeNodeInstance(
+                InstanceDefinitionBuilderFactory.newInstance(type, each, repository.get(ComputeNode.getOnlineInstanceNodePath(each, type))))).collect(Collectors.toList());
     }
     
     /**
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 5bb5928cece..37fabc3fdb8 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,7 +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.instance.definition.InstanceDefinitionBuilderFactory;
 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;
@@ -101,7 +101,7 @@ public final class ComputeNodeStateChangedWatcher implements GovernanceWatcher<G
     private Optional<GovernanceEvent> createInstanceEvent(final DataChangedEvent event) {
         Matcher matcher = matchInstanceOnlinePath(event.getKey());
         if (matcher.find()) {
-            InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceType.valueOf(matcher.group(1).toUpperCase()), matcher.group(2), event.getValue());
+            InstanceDefinition instanceDefinition = InstanceDefinitionBuilderFactory.newInstance(matcher.group(1).toUpperCase(), matcher.group(2), event.getValue());
             if (Type.ADDED == event.getType()) {
                 return Optional.of(new InstanceOnlineEvent(instanceDefinition));
             }
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 0c6ddf8ec5a..93e7abadc56 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
@@ -144,7 +144,7 @@ public final class ClusterContextManagerCoordinatorTest {
     
     private ContextManagerBuilderParameter createContextManagerBuilderParameter() {
         ModeConfiguration modeConfig = new ModeConfiguration("Cluster", new ClusterPersistRepositoryConfiguration("FIXTURE", "", "", new Properties()), false);
-        InstanceDefinition instanceDefinition = new InstanceDefinition(3307, "foo_instance_id");
+        InstanceDefinition instanceDefinition = new InstanceDefinition("foo_instance_id", "Proxy", 3307);
         return new ContextManagerBuilderParameter(modeConfig, Collections.emptyMap(), Collections.emptyList(), new Properties(), Collections.emptyList(), instanceDefinition);
     }
     
@@ -333,12 +333,12 @@ public final class ClusterContextManagerCoordinatorTest {
     
     @Test
     public void assertRenewInstanceOnlineEvent() {
-        InstanceDefinition instanceDefinition1 = new InstanceDefinition(3307, "foo_instance_3307");
+        InstanceDefinition instanceDefinition1 = new InstanceDefinition("foo_instance_3307", "Proxy", 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(3308, "foo_instance_3308");
+        InstanceDefinition instanceDefinition2 = new InstanceDefinition("foo_instance_3308", "Proxy", 3308);
         InstanceOnlineEvent instanceOnlineEvent2 = new InstanceOnlineEvent(instanceDefinition2);
         coordinator.renew(instanceOnlineEvent2);
         assertThat(contextManager.getInstanceContext().getComputeNodeInstances().size(), is(2));
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/DefaultInstanceIdGenerator.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/fixture/InstanceDefinitionBuilderFixture.java
similarity index 58%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/DefaultInstanceIdGenerator.java
copy to shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/fixture/InstanceDefinitionBuilderFixture.java
index 0a29681b08d..07cd041bcaa 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/DefaultInstanceIdGenerator.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/fixture/InstanceDefinitionBuilderFixture.java
@@ -15,25 +15,25 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.instance.instanceid;
+package org.apache.shardingsphere.mode.manager.cluster.coordinator.fixture;
 
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
+import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
+import org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilder;
 
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
-
-/**
- * Default instance id generator.
- */
-public final class DefaultInstanceIdGenerator implements InstanceIdGenerator {
+public final class InstanceDefinitionBuilderFixture implements InstanceDefinitionBuilder {
+    
+    @Override
+    public InstanceDefinition build(final String instanceId, final int port) {
+        return new InstanceDefinition(instanceId, getType(), port);
+    }
     
     @Override
-    public String generate(final InstanceType instanceType) {
-        return new UUID(ThreadLocalRandom.current().nextLong(), ThreadLocalRandom.current().nextLong()).toString();
+    public InstanceDefinition build(final String instanceId, final String attributes) {
+        return new InstanceDefinition(instanceId, getType(), attributes);
     }
     
     @Override
-    public boolean isDefault() {
-        return true;
+    public String getType() {
+        return "FIXTURE";
     }
 }
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/lock/DistributedLockContextTest.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/lock/DistributedLockContextTest.java
index 4257371100e..597dd24cc37 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/lock/DistributedLockContextTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/lock/DistributedLockContextTest.java
@@ -38,14 +38,14 @@ public final class DistributedLockContextTest {
     @Test
     public void assertGetDistributedLock() {
         DistributedLockContext distributedLockContext = new DistributedLockContext(mock(ClusterPersistRepository.class));
-        ComputeNodeInstance currentInstance = new ComputeNodeInstance(new InstanceDefinition(3307, "1"));
+        ComputeNodeInstance currentInstance = new ComputeNodeInstance(new InstanceDefinition("1", "Proxy", 3307));
         new InstanceContext(currentInstance, mock(WorkerIdGenerator.class), mock(ModeConfiguration.class), distributedLockContext);
         assertThat(distributedLockContext.getLock(), instanceOf(ShardingSphereLock.class));
     }
     
     @Test
     public void assertTryLock() {
-        ComputeNodeInstance currentInstance = new ComputeNodeInstance(new InstanceDefinition(3307, "1"));
+        ComputeNodeInstance currentInstance = new ComputeNodeInstance(new InstanceDefinition("1", "Proxy", 3307));
         DistributedLockContext distributedLockContext = new DistributedLockContext(mock(ClusterPersistRepository.class));
         new InstanceContext(currentInstance, mock(WorkerIdGenerator.class), mock(ModeConfiguration.class), distributedLockContext);
         assertNotNull(distributedLockContext.getLock());
@@ -53,7 +53,7 @@ public final class DistributedLockContextTest {
     
     @Test
     public void assertReleaseLock() {
-        ComputeNodeInstance currentInstance = new ComputeNodeInstance(new InstanceDefinition(3307, "1"));
+        ComputeNodeInstance currentInstance = new ComputeNodeInstance(new InstanceDefinition("1", "Proxy", 3307));
         DistributedLockContext distributedLockContext = new DistributedLockContext(mock(ClusterPersistRepository.class));
         new InstanceContext(currentInstance, mock(WorkerIdGenerator.class), mock(ModeConfiguration.class), distributedLockContext);
         distributedLockContext.releaseLock(LockNameDefinitionFactory.newDatabaseDefinition("database"));
@@ -61,7 +61,7 @@ public final class DistributedLockContextTest {
     
     @Test
     public void assertIsLockedDatabase() {
-        ComputeNodeInstance currentInstance = new ComputeNodeInstance(new InstanceDefinition(3307, "1"));
+        ComputeNodeInstance currentInstance = new ComputeNodeInstance(new InstanceDefinition("1", "Proxy", 3307));
         DistributedLockContext distributedLockContext = new DistributedLockContext(mock(ClusterPersistRepository.class));
         new InstanceContext(currentInstance, mock(WorkerIdGenerator.class), mock(ModeConfiguration.class), distributedLockContext);
         assertFalse(distributedLockContext.isLocked(LockNameDefinitionFactory.newDatabaseDefinition("database")));
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/process/subscriber/ProcessRegistrySubscriberTest.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/process/subscriber/ProcessRegistrySubscriberTest.java
index 9159d4d29ff..a4c4d7b87d1 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/process/subscriber/ProcessRegistrySubscriberTest.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/process/subscriber/ProcessRegistrySubscriberTest.java
@@ -22,7 +22,6 @@ import org.apache.shardingsphere.infra.executor.sql.process.model.ExecuteProcess
 import org.apache.shardingsphere.infra.executor.sql.process.model.ExecuteProcessUnit;
 import org.apache.shardingsphere.infra.executor.sql.process.model.yaml.YamlExecuteProcessContext;
 import org.apache.shardingsphere.infra.executor.sql.process.model.yaml.YamlExecuteProcessUnit;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.process.ShowProcessListManager;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.process.event.ExecuteProcessReportEvent;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.process.event.ExecuteProcessSummaryReportEvent;
@@ -63,7 +62,7 @@ public final class ProcessRegistrySubscriberTest {
     
     @Test
     public void assertLoadShowProcessListData() {
-        when(repository.getChildrenKeys(ComputeNode.getOnlineNodePath(InstanceType.JDBC))).thenReturn(Collections.singletonList("abc"));
+        when(repository.getChildrenKeys(ComputeNode.getOnlineNodePath("FIXTURE"))).thenReturn(Collections.singletonList("abc"));
         when(repository.get(any())).thenReturn(null);
         ShowProcessListRequestEvent showProcessListRequestEvent = mock(ShowProcessListRequestEvent.class);
         processRegistrySubscriber.loadShowProcessListData(showProcessListRequestEvent);
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 bfae0728fef..90350ada8de 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
@@ -46,7 +46,7 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertRegisterOnline() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(3307, "foo_instance_id");
+        InstanceDefinition instanceDefinition = new InstanceDefinition("foo_instance_id", "Proxy", 3307);
         new ComputeNodeStatusService(repository).registerOnline(instanceDefinition);
         verify(repository).persistEphemeral(eq("/nodes/compute_nodes/online/proxy/" + instanceDefinition.getInstanceId()), anyString());
     }
@@ -54,7 +54,7 @@ public final class ComputeNodeStatusServiceTest {
     @Test
     public void assertPersistInstanceLabels() {
         ComputeNodeStatusService computeNodeStatusService = new ComputeNodeStatusService(repository);
-        InstanceDefinition instanceDefinition = new InstanceDefinition(3307, "foo_instance_id");
+        InstanceDefinition instanceDefinition = new InstanceDefinition("foo_instance_id", "Proxy", 3307);
         final String instanceId = instanceDefinition.getInstanceId();
         computeNodeStatusService.persistInstanceLabels(instanceId, Collections.singletonList("test"));
         verify(repository, times(1)).persistEphemeral(ComputeNode.getInstanceLabelsNodePath(instanceId), YamlEngine.marshal(Collections.singletonList("test")));
@@ -64,7 +64,7 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertPersistInstanceWorkerId() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(3307, "foo_instance_id");
+        InstanceDefinition instanceDefinition = new InstanceDefinition("foo_instance_id", "Proxy", 3307);
         final String instanceId = instanceDefinition.getInstanceId();
         new ComputeNodeStatusService(repository).persistInstanceWorkerId(instanceId, 100L);
         verify(repository).persistEphemeral(ComputeNode.getInstanceWorkerIdNodePath(instanceId), String.valueOf(100L));
@@ -72,7 +72,7 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertLoadInstanceLabels() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(3307, "foo_instance_id");
+        InstanceDefinition instanceDefinition = new InstanceDefinition("foo_instance_id", "Proxy", 3307);
         final String instanceId = instanceDefinition.getInstanceId();
         new ComputeNodeStatusService(repository).loadInstanceLabels(instanceId);
         verify(repository).get(ComputeNode.getInstanceLabelsNodePath(instanceId));
@@ -80,7 +80,7 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertLoadInstanceStatus() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(3307, "foo_instance_id");
+        InstanceDefinition instanceDefinition = new InstanceDefinition("foo_instance_id", "Proxy", 3307);
         final String instanceId = instanceDefinition.getInstanceId();
         new ComputeNodeStatusService(repository).loadInstanceStatus(instanceId);
         verify(repository).get(ComputeNode.getInstanceStatusNodePath(instanceId));
@@ -88,7 +88,7 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertLoadInstanceWorkerId() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(3307, "foo_instance_id");
+        InstanceDefinition instanceDefinition = new InstanceDefinition("foo_instance_id", "Proxy", 3307);
         final String instanceId = instanceDefinition.getInstanceId();
         new ComputeNodeStatusService(repository).loadInstanceWorkerId(instanceId);
         verify(repository).get(ComputeNode.getInstanceWorkerIdNodePath(instanceId));
@@ -96,17 +96,15 @@ public final class ComputeNodeStatusServiceTest {
     
     @Test
     public void assertLoadAllComputeNodeInstances() {
-        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");
+        when(repository.getChildrenKeys("/nodes/compute_nodes/online/fixture")).thenReturn(Collections.singletonList("foo_instance_3307"));
+        when(repository.get("/nodes/compute_nodes/online/fixture/foo_instance_3307")).thenReturn("127.0.0.1@3307");
         Collection<ComputeNodeInstance> actual = new ComputeNodeStatusService(repository).loadAllComputeNodeInstances();
-        assertThat(actual.size(), is(2));
+        assertThat(actual.size(), is(1));
     }
     
     @Test
     public void assertLoadComputeNodeInstance() {
-        InstanceDefinition instanceDefinition = new InstanceDefinition(3307, "foo_instance_id");
+        InstanceDefinition instanceDefinition = new InstanceDefinition("foo_instance_id", "Proxy", 3307);
         ComputeNodeInstance actual = new ComputeNodeStatusService(repository).loadComputeNodeInstance(instanceDefinition);
         assertThat(actual.getInstanceDefinition(), is(instanceDefinition));
     }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilder b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilder
new file mode 100644
index 00000000000..5ae446150b7
--- /dev/null
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilder
@@ -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.mode.manager.cluster.coordinator.fixture.InstanceDefinitionBuilderFixture
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java
index cd26f9fe329..b72ebcc2b5a 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java
@@ -24,7 +24,6 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContextFactory;
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabasesFactory;
@@ -65,7 +64,7 @@ public final class StandaloneContextManagerBuilder implements ContextManagerBuil
     }
     
     private MetaDataContexts createMetaDataContexts(final MetaDataPersistService persistService, final ContextManagerBuilderParameter parameter) throws SQLException {
-        Collection<String> databaseNames = InstanceType.JDBC == parameter.getInstanceDefinition().getInstanceType()
+        Collection<String> databaseNames = -1 == parameter.getInstanceDefinition().getPort()
                 ? parameter.getDatabaseConfigs().keySet()
                 : persistService.getSchemaMetaDataService().loadAllDatabaseNames();
         Map<String, DatabaseConfiguration> databaseConfigMap = getDatabaseConfigMap(databaseNames, persistService, parameter);
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 ba71d3d0d14..a104e9a8cc6 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
@@ -22,7 +22,6 @@ import org.apache.shardingsphere.infra.config.database.DatabaseConfiguration;
 import org.apache.shardingsphere.infra.config.database.impl.DataSourceProvidedDatabaseConfiguration;
 import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 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;
@@ -62,7 +61,7 @@ public final class StandaloneContextManagerBuilderTextTest {
         Map<String, DatabaseConfiguration> databaseConfigs = Collections.singletonMap(
                 "foo_db", new DataSourceProvidedDatabaseConfiguration(Collections.singletonMap("foo_ds", new MockedDataSource()), Collections.singleton(mock(RuleConfiguration.class))));
         Collection<RuleConfiguration> globalRuleConfigs = Collections.singleton(mock(RuleConfiguration.class));
-        InstanceDefinition instanceDefinition = new InstanceDefinition(3307, InstanceIdGeneratorFactory.getInstance(null).generate(InstanceType.PROXY));
+        InstanceDefinition instanceDefinition = new InstanceDefinition(InstanceIdGeneratorFactory.getInstance(null).generate("Proxy"), "Proxy", 3307);
         return new ContextManagerBuilderParameter(modeConfig, databaseConfigs, globalRuleConfigs, new Properties(), Collections.emptyList(), instanceDefinition);
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowInstanceHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowInstanceHandler.java
index 3053f9476ba..ba1e6eb9e06 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowInstanceHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowInstanceHandler.java
@@ -19,7 +19,6 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.queryabl
 
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.queryable.ShowInstanceStatement;
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.QueryableRALBackendHandler;
@@ -59,7 +58,7 @@ public final class ShowInstanceHandler extends QueryableRALBackendHandler<ShowIn
             return Collections.singletonList(buildRow(contextManager.getInstanceContext().getInstance(), modeType));
         }
         Collection<ComputeNodeInstance> instances = contextManager.getInstanceContext().getComputeNodeInstances().stream()
-                .filter(each -> InstanceType.PROXY.equals(each.getInstanceDefinition().getInstanceType())).collect(Collectors.toList());
+                .filter(each -> "Proxy".equals(each.getInstanceDefinition().getInstanceType())).collect(Collectors.toList());
         return instances.isEmpty() ? Collections.emptyList() : instances.stream().filter(Objects::nonNull).map(each -> buildRow(each, modeType)).collect(Collectors.toList());
     }
     
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 3d5a016e130..7f325f8b700 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
@@ -21,7 +21,7 @@ import lombok.RequiredArgsConstructor;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
+import org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilderFactory;
 import org.apache.shardingsphere.infra.yaml.config.swapper.mode.ModeConfigurationYamlSwapper;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilderFactory;
@@ -61,7 +61,8 @@ public final class BootstrapInitializer {
     
     private ContextManager createContextManager(final YamlProxyConfiguration yamlConfig, final ModeConfiguration modeConfig, final int port) throws SQLException {
         ProxyConfiguration proxyConfig = new YamlProxyConfigurationSwapper().swap(yamlConfig);
-        InstanceDefinition instanceDefinition = new InstanceDefinition(port, InstanceIdGeneratorFactory.getInstance(modeConfig).generate(InstanceType.PROXY));
+        InstanceDefinition instanceDefinition = InstanceDefinitionBuilderFactory.newInstance(
+                "Proxy", InstanceIdGeneratorFactory.getInstance(modeConfig).generate("Proxy"), port);
         ContextManagerBuilderParameter parameter = new ContextManagerBuilderParameter(modeConfig, proxyConfig.getDatabaseConfigurations(),
                 proxyConfig.getGlobalConfiguration().getRules(), proxyConfig.getGlobalConfiguration().getProperties(), proxyConfig.getGlobalConfiguration().getLabels(), instanceDefinition);
         return ContextManagerBuilderFactory.getInstance(modeConfig).build(parameter);
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/DefaultInstanceIdGenerator.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/instance/ProxyInstanceDefinitionBuilder.java
similarity index 58%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/DefaultInstanceIdGenerator.java
copy to shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/instance/ProxyInstanceDefinitionBuilder.java
index 0a29681b08d..10faefe93f5 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/instanceid/DefaultInstanceIdGenerator.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/instance/ProxyInstanceDefinitionBuilder.java
@@ -15,25 +15,28 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.instance.instanceid;
+package org.apache.shardingsphere.proxy.instance;
 
-import org.apache.shardingsphere.infra.instance.definition.InstanceType;
-
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
+import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
+import org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilder;
 
 /**
- * Default instance id generator.
+ * Proxy instance definition builder.
  */
-public final class DefaultInstanceIdGenerator implements InstanceIdGenerator {
+public final class ProxyInstanceDefinitionBuilder implements InstanceDefinitionBuilder {
+    
+    @Override
+    public InstanceDefinition build(final String instanceId, final int port) {
+        return new InstanceDefinition(instanceId, getType(), port);
+    }
     
     @Override
-    public String generate(final InstanceType instanceType) {
-        return new UUID(ThreadLocalRandom.current().nextLong(), ThreadLocalRandom.current().nextLong()).toString();
+    public InstanceDefinition build(final String instanceId, final String attributes) {
+        return new InstanceDefinition(instanceId, getType(), attributes);
     }
     
     @Override
-    public boolean isDefault() {
-        return true;
+    public String getType() {
+        return "Proxy";
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilder b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilder
new file mode 100644
index 00000000000..da2dbf527b3
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilder
@@ -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.proxy.instance.ProxyInstanceDefinitionBuilder
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/test/java/org/apache/shardingsphere/proxy/instance/ProxyInstanceDefinitionBuilderTest.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/test/java/org/apache/shardingsphere/proxy/instance/ProxyInstanceDefinitionBuilderTest.java
new file mode 100644
index 00000000000..db81c29cff9
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/test/java/org/apache/shardingsphere/proxy/instance/ProxyInstanceDefinitionBuilderTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.proxy.instance;
+
+import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
+import org.apache.shardingsphere.infra.instance.definition.InstanceDefinitionBuilderFactory;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.endsWith;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+public final class ProxyInstanceDefinitionBuilderTest {
+    
+    @Test
+    public void assertNewInstance() {
+        InstanceDefinition actual = InstanceDefinitionBuilderFactory.newInstance("Proxy", "foo_id", 3307);
+        assertThat(actual.getInstanceId(), is("foo_id"));
+        assertNotNull(actual.getIp());
+        assertThat(actual.getPort(), is(3307));
+        assertThat(actual.getAttributes(), endsWith("@3307"));
+        assertThat(actual.getInstanceType(), is("Proxy"));
+    }
+    
+    @Test
+    public void assertNewInstanceWithAttributes() {
+        InstanceDefinition actual = InstanceDefinitionBuilderFactory.newInstance("Proxy", "foo_id", "127.0.0.1@3307");
+        assertThat(actual.getInstanceId(), is("foo_id"));
+        assertThat(actual.getIp(), is("127.0.0.1"));
+        assertThat(actual.getPort(), is(3307));
+        assertThat(actual.getAttributes(), is("127.0.0.1@3307"));
+        assertThat(actual.getInstanceType(), is("Proxy"));
+    }
+}