You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ji...@apache.org on 2022/01/30 05:35:03 UTC

[shardingsphere] branch master updated: Refactor show instance mode to get mode from InstanceContext (#15195)

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

jianglongtao 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 58585cb  Refactor show instance mode to get mode from InstanceContext (#15195)
58585cb is described below

commit 58585cbc365bcd5eb8b262f53d3f11a033e1e1e1
Author: Haoran Meng <me...@gmail.com>
AuthorDate: Sun Jan 30 13:34:06 2022 +0800

    Refactor show instance mode to get mode from InstanceContext (#15195)
    
    * Refactor show instance mode
    
    * Fix error md style && img links for blog
    
    * Fix code style
    
    * Fix code style
---
 .../collector/ProxyInfoCollectorTest.java          |  3 +-
 .../service/PrometheusPluginBootServiceTest.java   |  3 +-
 .../CosIdSnowflakeKeyGenerateAlgorithmTest.java    | 13 +++++---
 .../keygen/SnowflakeKeyGenerateAlgorithmTest.java  |  7 +++--
 .../infra/instance/InstanceContext.java            |  7 +++--
 .../cluster/ClusterContextManagerBuilder.java      |  9 ++++--
 .../memory/MemoryContextManagerBuilder.java        |  8 ++++-
 .../StandaloneContextManagerBuilder.java           |  2 +-
 .../show/executor/ShowInstanceModeExecutor.java    | 35 ++++++----------------
 .../common/show/ShowInstanceModeExecutorTest.java  | 19 +++---------
 .../proxy/frontend/ShardingSphereProxy.java        |  2 +-
 11 files changed, 50 insertions(+), 58 deletions(-)

diff --git a/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/collector/ProxyInfoCollectorTest.java b/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/collector/ProxyInfoCollectorTest.java
index ca2790e..2f2ba52 100644
--- a/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/collector/ProxyInfoCollectorTest.java
+++ b/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/collector/ProxyInfoCollectorTest.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.agent.metrics.prometheus.collector;
 
 import io.prometheus.client.Collector;
+import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.mode.manager.memory.workerid.generator.MemoryWorkerIdGenerator;
@@ -36,7 +37,7 @@ public final class ProxyInfoCollectorTest {
     @Test
     public void assertCollect() {
         ProxyContext.getInstance().getContextManager().init(mock(MetaDataContexts.class), mock(TransactionContexts.class), new InstanceContext(new ComputeNodeInstance(), 
-                new MemoryWorkerIdGenerator(), "Memory"));
+                new MemoryWorkerIdGenerator(), new ModeConfiguration("Memory", null, false)));
         ProxyInfoCollector proxyInfoCollector = new ProxyInfoCollector();
         List<Collector.MetricFamilySamples> metricFamilySamples = proxyInfoCollector.collect();
         assertFalse(metricFamilySamples.isEmpty());
diff --git a/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/service/PrometheusPluginBootServiceTest.java b/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/service/PrometheusPluginBootServiceTest.java
index 28d05db..b485e52 100644
--- a/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/service/PrometheusPluginBootServiceTest.java
+++ b/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/service/PrometheusPluginBootServiceTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.agent.metrics.prometheus.service;
 import io.prometheus.client.exporter.HTTPServer;
 import lombok.SneakyThrows;
 import org.apache.shardingsphere.agent.config.PluginConfiguration;
+import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.mode.manager.memory.workerid.generator.MemoryWorkerIdGenerator;
@@ -45,7 +46,7 @@ public final class PrometheusPluginBootServiceTest {
     @Test
     public void assertStart() {
         ProxyContext.getInstance().getContextManager().init(mock(MetaDataContexts.class), mock(TransactionContexts.class), new InstanceContext(new ComputeNodeInstance(), 
-                new MemoryWorkerIdGenerator(), "Memory"));
+                new MemoryWorkerIdGenerator(), new ModeConfiguration("Memory", null, false)));
         Properties props = new Properties();
         props.setProperty("JVM_INFORMATION_COLLECTOR_ENABLED", "true");
         PluginConfiguration configuration = new PluginConfiguration("localhost", 8090, "", props);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java
index 531a930..847a6cf 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java
@@ -20,6 +20,7 @@ import me.ahoo.cosid.snowflake.MillisecondSnowflakeId;
 import me.ahoo.cosid.snowflake.MillisecondSnowflakeIdStateParser;
 import me.ahoo.cosid.snowflake.SnowflakeIdState;
 import me.ahoo.cosid.snowflake.SnowflakeIdStateParser;
+import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.sharding.algorithm.keygen.fixture.WorkerIdGeneratorFixture;
@@ -46,7 +47,8 @@ public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
     public void assertGenerateKey() {
         CosIdSnowflakeKeyGenerateAlgorithm cosIdSnowflakeKeyGenerateAlgorithm = new CosIdSnowflakeKeyGenerateAlgorithm();
         Properties properties = new Properties();
-        cosIdSnowflakeKeyGenerateAlgorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(), new WorkerIdGeneratorFixture(FIXTURE_WORKER_ID), "Memory"));
+        cosIdSnowflakeKeyGenerateAlgorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(), new WorkerIdGeneratorFixture(FIXTURE_WORKER_ID), 
+                new ModeConfiguration("Memory", null, false)));
         cosIdSnowflakeKeyGenerateAlgorithm.setProps(properties);
         cosIdSnowflakeKeyGenerateAlgorithm.init();
         long firstActualKey = (Long) cosIdSnowflakeKeyGenerateAlgorithm.generateKey();
@@ -64,7 +66,8 @@ public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
         CosIdSnowflakeKeyGenerateAlgorithm cosIdSnowflakeKeyGenerateAlgorithm = new CosIdSnowflakeKeyGenerateAlgorithm();
         Properties properties = new Properties();
         properties.setProperty(CosIdSnowflakeKeyGenerateAlgorithm.AS_STRING_KEY, "true");
-        cosIdSnowflakeKeyGenerateAlgorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(), new WorkerIdGeneratorFixture(FIXTURE_WORKER_ID), "Memory"));
+        cosIdSnowflakeKeyGenerateAlgorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(), new WorkerIdGeneratorFixture(FIXTURE_WORKER_ID), 
+                new ModeConfiguration("Memory", null, false)));
         cosIdSnowflakeKeyGenerateAlgorithm.setProps(properties);
         cosIdSnowflakeKeyGenerateAlgorithm.init();
         Comparable<?> actualKey = cosIdSnowflakeKeyGenerateAlgorithm.generateKey();
@@ -90,7 +93,8 @@ public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
     public void assertGenerateKeyWhenNegative() {
         CosIdSnowflakeKeyGenerateAlgorithm cosIdSnowflakeKeyGenerateAlgorithm = new CosIdSnowflakeKeyGenerateAlgorithm();
         Properties properties = new Properties();
-        cosIdSnowflakeKeyGenerateAlgorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(), new WorkerIdGeneratorFixture(-1), "Memory"));
+        cosIdSnowflakeKeyGenerateAlgorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(), new WorkerIdGeneratorFixture(-1), 
+                new ModeConfiguration("Memory", null, false)));
         cosIdSnowflakeKeyGenerateAlgorithm.setProps(properties);
         cosIdSnowflakeKeyGenerateAlgorithm.init();
         cosIdSnowflakeKeyGenerateAlgorithm.generateKey();
@@ -100,7 +104,8 @@ public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
     public void assertGenerateKeyWhenGreaterThen1023() {
         CosIdSnowflakeKeyGenerateAlgorithm cosIdSnowflakeKeyGenerateAlgorithm = new CosIdSnowflakeKeyGenerateAlgorithm();
         Properties properties = new Properties();
-        cosIdSnowflakeKeyGenerateAlgorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(), new WorkerIdGeneratorFixture(1024), "Memory"));
+        cosIdSnowflakeKeyGenerateAlgorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(), new WorkerIdGeneratorFixture(1024), 
+                new ModeConfiguration("Memory", null, false)));
         cosIdSnowflakeKeyGenerateAlgorithm.setProps(properties);
         cosIdSnowflakeKeyGenerateAlgorithm.init();
         cosIdSnowflakeKeyGenerateAlgorithm.generateKey();
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
index b146847..9f5c64e 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.sharding.algorithm.keygen;
 
 import lombok.SneakyThrows;
+import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.sharding.algorithm.keygen.fixture.FixedTimeService;
@@ -175,7 +176,8 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
     
     @Test(expected = IllegalArgumentException.class)
     public void assertSetWorkerIdFailureWhenNegative() {
-        keyGenerateAlgorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(), new WorkerIdGeneratorFixture(-1L), "Memory"));
+        keyGenerateAlgorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(), new WorkerIdGeneratorFixture(-1L), 
+                new ModeConfiguration("Memory", null, false)));
         keyGenerateAlgorithm.init();
         keyGenerateAlgorithm.generateKey();
         clearInstanceContext();
@@ -192,7 +194,8 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
     
     @Test(expected = IllegalArgumentException.class)
     public void assertSetWorkerIdFailureWhenOutOfRange() {
-        keyGenerateAlgorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(), new WorkerIdGeneratorFixture(Long.MIN_VALUE), "Memory"));
+        keyGenerateAlgorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(), new WorkerIdGeneratorFixture(Long.MIN_VALUE), 
+                new ModeConfiguration("Memory", null, false)));
         keyGenerateAlgorithm.init();
         keyGenerateAlgorithm.generateKey();
         clearInstanceContext();
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 7c8715d..c18bff8 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
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.infra.instance;
 
 import lombok.Getter;
+import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.instance.workerid.WorkerIdGenerator;
 import org.apache.shardingsphere.infra.state.StateContext;
 import org.apache.shardingsphere.infra.state.StateType;
@@ -38,13 +39,13 @@ public final class InstanceContext {
     
     private final WorkerIdGenerator workerIdGenerator;
     
-    private final String modeType;
+    private final ModeConfiguration modeConfiguration;
     
-    public InstanceContext(final ComputeNodeInstance instance, final WorkerIdGenerator workerIdGenerator, final String modeType) {
+    public InstanceContext(final ComputeNodeInstance instance, final WorkerIdGenerator workerIdGenerator, final ModeConfiguration modeConfiguration) {
         this.instance = instance;
         switchInstanceState(instance.getStatus());
         this.workerIdGenerator = workerIdGenerator;
-        this.modeType = modeType;
+        this.modeConfiguration = modeConfiguration;
     }
     
     /**
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 4589853..7b82c27 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
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.mode.manager.cluster;
 
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
+import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.config.schema.impl.DataSourceProvidedSchemaConfiguration;
 import org.apache.shardingsphere.infra.datasource.pool.creator.DataSourcePoolCreator;
 import org.apache.shardingsphere.infra.datasource.pool.destroyer.DataSourcePoolDestroyerFactory;
@@ -64,7 +65,8 @@ public final class ClusterContextManagerBuilder implements ContextManagerBuilder
         persistConfigurations(metaDataPersistService, parameter);
         MetaDataContextsBuilder metaDataContextsBuilder = createMetaDataContextsBuilder(metaDataPersistService, parameter);
         persistMetaData(metaDataPersistService, metaDataContextsBuilder.getSchemaMap());
-        ContextManager result = createContextManager(repository, metaDataPersistService, parameter.getInstanceDefinition(), metaDataContextsBuilder.build(metaDataPersistService));
+        ContextManager result = createContextManager(repository, metaDataPersistService, parameter.getInstanceDefinition(), metaDataContextsBuilder.build(metaDataPersistService), 
+                parameter.getModeConfig());
         registerOnline(repository, metaDataPersistService, parameter.getInstanceDefinition(), result);
         return result;
     }
@@ -132,10 +134,11 @@ public final class ClusterContextManagerBuilder implements ContextManagerBuilder
     }
     
     private ContextManager createContextManager(final ClusterPersistRepository repository,
-                                                final MetaDataPersistService metaDataPersistService, final InstanceDefinition instanceDefinition, final MetaDataContexts metaDataContexts) {
+                                                final MetaDataPersistService metaDataPersistService, final InstanceDefinition instanceDefinition, final MetaDataContexts metaDataContexts, 
+                                                final ModeConfiguration modeConfiguration) {
         TransactionContexts transactionContexts = new TransactionContextsBuilder(metaDataContexts.getMetaDataMap(), metaDataContexts.getGlobalRuleMetaData().getRules()).build();
         InstanceContext instanceContext = new InstanceContext(metaDataPersistService.getComputeNodePersistService().loadComputeNodeInstance(instanceDefinition),
-                new ClusterWorkerIdGenerator(repository, metaDataPersistService, instanceDefinition), getType());
+                new ClusterWorkerIdGenerator(repository, metaDataPersistService, instanceDefinition), modeConfiguration);
         ContextManager result = new ContextManager();
         result.init(metaDataContexts, transactionContexts, instanceContext);
         return result;
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-memory-mode/shardingsphere-memory-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/memory/MemoryContextManagerBuilder.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-memory-mode/shardingsphere-memory-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/memory/MemoryContextManagerBuilder.java
index 7ba305e..e979b35 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-memory-mode/shardingsphere-memory-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/memory/MemoryContextManagerBuilder.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-memory-mode/shardingsphere-memory-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/memory/MemoryContextManagerBuilder.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.mode.manager.memory;
 
+import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.config.schema.SchemaConfiguration;
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
@@ -32,6 +33,7 @@ import org.apache.shardingsphere.transaction.context.TransactionContextsBuilder;
 
 import java.sql.SQLException;
 import java.util.Map.Entry;
+import java.util.Optional;
 
 /**
  * Memory context manager builder.
@@ -56,7 +58,7 @@ public final class MemoryContextManagerBuilder implements ContextManagerBuilder
         ComputeNodeInstance instance = new ComputeNodeInstance();
         instance.setInstanceDefinition(parameter.getInstanceDefinition());
         instance.setLabels(parameter.getLabels());
-        return new InstanceContext(instance, new MemoryWorkerIdGenerator(), getType());
+        return new InstanceContext(instance, new MemoryWorkerIdGenerator(), buildMemoryModeConfiguration(parameter.getModeConfig()));
     }
     
     private void setInstanceContext(final ContextManager contextManager) {
@@ -65,6 +67,10 @@ public final class MemoryContextManagerBuilder implements ContextManagerBuilder
             .forEach(each -> ((InstanceAwareRule) each).setInstanceContext(contextManager.getInstanceContext())));
     }
     
+    private ModeConfiguration buildMemoryModeConfiguration(final ModeConfiguration modeConfiguration) {
+        return Optional.ofNullable(modeConfiguration).orElseGet(() -> new ModeConfiguration(getType(), null, false));
+    }
+    
     @Override
     public String getType() {
         return "Memory";
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 4ba5324..35b9254 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
@@ -115,7 +115,7 @@ public final class StandaloneContextManagerBuilder implements ContextManagerBuil
         ContextManager result = new ContextManager();
         TransactionContexts transactionContexts = new TransactionContextsBuilder(metaDataContexts.getMetaDataMap(), metaDataContexts.getGlobalRuleMetaData().getRules()).build();
         InstanceContext instanceContext = new InstanceContext(
-                metaDataPersistService.getComputeNodePersistService().loadComputeNodeInstance(parameter.getInstanceDefinition()), new StandaloneWorkerIdGenerator(), getType());
+                metaDataPersistService.getComputeNodePersistService().loadComputeNodeInstance(parameter.getInstanceDefinition()), new StandaloneWorkerIdGenerator(), parameter.getModeConfig());
         result.init(metaDataContexts, transactionContexts, instanceContext);
         setInstanceContext(result);
         return result;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/show/executor/ShowInstanceModeExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/show/executor/ShowInstanceModeExecutor.java
index 26fddd0..71564b3 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/show/executor/ShowInstanceModeExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/show/executor/ShowInstanceModeExecutor.java
@@ -17,14 +17,10 @@
 
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.show.executor;
 
-import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
+import org.apache.shardingsphere.infra.config.mode.PersistRepositoryConfiguration;
+import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.properties.PropertiesConverter;
-import org.apache.shardingsphere.mode.manager.ContextManager;
-import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
-import org.apache.shardingsphere.mode.persist.PersistRepository;
-import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
-import org.apache.shardingsphere.mode.repository.standalone.StandalonePersistRepository;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.response.header.query.impl.QueryHeader;
 import org.apache.shardingsphere.sharding.merge.dal.common.MultipleLocalDataMergedResult;
@@ -60,28 +56,15 @@ public final class ShowInstanceModeExecutor extends AbstractShowExecutor {
     
     @Override
     protected MergedResult createMergedResult() {
-        ContextManager contextManager = ProxyContext.getInstance().getContextManager();
-        MetaDataPersistService persistService = contextManager.getMetaDataContexts().getMetaDataPersistService().orElse(null);
-        ComputeNodeInstance instance = contextManager.getInstanceContext().getInstance();
-        if (null == persistService || null == persistService.getRepository()) {
-            return new MultipleLocalDataMergedResult(Collections.emptyList());
-        }
-        return new MultipleLocalDataMergedResult(buildRows(instance, persistService.getRepository()));
+        return new MultipleLocalDataMergedResult(buildRows());
     }
     
-    private Collection<List<Object>> buildRows(final ComputeNodeInstance instance, final PersistRepository persistService) {
+    private Collection<List<Object>> buildRows() {
         // TODO Add display of overwrite after metadata save overwrite.
-        return Collections.singleton(Arrays.asList(instance.getInstanceDefinition().getInstanceId().getId(), getTypeName(persistService), persistService.getType(),
-                PropertiesConverter.convert(persistService.getProps())));
-    }
-    
-    private String getTypeName(final PersistRepository persistRepository) {
-        if (persistRepository instanceof ClusterPersistRepository) {
-            return "Cluster";
-        } else if (persistRepository instanceof StandalonePersistRepository) {
-            return "Standalone";
-        } else {
-            return "";
-        }
+        InstanceContext instanceContext = ProxyContext.getInstance().getContextManager().getInstanceContext();
+        PersistRepositoryConfiguration repositoryConfiguration = instanceContext.getModeConfiguration().getRepository();
+        return Collections.singleton(Arrays.asList(instanceContext.getInstance().getInstanceDefinition().getInstanceId().getId(), instanceContext.getModeConfiguration().getType(),
+                null == repositoryConfiguration ? "" : repositoryConfiguration.getType(),
+                null == repositoryConfiguration ? "" : PropertiesConverter.convert(repositoryConfiguration.getProps())));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/show/ShowInstanceModeExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/show/ShowInstanceModeExecutorTest.java
index 151e9e7..4eb4035 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/show/ShowInstanceModeExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/show/ShowInstanceModeExecutorTest.java
@@ -17,11 +17,10 @@
 
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.show;
 
+import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.mode.manager.ContextManager;
-import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
-import org.apache.shardingsphere.mode.persist.PersistRepository;
-import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseRow;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.show.executor.ShowInstanceModeExecutor;
@@ -29,7 +28,6 @@ import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.ArrayList;
-import java.util.Optional;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
@@ -43,8 +41,6 @@ public final class ShowInstanceModeExecutorTest {
     @Test
     public void assertExecutor() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        Optional<MetaDataPersistService> metaDataPersistService = Optional.of(createMetaDataPersistService());
-        when(contextManager.getMetaDataContexts().getMetaDataPersistService()).thenReturn(metaDataPersistService);
         InstanceContext instanceContext = createInstanceContext();
         when(contextManager.getInstanceContext()).thenReturn(instanceContext);
         ShowInstanceModeExecutor executor = new ShowInstanceModeExecutor();
@@ -60,18 +56,11 @@ public final class ShowInstanceModeExecutorTest {
         assertThat(data.get(3), is("key=value"));
     }
     
-    private MetaDataPersistService createMetaDataPersistService() {
-        MetaDataPersistService result = mock(MetaDataPersistService.class);
-        PersistRepository repository = mock(ClusterPersistRepository.class, RETURNS_DEEP_STUBS);
-        when(result.getRepository()).thenReturn(repository);
-        when(repository.getType()).thenReturn("ZooKeeper");
-        when(repository.getProps()).thenReturn(createProperties("key", "value"));
-        return result;
-    }
-    
     private InstanceContext createInstanceContext() {
         InstanceContext result = mock(InstanceContext.class, RETURNS_DEEP_STUBS);
         when(result.getInstance().getInstanceDefinition().getInstanceId().getId()).thenReturn("127.0.0.1@3309");
+        when(result.getModeConfiguration()).thenReturn(new ModeConfiguration("Cluster", 
+                new ClusterPersistRepositoryConfiguration("ZooKeeper", "governance_ds", "127.0.0.1:2181", createProperties("key", "value")), false));
         return result;
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/ShardingSphereProxy.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/ShardingSphereProxy.java
index 57e3cef..64a3005 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/ShardingSphereProxy.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/ShardingSphereProxy.java
@@ -74,7 +74,7 @@ public final class ShardingSphereProxy {
     }
     
     private void accept(final ChannelFuture future) throws InterruptedException {
-        log.info("ShardingSphere-Proxy {} mode started successfully", ProxyContext.getInstance().getContextManager().getInstanceContext().getModeType());
+        log.info("ShardingSphere-Proxy {} mode started successfully", ProxyContext.getInstance().getContextManager().getInstanceContext().getModeConfiguration().getType());
         future.channel().closeFuture().sync();
     }