You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2020/07/29 11:59:21 UTC

[shardingsphere] branch master updated: Refactor OrchestrationShardingSphereDataSource (#6515)

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

zhangyonglun 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 a99420e  Refactor OrchestrationShardingSphereDataSource (#6515)
a99420e is described below

commit a99420eb0bb18006b0060732530de7632000df6b
Author: Liang Zhang <te...@163.com>
AuthorDate: Wed Jul 29 19:59:07 2020 +0800

    Refactor OrchestrationShardingSphereDataSource (#6515)
    
    * Remove log of OrchestrationFacade
    
    * Rename shardingSchemaNames to schemaNames
    
    * Refactor OrchestrationShardingSphereDataSource
    
    * Refactor OrchestrationShardingSphereDataSource
    
    * Refactor OrchestrationShardingSphereDataSource
    
    * Fix javadoc
---
 .../panel/spi/engine/ControlPanelFacadeEngine.java |  12 +-
 .../listener/ConfigurationListenerManager.java     |   4 +-
 .../config/listener/SchemaChangedListener.java     |   6 +-
 .../core/facade/OrchestrationFacade.java           |  28 ++--
 .../listener/OrchestrationListenerManager.java     |   7 +-
 .../core/facade/OrchestrationFacadeTest.java       |   8 +-
 .../metadata/listener/MetaDataChangedListener.java |   6 +-
 .../metadata/listener/MetaDataListenerManager.java |   4 +-
 .../OrchestrationShardingSphereDataSource.java     | 164 ++++++++++-----------
 .../kernel/context/SchemaContext.java              |   3 +
 .../kernel/context/SchemaContexts.java             |   3 +
 .../kernel/context/SchemaContextsAware.java        |   1 -
 .../kernel/context/SchemaContextsBuilder.java      |   3 +
 .../kernel/context/runtime/RuntimeContext.java     |  13 +-
 .../context/schema/ShardingSphereSchema.java       |   3 +
 .../OrchestrationConfigurationConverter.java       |   4 +-
 16 files changed, 134 insertions(+), 135 deletions(-)

diff --git a/shardingsphere-control-panel/shardingsphere-control-panel-spi/src/main/java/org/apache/shardingsphere/control/panel/spi/engine/ControlPanelFacadeEngine.java b/shardingsphere-control-panel/shardingsphere-control-panel-spi/src/main/java/org/apache/shardingsphere/control/panel/spi/engine/ControlPanelFacadeEngine.java
index acca490..77af7df 100644
--- a/shardingsphere-control-panel/shardingsphere-control-panel-spi/src/main/java/org/apache/shardingsphere/control/panel/spi/engine/ControlPanelFacadeEngine.java
+++ b/shardingsphere-control-panel/shardingsphere-control-panel-spi/src/main/java/org/apache/shardingsphere/control/panel/spi/engine/ControlPanelFacadeEngine.java
@@ -17,18 +17,18 @@
 
 package org.apache.shardingsphere.control.panel.spi.engine;
 
-import java.util.Collection;
-import java.util.Map;
-import java.util.stream.Collectors;
 import org.apache.shardingsphere.control.panel.spi.ControlPanelFacade;
 import org.apache.shardingsphere.control.panel.spi.FacadeConfiguration;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.order.OrderedSPIRegistry;
 
+import java.util.Collection;
+import java.util.Map;
+import java.util.stream.Collectors;
+
 /**
  * Control panel facade engine.
  */
-@SuppressWarnings("ALL")
 public final class ControlPanelFacadeEngine {
     
     static {
@@ -36,10 +36,11 @@ public final class ControlPanelFacadeEngine {
     }
     
     /**
-     * Init control panel facade.
+     * Initialize control panel facade.
      *
      * @param facadeConfigurations facade configurations
      */
+    @SuppressWarnings("rawtypes")
     public void init(final Collection<FacadeConfiguration> facadeConfigurations) {
         Collection<Class<?>> facadeClassTypes = facadeConfigurations.stream().map(FacadeConfiguration::getClass).collect(Collectors.toList());
         for (Map.Entry<Class<?>, ControlPanelFacade> entry : OrderedSPIRegistry.getRegisteredServicesByClass(facadeClassTypes, ControlPanelFacade.class).entrySet()) {
@@ -47,6 +48,7 @@ public final class ControlPanelFacadeEngine {
         }
     }
     
+    @SuppressWarnings({"unchecked", "rawtypes"})
     private void doInit(final Collection<FacadeConfiguration> facadeConfigurations, final Class<?> configurationType, final ControlPanelFacade facade) {
         for (FacadeConfiguration each : facadeConfigurations) {
             if (each.getClass().equals(configurationType)) {
diff --git a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-config/src/main/java/org/apache/shardingsphere/orchestration/core/config/listener/ConfigurationListenerManager.java b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-config/src/main/java/org/apache/shardingsphere/orchestration/core/config/listener/ConfigurationListenerManager.java
index 22e5781..3b2c49f 100644
--- a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-config/src/main/java/org/apache/shardingsphere/orchestration/core/config/listener/ConfigurationListenerManager.java
+++ b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-config/src/main/java/org/apache/shardingsphere/orchestration/core/config/listener/ConfigurationListenerManager.java
@@ -37,8 +37,8 @@ public final class ConfigurationListenerManager {
     
     private final ClusterConfigurationChangedListener clusterConfigurationChangedListener;
     
-    public ConfigurationListenerManager(final String name, final ConfigurationRepository configurationRepository, final Collection<String> shardingSchemaNames) {
-        schemaChangedListener = new SchemaChangedListener(name, configurationRepository, shardingSchemaNames);
+    public ConfigurationListenerManager(final String name, final ConfigurationRepository configurationRepository, final Collection<String> schemaNames) {
+        schemaChangedListener = new SchemaChangedListener(name, configurationRepository, schemaNames);
         propertiesChangedListener = new PropertiesChangedListener(name, configurationRepository);
         authenticationChangedListener = new AuthenticationChangedListener(name, configurationRepository);
         metricsConfigurationChangedListener = new MetricsConfigurationChangedListener(name, configurationRepository);
diff --git a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-config/src/main/java/org/apache/shardingsphere/orchestration/core/config/listener/SchemaChangedListener.java b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-config/src/main/java/org/apache/shardingsphere/orchestration/core/config/listener/SchemaChangedListener.java
index 1301045..7127125 100644
--- a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-config/src/main/java/org/apache/shardingsphere/orchestration/core/config/listener/SchemaChangedListener.java
+++ b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-config/src/main/java/org/apache/shardingsphere/orchestration/core/config/listener/SchemaChangedListener.java
@@ -58,11 +58,11 @@ public final class SchemaChangedListener extends PostOrchestrationRepositoryEven
     
     private final Collection<String> existedSchemaNames = new LinkedList<>();
     
-    public SchemaChangedListener(final String name, final ConfigurationRepository configurationRepository, final Collection<String> shardingSchemaNames) {
-        super(configurationRepository, new ConfigCenterNode(name).getAllSchemaConfigPaths(shardingSchemaNames));
+    public SchemaChangedListener(final String name, final ConfigurationRepository configurationRepository, final Collection<String> schemaNames) {
+        super(configurationRepository, new ConfigCenterNode(name).getAllSchemaConfigPaths(schemaNames));
         configurationService = new ConfigCenter(name, configurationRepository);
         configurationNode = new ConfigCenterNode(name);
-        existedSchemaNames.addAll(shardingSchemaNames);
+        existedSchemaNames.addAll(schemaNames);
     }
     
     @Override
diff --git a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-facade/src/main/java/org/apache/shardingsphere/orchestration/core/facade/OrchestrationFacade.java b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-facade/src/main/java/org/apache/shardingsphere/orchestration/core/facade/OrchestrationFacade.java
index d993abd..a6f3105 100644
--- a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-facade/src/main/java/org/apache/shardingsphere/orchestration/core/facade/OrchestrationFacade.java
+++ b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-facade/src/main/java/org/apache/shardingsphere/orchestration/core/facade/OrchestrationFacade.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.orchestration.core.facade;
 
 import lombok.Getter;
-import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.cluster.configuration.config.ClusterConfiguration;
 import org.apache.shardingsphere.infra.auth.Authentication;
 import org.apache.shardingsphere.infra.config.DataSourceConfiguration;
@@ -39,7 +38,6 @@ import java.util.Properties;
 /**
  * Orchestration facade.
  */
-@Slf4j
 public final class OrchestrationFacade implements AutoCloseable {
     
     private boolean isOverwrite;
@@ -74,29 +72,29 @@ public final class OrchestrationFacade implements AutoCloseable {
     }
     
     /**
-     * Initialize configurations of orchestration.
+     * Online instance.
+     */
+    public void onlineInstance() {
+        registryCenter.persistInstanceOnline();
+        registryCenter.persistDataSourcesNode();
+        listenerManager.init();
+    }
+    
+    /**
+     * Online instance.
      *
      * @param dataSourceConfigurationMap schema data source configuration map
      * @param schemaRuleMap schema rule map
      * @param authentication authentication
      * @param props properties
      */
-    public void initConfigurations(final Map<String, Map<String, DataSourceConfiguration>> dataSourceConfigurationMap, 
-                                   final Map<String, Collection<RuleConfiguration>> schemaRuleMap, final Authentication authentication, final Properties props) {
+    public void onlineInstance(final Map<String, Map<String, DataSourceConfiguration>> dataSourceConfigurationMap,
+                               final Map<String, Collection<RuleConfiguration>> schemaRuleMap, final Authentication authentication, final Properties props) {
         configCenter.persistGlobalConfiguration(authentication, props, isOverwrite);
         for (Entry<String, Map<String, DataSourceConfiguration>> entry : dataSourceConfigurationMap.entrySet()) {
             configCenter.persistConfigurations(entry.getKey(), dataSourceConfigurationMap.get(entry.getKey()), schemaRuleMap.get(entry.getKey()), isOverwrite);
         }
-        initConfigurations();
-    }
-    
-    /**
-     * Initialize configurations of orchestration.
-     */
-    public void initConfigurations() {
-        registryCenter.persistInstanceOnline();
-        registryCenter.persistDataSourcesNode();
-        listenerManager.init();
+        onlineInstance();
     }
     
     /**
diff --git a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-facade/src/main/java/org/apache/shardingsphere/orchestration/core/facade/listener/OrchestrationListenerManager.java b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-facade/src/main/java/org/apache/shardingsphere/orchestration/core/facade/listener/OrchestrationListenerManager.java
index f46fbad..8b2a992 100644
--- a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-facade/src/main/java/org/apache/shardingsphere/orchestration/core/facade/listener/OrchestrationListenerManager.java
+++ b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-facade/src/main/java/org/apache/shardingsphere/orchestration/core/facade/listener/OrchestrationListenerManager.java
@@ -36,11 +36,10 @@ public final class OrchestrationListenerManager {
     
     private final MetaDataListenerManager metaDataListenerManager;
     
-    public OrchestrationListenerManager(final String name, 
-                                        final RegistryRepository registryRepository, final ConfigurationRepository configurationRepository, final Collection<String> shardingSchemaNames) {
-        configurationListenerManager = new ConfigurationListenerManager(name, configurationRepository, shardingSchemaNames);
+    public OrchestrationListenerManager(final String name, final RegistryRepository registryRepository, final ConfigurationRepository configurationRepository, final Collection<String> schemaNames) {
+        configurationListenerManager = new ConfigurationListenerManager(name, configurationRepository, schemaNames);
         registryListenerManager = new RegistryListenerManager(name, registryRepository);
-        metaDataListenerManager = new MetaDataListenerManager(name, configurationRepository, shardingSchemaNames);
+        metaDataListenerManager = new MetaDataListenerManager(name, configurationRepository, schemaNames);
     }
     
     /**
diff --git a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-facade/src/test/java/org/apache/shardingsphere/orchestration/core/facade/OrchestrationFacadeTest.java b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-facade/src/test/java/org/apache/shardingsphere/orchestration/core/facade/OrchestrationFacadeTest.java
index 55d064a..dc2e49e 100644
--- a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-facade/src/test/java/org/apache/shardingsphere/orchestration/core/facade/OrchestrationFacadeTest.java
+++ b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-facade/src/test/java/org/apache/shardingsphere/orchestration/core/facade/OrchestrationFacadeTest.java
@@ -77,14 +77,14 @@ public final class OrchestrationFacadeTest {
     }
     
     @Test
-    public void assertInitWithParameters() {
+    public void assertOnlineInstanceWithParameters() {
         Map<String, DataSourceConfiguration> dataSourceConfigurationMap = Collections.singletonMap("test_ds", mock(DataSourceConfiguration.class));
         Map<String, Collection<RuleConfiguration>> ruleConfigurationMap = Collections.singletonMap("sharding_db", Collections.singletonList(mock(RuleConfiguration.class)));
         ProxyUser proxyUser = new ProxyUser("root", Collections.singleton("db1"));
         Authentication authentication = new Authentication();
         authentication.getUsers().put("root", proxyUser);
         Properties props = new Properties();
-        orchestrationFacade.initConfigurations(Collections.singletonMap("sharding_db", dataSourceConfigurationMap), ruleConfigurationMap, authentication, props);
+        orchestrationFacade.onlineInstance(Collections.singletonMap("sharding_db", dataSourceConfigurationMap), ruleConfigurationMap, authentication, props);
         verify(configCenter).persistConfigurations("sharding_db", dataSourceConfigurationMap, ruleConfigurationMap.get("sharding_db"), false);
         verify(configCenter).persistGlobalConfiguration(authentication, props, false);
         verify(registryCenter).persistInstanceOnline();
@@ -100,8 +100,8 @@ public final class OrchestrationFacadeTest {
     }
     
     @Test
-    public void assertInitWithoutParameters() {
-        orchestrationFacade.initConfigurations();
+    public void assertOnlineInstanceWithoutParameters() {
+        orchestrationFacade.onlineInstance();
         verify(registryCenter).persistInstanceOnline();
         verify(registryCenter).persistDataSourcesNode();
         verify(listenerManager).init();
diff --git a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-metadata/src/main/java/org/apache/shardingsphere/orchestration/core/metadata/listener/MetaDataChangedListener.java b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-metadata/src/main/java/org/apache/shardingsphere/orchestration/core/metadata/listener/MetaDataChangedListener.java
index 1639c69..1e4a4a9 100644
--- a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-metadata/src/main/java/org/apache/shardingsphere/orchestration/core/metadata/listener/MetaDataChangedListener.java
+++ b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-metadata/src/main/java/org/apache/shardingsphere/orchestration/core/metadata/listener/MetaDataChangedListener.java
@@ -37,9 +37,9 @@ public final class MetaDataChangedListener extends PostOrchestrationRepositoryEv
     
     private final Collection<String> schemaNames;
     
-    public MetaDataChangedListener(final String name, final OrchestrationRepository orchestrationRepository, final Collection<String> shardingSchemaNames) {
-        super(orchestrationRepository, new MetaDataCenterNode(name).getAllSchemaMetadataPaths(shardingSchemaNames));
-        this.schemaNames = shardingSchemaNames;
+    public MetaDataChangedListener(final String name, final OrchestrationRepository orchestrationRepository, final Collection<String> schemaNames) {
+        super(orchestrationRepository, new MetaDataCenterNode(name).getAllSchemaMetadataPaths(schemaNames));
+        this.schemaNames = schemaNames;
     }
     
     @Override
diff --git a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-metadata/src/main/java/org/apache/shardingsphere/orchestration/core/metadata/listener/MetaDataListenerManager.java b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-metadata/src/main/java/org/apache/shardingsphere/orchestration/core/metadata/listener/MetaDataListenerManager.java
index 889bb97..e1e905b 100644
--- a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-metadata/src/main/java/org/apache/shardingsphere/orchestration/core/metadata/listener/MetaDataListenerManager.java
+++ b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-metadata/src/main/java/org/apache/shardingsphere/orchestration/core/metadata/listener/MetaDataListenerManager.java
@@ -29,8 +29,8 @@ public final class MetaDataListenerManager {
     
     private final MetaDataChangedListener metaDataChangedListener;
     
-    public MetaDataListenerManager(final String name, final OrchestrationRepository orchestrationRepository, final Collection<String> shardingSchemaNames) {
-        metaDataChangedListener = new MetaDataChangedListener(name, orchestrationRepository, shardingSchemaNames);
+    public MetaDataListenerManager(final String name, final OrchestrationRepository orchestrationRepository, final Collection<String> schemaNames) {
+        metaDataChangedListener = new MetaDataChangedListener(name, orchestrationRepository, schemaNames);
     }
     
     /**
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/main/java/org/apache/shardingsphere/driver/orchestration/internal/datasource/OrchestrationShardingSphereDataSource.java b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/main/java/org/apache/shardingsphere/driver/orchestration/internal/datasource/OrchestrationShardingSphereDataSource.java
index 4a62e5c..414cc06 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/main/java/org/apache/shardingsphere/driver/orchestration/internal/datasource/OrchestrationShardingSphereDataSource.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/main/java/org/apache/shardingsphere/driver/orchestration/internal/datasource/OrchestrationShardingSphereDataSource.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.driver.orchestration.internal.datasource;
 
 import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 import com.google.common.eventbus.Subscribe;
 import lombok.Getter;
@@ -46,7 +45,6 @@ import org.apache.shardingsphere.kernel.context.SchemaContext;
 import org.apache.shardingsphere.kernel.context.SchemaContexts;
 import org.apache.shardingsphere.kernel.context.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.masterslave.rule.MasterSlaveRule;
-import org.apache.shardingsphere.orchestration.repository.api.config.OrchestrationConfiguration;
 import org.apache.shardingsphere.orchestration.core.common.event.DataSourceChangedEvent;
 import org.apache.shardingsphere.orchestration.core.common.event.PropertiesChangedEvent;
 import org.apache.shardingsphere.orchestration.core.common.event.RuleConfigurationsChangedEvent;
@@ -57,6 +55,7 @@ import org.apache.shardingsphere.orchestration.core.metadata.event.MetaDataChang
 import org.apache.shardingsphere.orchestration.core.registry.event.CircuitStateChangedEvent;
 import org.apache.shardingsphere.orchestration.core.registry.event.DisabledStateChangedEvent;
 import org.apache.shardingsphere.orchestration.core.registry.schema.OrchestrationSchema;
+import org.apache.shardingsphere.orchestration.repository.api.config.OrchestrationConfiguration;
 
 import javax.sql.DataSource;
 import java.io.PrintWriter;
@@ -70,7 +69,6 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Properties;
 import java.util.logging.Logger;
 import java.util.stream.Collectors;
 
@@ -90,28 +88,27 @@ public final class OrchestrationShardingSphereDataSource extends AbstractUnsuppo
     private ShardingSphereDataSource dataSource;
     
     public OrchestrationShardingSphereDataSource(final OrchestrationConfiguration orchestrationConfig) throws SQLException {
-        initOrchestration(orchestrationConfig);
-        dataSource = createDataSource();
-        initOrchestrationFacade(null);
+        init(orchestrationConfig);
+        dataSource = loadDataSource();
+        onlineInstanceWithOrchestrationCenter(null);
         disableDataSources();
         persistMetaData(dataSource.getSchemaContexts().getDefaultSchemaContext().getSchema().getMetaData().getSchema());
         initCluster();
     }
     
-    public OrchestrationShardingSphereDataSource(final ShardingSphereDataSource shardingSphereDataSource, final OrchestrationConfiguration orchestrationConfig) {
-        initOrchestration(orchestrationConfig);
-        dataSource = shardingSphereDataSource;
-        initOrchestrationFacade(Collections.singletonMap(DefaultSchema.LOGIC_NAME, DataSourceConverter.getDataSourceConfigurationMap(dataSource.getDataSourceMap())),
-                getRuleConfigurationMap(), dataSource.getSchemaContexts().getProps().getProps(), null);
+    public OrchestrationShardingSphereDataSource(final OrchestrationConfiguration orchestrationConfig, final ClusterConfiguration clusterConfiguration) throws SQLException {
+        init(orchestrationConfig);
+        dataSource = loadDataSource();
+        onlineInstanceWithOrchestrationCenter(clusterConfiguration);
         disableDataSources();
         persistMetaData(dataSource.getSchemaContexts().getDefaultSchemaContext().getSchema().getMetaData().getSchema());
         initCluster();
     }
     
-    public OrchestrationShardingSphereDataSource(final OrchestrationConfiguration orchestrationConfig, final ClusterConfiguration clusterConfiguration) throws SQLException {
-        initOrchestration(orchestrationConfig);
-        dataSource = createDataSource();
-        initOrchestrationFacade(clusterConfiguration);
+    public OrchestrationShardingSphereDataSource(final ShardingSphereDataSource shardingSphereDataSource, final OrchestrationConfiguration orchestrationConfig) {
+        init(orchestrationConfig);
+        dataSource = shardingSphereDataSource;
+        onlineInstanceWithLocalConfiguration(null);
         disableDataSources();
         persistMetaData(dataSource.getSchemaContexts().getDefaultSchemaContext().getSchema().getMetaData().getSchema());
         initCluster();
@@ -119,88 +116,71 @@ public final class OrchestrationShardingSphereDataSource extends AbstractUnsuppo
     
     public OrchestrationShardingSphereDataSource(final ShardingSphereDataSource shardingSphereDataSource, 
                                                  final OrchestrationConfiguration orchestrationConfig, final ClusterConfiguration clusterConfiguration) {
-        initOrchestration(orchestrationConfig);
+        init(orchestrationConfig);
         dataSource = shardingSphereDataSource;
-        initOrchestrationFacade(Collections.singletonMap(DefaultSchema.LOGIC_NAME, DataSourceConverter.getDataSourceConfigurationMap(dataSource.getDataSourceMap())),
-                getRuleConfigurationMap(), dataSource.getSchemaContexts().getProps().getProps(), clusterConfiguration);
+        onlineInstanceWithLocalConfiguration(clusterConfiguration);
         disableDataSources();
         persistMetaData(dataSource.getSchemaContexts().getDefaultSchemaContext().getSchema().getMetaData().getSchema());
         initCluster();
     }
     
-    private void initOrchestration(final OrchestrationConfiguration orchestrationConfig) {
+    private void init(final OrchestrationConfiguration orchestrationConfig) {
         orchestrationFacade.init(orchestrationConfig, Collections.singletonList(DefaultSchema.LOGIC_NAME));
         OrchestrationEventBus.getInstance().register(this);
     }
     
-    private ShardingSphereDataSource createDataSource() throws SQLException {
-        ConfigCenter configService = orchestrationFacade.getConfigCenter();
-        Collection<RuleConfiguration> configurations = configService.loadRuleConfigurations(DefaultSchema.LOGIC_NAME);
-        Preconditions.checkState(!configurations.isEmpty(), "Missing the sharding rule configuration on registry center");
-        Map<String, DataSourceConfiguration> dataSourceConfigurations = configService.loadDataSourceConfigurations(DefaultSchema.LOGIC_NAME);
-        return new ShardingSphereDataSource(DataSourceConverter.getDataSourceMap(dataSourceConfigurations), configurations, configService.loadProperties());
-    }
-    
-    private Map<String, Collection<RuleConfiguration>> getRuleConfigurationMap() {
-        Map<String, Collection<RuleConfiguration>> result = new LinkedHashMap<>(1, 1);
-        result.put(DefaultSchema.LOGIC_NAME, dataSource.getSchemaContexts().getDefaultSchemaContext().getSchema().getConfigurations());
-        return result;
+    private ShardingSphereDataSource loadDataSource() throws SQLException {
+        ConfigCenter configCenter = orchestrationFacade.getConfigCenter();
+        Map<String, DataSourceConfiguration> dataSourceConfigurations = configCenter.loadDataSourceConfigurations(DefaultSchema.LOGIC_NAME);
+        Collection<RuleConfiguration> ruleConfigurations = configCenter.loadRuleConfigurations(DefaultSchema.LOGIC_NAME);
+        return new ShardingSphereDataSource(DataSourceConverter.getDataSourceMap(dataSourceConfigurations), ruleConfigurations, configCenter.loadProperties());
     }
     
-    private void initOrchestrationFacade(final ClusterConfiguration clusterConfiguration) {
-        orchestrationFacade.initConfigurations();
+    private void onlineInstanceWithOrchestrationCenter(final ClusterConfiguration clusterConfiguration) {
+        orchestrationFacade.onlineInstance();
         if (null != clusterConfiguration) {
             orchestrationFacade.initClusterConfiguration(clusterConfiguration);
         }
         dataSourceConfigurations.putAll(orchestrationFacade.getConfigCenter().loadDataSourceConfigurations(DefaultSchema.LOGIC_NAME));
     }
     
-    private void initOrchestrationFacade(final Map<String, Map<String, DataSourceConfiguration>> dataSourceConfigurations,
-                                         final Map<String, Collection<RuleConfiguration>> schemaRules, final Properties props, final ClusterConfiguration clusterConfiguration) {
-        orchestrationFacade.initConfigurations(dataSourceConfigurations, schemaRules, null, props);
+    private void onlineInstanceWithLocalConfiguration(final ClusterConfiguration clusterConfiguration) {
+        Map<String, DataSourceConfiguration> dataSourceConfigurations = DataSourceConverter.getDataSourceConfigurationMap(dataSource.getDataSourceMap());
+        Collection<RuleConfiguration> ruleConfigurations = dataSource.getSchemaContexts().getDefaultSchemaContext().getSchema().getConfigurations();
+        orchestrationFacade.onlineInstance(Collections.singletonMap(DefaultSchema.LOGIC_NAME, dataSourceConfigurations), Collections.singletonMap(DefaultSchema.LOGIC_NAME, ruleConfigurations), 
+                null, dataSource.getSchemaContexts().getProps().getProps());
         if (null != clusterConfiguration) {
             orchestrationFacade.initClusterConfiguration(clusterConfiguration);
         }
-        this.dataSourceConfigurations.putAll(dataSourceConfigurations.get(DefaultSchema.LOGIC_NAME));
-    }
-    
-    private synchronized Map<String, DataSource> getChangedDataSources(final Map<String, DataSource> oldDataSources, final Map<String, DataSourceConfiguration> newDataSources) {
-        Map<String, DataSource> result = new LinkedHashMap<>(oldDataSources);
-        Map<String, DataSourceConfiguration> modifiedDataSources = getModifiedDataSources(newDataSources);
-        result.keySet().removeAll(getDeletedDataSources(newDataSources));
-        result.keySet().removeAll(modifiedDataSources.keySet());
-        result.putAll(DataSourceConverter.getDataSourceMap(modifiedDataSources));
-        result.putAll(DataSourceConverter.getDataSourceMap(getAddedDataSources(newDataSources)));
-        return result;
-    }
-    
-    private synchronized Map<String, DataSourceConfiguration> getModifiedDataSources(final Map<String, DataSourceConfiguration> dataSourceConfigurations) {
-        return dataSourceConfigurations.entrySet().stream().filter(this::isModifiedDataSource).collect(Collectors.toMap(Entry::getKey, Entry::getValue, (key, repeatKey) -> key, LinkedHashMap::new));
-    }
-    
-    private synchronized boolean isModifiedDataSource(final Entry<String, DataSourceConfiguration> dataSourceNameAndConfig) {
-        return dataSourceConfigurations.containsKey(dataSourceNameAndConfig.getKey()) && !dataSourceConfigurations.get(dataSourceNameAndConfig.getKey()).equals(dataSourceNameAndConfig.getValue());
+        this.dataSourceConfigurations.putAll(dataSourceConfigurations);
     }
     
-    private synchronized List<String> getDeletedDataSources(final Map<String, DataSourceConfiguration> dataSourceConfigurations) {
-        List<String> result = new LinkedList<>(this.dataSourceConfigurations.keySet());
-        result.removeAll(dataSourceConfigurations.keySet());
-        return result;
+    // TODO decouple MasterSlaveRule
+    private void disableDataSources() {
+        Collection<String> disabledDataSources = orchestrationFacade.getRegistryCenter().loadDisabledDataSources();
+        if (!disabledDataSources.isEmpty()) {
+            dataSource.getSchemaContexts().getSchemaContexts().forEach((key, value)
+                    -> value.getSchema().getRules().stream().filter(each -> each instanceof MasterSlaveRule).forEach(each -> disableDataSources((MasterSlaveRule) each, disabledDataSources, key)));
+        }
     }
     
-    private synchronized Map<String, DataSourceConfiguration> getAddedDataSources(final Map<String, DataSourceConfiguration> dataSourceConfigurations) {
-        return Maps.filterEntries(dataSourceConfigurations, input -> !this.dataSourceConfigurations.containsKey(input.getKey()));
+    private void disableDataSources(final MasterSlaveRule masterSlaveRule, final Collection<String> disabledDataSources, final String schemaName) {
+        masterSlaveRule.getSingleDataSourceRule().getSlaveDataSourceNames().forEach(each -> {
+            if (disabledDataSources.contains(Joiner.on(".").join(schemaName, each))) {
+                masterSlaveRule.updateRuleStatus(new DataSourceNameDisabledEvent(each, true));
+            }
+        });
     }
     
-    private void persistMetaData(final RuleSchemaMetaData ruleSchemaMetaData) {
-        OrchestrationFacade.getInstance().getMetaDataCenter().persistMetaDataCenterNode(DefaultSchema.LOGIC_NAME, ruleSchemaMetaData);
+    private void persistMetaData(final RuleSchemaMetaData metaData) {
+        orchestrationFacade.getMetaDataCenter().persistMetaDataCenterNode(DefaultSchema.LOGIC_NAME, metaData);
     }
     
     private void initCluster() {
-        ClusterConfiguration clusterConfiguration = orchestrationFacade.getConfigCenter().loadClusterConfiguration();
-        if (null != clusterConfiguration && null != clusterConfiguration.getHeartbeat()) {
-            List<FacadeConfiguration> facadeConfigurations = new LinkedList<>();
-            facadeConfigurations.add(clusterConfiguration);
+        ClusterConfiguration clusterConfig = orchestrationFacade.getConfigCenter().loadClusterConfiguration();
+        if (null != clusterConfig && null != clusterConfig.getHeartbeat()) {
+            Collection<FacadeConfiguration> facadeConfigurations = new LinkedList<>();
+            facadeConfigurations.add(clusterConfig);
             new ControlPanelFacadeEngine().init(facadeConfigurations);
         }
     }
@@ -218,12 +198,12 @@ public final class OrchestrationShardingSphereDataSource extends AbstractUnsuppo
         Map<String, SchemaContext> schemaContexts = new HashMap<>(dataSource.getSchemaContexts().getSchemaContexts().size());
         SchemaContext oldSchemaContext = dataSource.getSchemaContexts().getSchemaContexts().get(DefaultSchema.LOGIC_NAME);
         schemaContexts.put(DefaultSchema.LOGIC_NAME, new SchemaContext(oldSchemaContext.getName(),
-                getChangedShardingSphereSchema(oldSchemaContext.getSchema(), event.getRuleSchemaMetaData()), oldSchemaContext.getRuntimeContext()));
+                getChangedSchema(oldSchemaContext.getSchema(), event.getRuleSchemaMetaData()), oldSchemaContext.getRuntimeContext()));
         dataSource = new ShardingSphereDataSource(new SchemaContexts(schemaContexts, dataSource.getSchemaContexts().getProps(), dataSource.getSchemaContexts().getAuthentication()));
     }
     
     /**
-     * Renew sharding rule.
+     * Renew rule configuration.
      *
      * @param ruleConfigurationsChangedEvent rule configurations changed event
      */
@@ -238,7 +218,7 @@ public final class OrchestrationShardingSphereDataSource extends AbstractUnsuppo
     }
     
     /**
-     * Renew sharding data source.
+     * Renew data sources.
      *
      * @param dataSourceChangedEvent data source changed event
      */
@@ -310,26 +290,38 @@ public final class OrchestrationShardingSphereDataSource extends AbstractUnsuppo
         }
     }
     
-    private ShardingSphereSchema getChangedShardingSphereSchema(final ShardingSphereSchema oldShardingSphereSchema, final RuleSchemaMetaData newRuleSchemaMetaData) {
-        ShardingSphereMetaData metaData = new ShardingSphereMetaData(oldShardingSphereSchema.getMetaData().getDataSources(), newRuleSchemaMetaData);
-        return new ShardingSphereSchema(oldShardingSphereSchema.getDatabaseType(), oldShardingSphereSchema.getConfigurations(),
-                oldShardingSphereSchema.getRules(), oldShardingSphereSchema.getDataSources(), metaData);
+    private synchronized Map<String, DataSource> getChangedDataSources(final Map<String, DataSource> oldDataSources, final Map<String, DataSourceConfiguration> newDataSources) {
+        Map<String, DataSource> result = new LinkedHashMap<>(oldDataSources);
+        Map<String, DataSourceConfiguration> modifiedDataSources = getModifiedDataSources(newDataSources);
+        result.keySet().removeAll(getDeletedDataSources(newDataSources));
+        result.keySet().removeAll(modifiedDataSources.keySet());
+        result.putAll(DataSourceConverter.getDataSourceMap(modifiedDataSources));
+        result.putAll(DataSourceConverter.getDataSourceMap(getAddedDataSources(newDataSources)));
+        return result;
     }
     
-    private void disableDataSources() {
-        Collection<String> disabledDataSources = OrchestrationFacade.getInstance().getRegistryCenter().loadDisabledDataSources();
-        if (!disabledDataSources.isEmpty()) {
-            dataSource.getSchemaContexts().getSchemaContexts().forEach((key, value)
-                -> value.getSchema().getRules().stream().filter(each -> each instanceof MasterSlaveRule).forEach(e -> disableDataSources((MasterSlaveRule) e, disabledDataSources, key)));
-        }
+    private synchronized Map<String, DataSourceConfiguration> getModifiedDataSources(final Map<String, DataSourceConfiguration> dataSourceConfigurations) {
+        return dataSourceConfigurations.entrySet().stream().filter(this::isModifiedDataSource).collect(Collectors.toMap(Entry::getKey, Entry::getValue, (key, repeatKey) -> key, LinkedHashMap::new));
     }
     
-    private void disableDataSources(final MasterSlaveRule masterSlaveRule, final Collection<String> disabledDataSources, final String schemaName) {
-        masterSlaveRule.getSingleDataSourceRule().getSlaveDataSourceNames().forEach(each -> {
-            if (disabledDataSources.contains(Joiner.on(".").join(schemaName, each))) {
-                masterSlaveRule.updateRuleStatus(new DataSourceNameDisabledEvent(each, true));
-            }
-        });
+    private synchronized boolean isModifiedDataSource(final Entry<String, DataSourceConfiguration> dataSourceNameAndConfig) {
+        return dataSourceConfigurations.containsKey(dataSourceNameAndConfig.getKey()) && !dataSourceConfigurations.get(dataSourceNameAndConfig.getKey()).equals(dataSourceNameAndConfig.getValue());
+    }
+    
+    private synchronized List<String> getDeletedDataSources(final Map<String, DataSourceConfiguration> dataSourceConfigurations) {
+        List<String> result = new LinkedList<>(this.dataSourceConfigurations.keySet());
+        result.removeAll(dataSourceConfigurations.keySet());
+        return result;
+    }
+    
+    private synchronized Map<String, DataSourceConfiguration> getAddedDataSources(final Map<String, DataSourceConfiguration> dataSourceConfigurations) {
+        return Maps.filterEntries(dataSourceConfigurations, input -> !this.dataSourceConfigurations.containsKey(input.getKey()));
+    }
+    
+    private ShardingSphereSchema getChangedSchema(final ShardingSphereSchema oldSchema, final RuleSchemaMetaData newRuleSchemaMetaData) {
+        ShardingSphereMetaData metaData = new ShardingSphereMetaData(oldSchema.getMetaData().getDataSources(), newRuleSchemaMetaData);
+        return new ShardingSphereSchema(oldSchema.getDatabaseType(), oldSchema.getConfigurations(),
+                oldSchema.getRules(), oldSchema.getDataSources(), metaData);
     }
     
     @Override
diff --git a/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContext.java b/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContext.java
index b0d7a3a..6b6f557 100644
--- a/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContext.java
+++ b/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContext.java
@@ -21,6 +21,9 @@ import lombok.Getter;
 import org.apache.shardingsphere.kernel.context.runtime.RuntimeContext;
 import org.apache.shardingsphere.kernel.context.schema.ShardingSphereSchema;
 
+/**
+ * Schema context.
+ */
 @Getter
 public final class SchemaContext {
     
diff --git a/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContexts.java b/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContexts.java
index 4252a5c..845e981 100644
--- a/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContexts.java
+++ b/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContexts.java
@@ -30,6 +30,9 @@ import java.util.LinkedList;
 import java.util.Map;
 import java.util.Properties;
 
+/**
+ * Schema contexts.
+ */
 @Getter
 public final class SchemaContexts implements SchemaContextsAware {
     
diff --git a/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContextsAware.java b/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContextsAware.java
index 5730566..0d03df5 100644
--- a/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContextsAware.java
+++ b/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContextsAware.java
@@ -24,7 +24,6 @@ import java.util.Map;
 
 /**
  * Schema contexts aware.
- * 
  */
 public interface SchemaContextsAware extends AutoCloseable {
     
diff --git a/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContextsBuilder.java b/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContextsBuilder.java
index c9300a8..25e99e5 100644
--- a/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContextsBuilder.java
+++ b/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/SchemaContextsBuilder.java
@@ -50,6 +50,9 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
 
+/**
+ * Schema contexts builder.
+ */
 @Slf4j(topic = "ShardingSphere-schemaContexts")
 public final class SchemaContextsBuilder {
     
diff --git a/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/runtime/RuntimeContext.java b/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/runtime/RuntimeContext.java
index 52f5f70..2c963d8 100644
--- a/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/runtime/RuntimeContext.java
+++ b/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/runtime/RuntimeContext.java
@@ -18,10 +18,15 @@
 package org.apache.shardingsphere.kernel.context.runtime;
 
 import lombok.Getter;
+import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
 import org.apache.shardingsphere.sql.parser.SQLParserEngine;
 import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
 
+/**
+ * Runtime context.
+ */
+@RequiredArgsConstructor
 @Getter
 public final class RuntimeContext {
     
@@ -32,12 +37,4 @@ public final class RuntimeContext {
     private final SQLParserEngine sqlParserEngine;
     
     private final ShardingTransactionManagerEngine transactionManagerEngine;
-    
-    public RuntimeContext(final CachedDatabaseMetaData cachedDatabaseMetaData, 
-                          final ExecutorKernel executorKernel, final SQLParserEngine sqlParserEngine, final ShardingTransactionManagerEngine transactionManagerEngine) {
-        this.cachedDatabaseMetaData = cachedDatabaseMetaData;
-        this.executorKernel = executorKernel;
-        this.sqlParserEngine = sqlParserEngine;
-        this.transactionManagerEngine = transactionManagerEngine;
-    }
 }
diff --git a/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/schema/ShardingSphereSchema.java b/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/schema/ShardingSphereSchema.java
index 9ca478c..c617464 100644
--- a/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/schema/ShardingSphereSchema.java
+++ b/shardingsphere-kernel/shardingsphere-kernel-context/src/main/java/org/apache/shardingsphere/kernel/context/schema/ShardingSphereSchema.java
@@ -30,6 +30,9 @@ import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.Map;
 
+/**
+ * ShardingSphere schema.
+ */
 @Getter
 public final class ShardingSphereSchema {
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-orchestration/src/main/java/org/apache/shardingsphere/proxy/orchestration/OrchestrationConfigurationConverter.java b/shardingsphere-proxy/shardingsphere-proxy-orchestration/src/main/java/org/apache/shardingsphere/proxy/orchestration/OrchestrationConfigurationConverter.java
index e803df0..2e6c95c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-orchestration/src/main/java/org/apache/shardingsphere/proxy/orchestration/OrchestrationConfigurationConverter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-orchestration/src/main/java/org/apache/shardingsphere/proxy/orchestration/OrchestrationConfigurationConverter.java
@@ -82,9 +82,9 @@ public final class OrchestrationConfigurationConverter extends AbstractConfigura
     private void initOrchestrationConfigurations(
             final YamlProxyServerConfiguration serverConfig, final Map<String, YamlProxyRuleConfiguration> ruleConfigs, final OrchestrationFacade orchestrationFacade) {
         if (isEmptyLocalConfiguration(serverConfig, ruleConfigs)) {
-            orchestrationFacade.initConfigurations();
+            orchestrationFacade.onlineInstance();
         } else {
-            orchestrationFacade.initConfigurations(getDataSourceConfigurationMap(ruleConfigs),
+            orchestrationFacade.onlineInstance(getDataSourceConfigurationMap(ruleConfigs),
                     getRuleConfigurations(ruleConfigs), new AuthenticationYamlSwapper().swapToObject(serverConfig.getAuthentication()), serverConfig.getProps());
         }
         orchestrationFacade.initMetricsConfiguration(Optional.ofNullable(serverConfig.getMetrics()).map(new MetricsConfigurationYamlSwapper()::swapToObject).orElse(null));