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/08/02 07:52:40 UTC

[shardingsphere] branch master updated: Remove ShardingSphereSchema.dataSourceParameters (#6569)

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

zhangliang 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 77c2100  Remove ShardingSphereSchema.dataSourceParameters (#6569)
77c2100 is described below

commit 77c21002b5ea5ffaf5d064e3ed9f563f047b4352
Author: Liang Zhang <te...@163.com>
AuthorDate: Sun Aug 2 15:52:21 2020 +0800

    Remove ShardingSphereSchema.dataSourceParameters (#6569)
    
    * Decouple ShardingSphereSchema.dataSourceParameters
    
    * Remove ShardingSphereSchema.dataSourceParameters
    
    * Add ProxyDataSourceContext
    
    * Refactor SchemaContextsBuilder
---
 .../core/schema/OrchestrationSchemaContexts.java   | 19 +++---
 .../kernel/context/SchemaContextsBuilder.java      | 27 +++-----
 .../context/schema/ShardingSphereSchema.java       |  9 ---
 .../backend/schema/ProxyDataSourceContext.java     | 71 ++++++++++++++++++++++
 .../org/apache/shardingsphere/proxy/Bootstrap.java | 45 ++------------
 .../proxy/config/util/DataSourceConverter.java     | 23 +++++++
 .../schema/ProxyOrchestrationSchemaContexts.java   | 25 ++++----
 7 files changed, 127 insertions(+), 92 deletions(-)

diff --git a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-schema/src/main/java/org/apache/shardingsphere/orchestration/core/schema/OrchestrationSchemaContexts.java b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-schema/src/main/java/org/apache/shardingsphere/orchestration/core/schema/OrchestrationSchemaContexts.java
index 34a916d..a38b5e3 100644
--- a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-schema/src/main/java/org/apache/shardingsphere/orchestration/core/schema/OrchestrationSchemaContexts.java
+++ b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-schema/src/main/java/org/apache/shardingsphere/orchestration/core/schema/OrchestrationSchemaContexts.java
@@ -327,9 +327,8 @@ public abstract class OrchestrationSchemaContexts implements SchemaContextsAware
         Map<String, Map<String, DataSource>> dataSourcesMap = createDataSourcesMap(Collections.singletonMap(schemaName, schemaAddedEvent.getDataSourceConfigurations()));
         Map<String, Map<String, DataSourceParameter>> dataSourceParametersMap = createDataSourceParametersMap(Collections.singletonMap(schemaName, schemaAddedEvent.getDataSourceConfigurations()));
         DatabaseType databaseType = getDatabaseType(dataSourceParametersMap.values().iterator().next().values().iterator().next());
-        SchemaContextsBuilder schemaContextsBuilder = new SchemaContextsBuilder(dataSourcesMap, dataSourceParametersMap,
-                schemaContexts.getAuthentication(), databaseType, Collections.singletonMap(schemaName, schemaAddedEvent.getRuleConfigurations()),
-                schemaContexts.getProps().getProps());
+        SchemaContextsBuilder schemaContextsBuilder = new SchemaContextsBuilder(dataSourcesMap, databaseType, 
+                Collections.singletonMap(schemaName, schemaAddedEvent.getRuleConfigurations()), schemaContexts.getAuthentication(), schemaContexts.getProps().getProps());
         return schemaContextsBuilder.build().getSchemaContexts().get(schemaName);
     }
     
@@ -359,8 +358,7 @@ public abstract class OrchestrationSchemaContexts implements SchemaContextsAware
     private SchemaContext getChangedSchemaContext(final SchemaContext oldSchemaContext, final Collection<RuleConfiguration> configurations) throws SQLException {
         ShardingSphereSchema oldSchema = oldSchemaContext.getSchema();
         SchemaContextsBuilder builder = new SchemaContextsBuilder(Collections.singletonMap(oldSchemaContext.getName(), oldSchema.getDataSources()),
-                Collections.singletonMap(oldSchemaContext.getName(), oldSchema.getDataSourceParameters()),
-                schemaContexts.getAuthentication(), oldSchema.getDatabaseType(), Collections.singletonMap(oldSchemaContext.getName(), configurations), schemaContexts.getProps().getProps());
+                oldSchema.getDatabaseType(), Collections.singletonMap(oldSchemaContext.getName(), configurations), schemaContexts.getAuthentication(), schemaContexts.getProps().getProps());
         return builder.build().getSchemaContexts().values().iterator().next();
     }
     
@@ -372,9 +370,8 @@ public abstract class OrchestrationSchemaContexts implements SchemaContextsAware
         oldSchemaContext.getRuntimeContext().getTransactionManagerEngine().close();
         Map<String, Map<String, DataSource>> dataSourcesMap = Collections.singletonMap(oldSchemaContext.getName(), getNewDataSources(oldSchemaContext.getSchema().getDataSources(), 
                 deletedDataSources, getAddedDataSources(oldSchemaContext, newDataSources), modifiedDataSources));
-        Map<String, Map<String, DataSourceParameter>> dataSourceParametersMap = createDataSourceParametersMap(Collections.singletonMap(oldSchemaContext.getName(), newDataSources));
-        return new SchemaContextsBuilder(dataSourcesMap, dataSourceParametersMap, schemaContexts.getAuthentication(), oldSchemaContext.getSchema().getDatabaseType(), 
-                Collections.singletonMap(oldSchemaContext.getName(), oldSchemaContext.getSchema().getConfigurations()), 
+        return new SchemaContextsBuilder(dataSourcesMap, oldSchemaContext.getSchema().getDatabaseType(), 
+                Collections.singletonMap(oldSchemaContext.getName(), oldSchemaContext.getSchema().getConfigurations()), schemaContexts.getAuthentication(), 
                 schemaContexts.getProps().getProps()).build().getSchemaContexts().get(oldSchemaContext.getName());
     }
     
@@ -389,13 +386,13 @@ public abstract class OrchestrationSchemaContexts implements SchemaContextsAware
     }
     
     private Collection<String> getDeletedDataSources(final SchemaContext oldSchemaContext, final Map<String, DataSourceConfiguration> newDataSources) {
-        Collection<String> result = new LinkedList<>(oldSchemaContext.getSchema().getDataSourceParameters().keySet());
+        Collection<String> result = new LinkedList<>(oldSchemaContext.getSchema().getDataSources().keySet());
         result.removeAll(newDataSources.keySet());
         return result;
     }
     
     /**
-     * Get added dataSources.
+     * Get added data sources.
      * 
      * @param oldSchemaContext old schema context
      * @param newDataSources new data sources
@@ -405,7 +402,7 @@ public abstract class OrchestrationSchemaContexts implements SchemaContextsAware
     public abstract Map<String, DataSource> getAddedDataSources(SchemaContext oldSchemaContext, Map<String, DataSourceConfiguration> newDataSources) throws Exception;
     
     /**
-     * Get modified dataSources.
+     * Get modified data sources.
      * 
      * @param oldSchemaContext old schema context
      * @param newDataSources new data sources
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 7876b94..7aa22df 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
@@ -35,7 +35,6 @@ import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRulesBuilder;
 import org.apache.shardingsphere.kernel.context.runtime.CachedDatabaseMetaData;
 import org.apache.shardingsphere.kernel.context.runtime.RuntimeContext;
-import org.apache.shardingsphere.kernel.context.schema.DataSourceParameter;
 import org.apache.shardingsphere.kernel.context.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngineFactory;
 import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
@@ -60,36 +59,27 @@ public final class SchemaContextsBuilder {
     
     private final Map<String, Map<String, DataSource>> dataSources;
     
-    private final Map<String, Map<String, DataSourceParameter>> dataSourceParameters = new LinkedHashMap<>();
-    
     private final Map<String, Collection<RuleConfiguration>> configurations;
     
     private final ConfigurationProperties props;
     
-    private final ExecutorKernel executorKernel;
-    
     private final Authentication authentication;
     
+    private final ExecutorKernel executorKernel;
+    
     public SchemaContextsBuilder(final Map<String, Map<String, DataSource>> dataSources,
                                  final DatabaseType databaseType, final Map<String, Collection<RuleConfiguration>> configurations, final Properties props) {
-        this.dataSources = dataSources;
-        this.databaseType = databaseType;
-        this.configurations = configurations;
-        this.props = new ConfigurationProperties(null == props ? new Properties() : props);
-        executorKernel = new ExecutorKernel(this.props.<Integer>getValue(ConfigurationPropertyKey.EXECUTOR_SIZE));
-        authentication = new Authentication();
-        log(configurations, props);
+        this(dataSources, databaseType, configurations, new Authentication(), props);
     }
     
-    public SchemaContextsBuilder(final Map<String, Map<String, DataSource>> dataSources, final Map<String, Map<String, DataSourceParameter>> dataSourceParameters, final Authentication authentication, 
-                                 final DatabaseType databaseType, final Map<String, Collection<RuleConfiguration>> configurations, final Properties props) {
+    public SchemaContextsBuilder(final Map<String, Map<String, DataSource>> dataSources, 
+                                 final DatabaseType databaseType, final Map<String, Collection<RuleConfiguration>> configurations, final Authentication authentication, final Properties props) {
         this.dataSources = dataSources;
         this.databaseType = databaseType;
         this.configurations = configurations;
+        this.authentication = authentication;
         this.props = new ConfigurationProperties(null == props ? new Properties() : props);
         executorKernel = new ExecutorKernel(this.props.<Integer>getValue(ConfigurationPropertyKey.EXECUTOR_SIZE));
-        this.dataSourceParameters.putAll(dataSourceParameters);
-        this.authentication = authentication;
         log(configurations, props);
     }
     
@@ -124,10 +114,7 @@ public final class SchemaContextsBuilder {
         Map<String, DataSource> dataSources = this.dataSources.get(schemaName);
         Collection<RuleConfiguration> configurations = this.configurations.get(schemaName);
         Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.build(configurations, dataSources.keySet());
-        if (dataSourceParameters.isEmpty()) {
-            return new ShardingSphereSchema(databaseType, configurations, rules, dataSources, createMetaData(dataSources, rules));
-        }
-        return new ShardingSphereSchema(databaseType, configurations, rules, dataSources, dataSourceParameters.get(schemaName), createMetaData(dataSources, rules));
+        return new ShardingSphereSchema(databaseType, configurations, rules, dataSources, createMetaData(dataSources, rules));
     }
     
     private ShardingSphereMetaData createMetaData(final Map<String, DataSource> dataSourceMap, final Collection<ShardingSphereRule> rules) throws SQLException {
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 c617464..514fb2e 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
@@ -44,8 +44,6 @@ public final class ShardingSphereSchema {
     
     private final Map<String, DataSource> dataSources = new LinkedHashMap<>();
     
-    private final Map<String, DataSourceParameter> dataSourceParameters = new LinkedHashMap<>();
-    
     private final ShardingSphereMetaData metaData;
     
     public ShardingSphereSchema(final DatabaseType databaseType, final Collection<RuleConfiguration> configurations, final Collection<ShardingSphereRule> rules, 
@@ -57,12 +55,6 @@ public final class ShardingSphereSchema {
         metaData = shardingSphereMetaData;
     }
     
-    public ShardingSphereSchema(final DatabaseType databaseType, final Collection<RuleConfiguration> configurations, final Collection<ShardingSphereRule> rules,
-                                final Map<String, DataSource> dataSourceMap, final Map<String, DataSourceParameter> dataSourceParameters, final ShardingSphereMetaData shardingSphereMetaData) {
-        this(databaseType, configurations, rules, dataSourceMap, shardingSphereMetaData);
-        this.dataSourceParameters.putAll(dataSourceParameters);
-    }
-    
     /**
      * Close data sources.
      * @param dataSources data sources
@@ -70,7 +62,6 @@ public final class ShardingSphereSchema {
     public void closeDataSources(final Collection<String> dataSources) {
         for (String each :dataSources) {
             close(this.dataSources.get(each));
-            dataSourceParameters.remove(each);
         }
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/schema/ProxyDataSourceContext.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/schema/ProxyDataSourceContext.java
new file mode 100644
index 0000000..028248a
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/schema/ProxyDataSourceContext.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.proxy.backend.schema;
+
+import lombok.Getter;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.kernel.context.schema.DataSourceParameter;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.datasource.JDBCRawBackendDataSourceFactory;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.JDBCDriverURLRecognizerEngine;
+
+import javax.sql.DataSource;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+
+/**
+ * Proxy data source context.
+ */
+@Getter
+public final class ProxyDataSourceContext {
+    
+    private final DatabaseType databaseType;
+    
+    private final Map<String, Map<String, DataSource>> dataSourcesMap;
+    
+    public ProxyDataSourceContext(final Map<String, Map<String, DataSourceParameter>> schemaDataSources) {
+        databaseType = schemaDataSources.isEmpty() ? new MySQLDatabaseType() : DatabaseTypes.getActualDatabaseType(getDatabaseTypeName(schemaDataSources));
+        dataSourcesMap = createDataSourcesMap(schemaDataSources);
+    }
+    
+    private static String getDatabaseTypeName(final Map<String, Map<String, DataSourceParameter>> schemaDataSources) {
+        return JDBCDriverURLRecognizerEngine.getJDBCDriverURLRecognizer(schemaDataSources.values().iterator().next().values().iterator().next().getUrl()).getDatabaseType();
+    }
+    
+    private static Map<String, Map<String, DataSource>> createDataSourcesMap(final Map<String, Map<String, DataSourceParameter>> schemaDataSources) {
+        return schemaDataSources.entrySet().stream().collect(Collectors.toMap(Entry::getKey, entry -> createDataSources(entry.getValue()), (oldValue, currentValue) -> oldValue, LinkedHashMap::new));
+    }
+    
+    private static Map<String, DataSource> createDataSources(final Map<String, DataSourceParameter> dataSourceParameters) {
+        Map<String, DataSource> result = new LinkedHashMap<>(dataSourceParameters.size(), 1);
+        for (Entry<String, DataSourceParameter> entry : dataSourceParameters.entrySet()) {
+            try {
+                result.put(entry.getKey(), JDBCRawBackendDataSourceFactory.getInstance().build(entry.getKey(), entry.getValue()));
+                // CHECKSTYLE:OFF
+            } catch (final Exception ex) {
+                // CHECKSTYLE:ON
+                throw new ShardingSphereException(String.format("Can not build data source, name is `%s`.", entry.getKey()), ex);
+            }
+        }
+        return result;
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/Bootstrap.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/Bootstrap.java
index 2b09861..5ddf415 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/Bootstrap.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/Bootstrap.java
@@ -28,19 +28,14 @@ import org.apache.shardingsphere.control.panel.spi.opentracing.OpenTracingConfig
 import org.apache.shardingsphere.db.protocol.mysql.constant.MySQLServerInfo;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.constant.Constants;
-import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
-import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.log.ConfigurationLogger;
 import org.apache.shardingsphere.kernel.context.SchemaContextsAware;
 import org.apache.shardingsphere.kernel.context.SchemaContextsBuilder;
-import org.apache.shardingsphere.kernel.context.schema.DataSourceParameter;
 import org.apache.shardingsphere.metrics.configuration.config.MetricsConfiguration;
 import org.apache.shardingsphere.orchestration.core.facade.OrchestrationFacade;
 import org.apache.shardingsphere.proxy.arg.BootstrapArguments;
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.datasource.JDBCRawBackendDataSourceFactory;
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.JDBCDriverURLRecognizerEngine;
+import org.apache.shardingsphere.proxy.backend.schema.ProxyDataSourceContext;
 import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
 import org.apache.shardingsphere.proxy.config.ProxyConfiguration;
 import org.apache.shardingsphere.proxy.config.ProxyConfigurationLoader;
@@ -54,13 +49,10 @@ import javax.sql.DataSource;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.SQLException;
-import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Objects;
-import java.util.stream.Collectors;
 
 /**
  * ShardingSphere-Proxy Bootstrap.
@@ -80,13 +72,12 @@ public final class Bootstrap {
         int port = bootstrapArgs.getPort();
         System.setProperty(Constants.PORT_KEY, String.valueOf(port));
         YamlProxyConfiguration yamlConfig = ProxyConfigurationLoader.load(bootstrapArgs.getConfigurationPath());
-        boolean orchestrationEnabled = null != yamlConfig.getServerConfiguration().getOrchestration();
-        if (orchestrationEnabled) {
+        if (null == yamlConfig.getServerConfiguration().getOrchestration()) {
+            init(new YamlProxyConfigurationSwapper().swap(yamlConfig), port, false);
+        } else {
             try (OrchestrationFacade orchestrationFacade = OrchestrationFacade.getInstance()) {
                 init(new OrchestrationBootstrap(orchestrationFacade).init(yamlConfig), port, true);
             }
-        } else {
-            init(new YamlProxyConfigurationSwapper().swap(yamlConfig), port, false);
         }
     }
     
@@ -105,36 +96,12 @@ public final class Bootstrap {
     }
     
     private static void initProxySchemaContexts(final ProxyConfiguration proxyConfig, final boolean orchestrationEnabled) throws SQLException {
-        // TODO Consider loading from configuration.
-        Map<String, Map<String, DataSourceParameter>> schemaDataSources = proxyConfig.getSchemaDataSources();
-        DatabaseType databaseType = schemaDataSources.isEmpty() ? new MySQLDatabaseType() : DatabaseTypes.getActualDatabaseType(getDatabaseTypeName(schemaDataSources));
+        ProxyDataSourceContext dataSourceContext = new ProxyDataSourceContext(proxyConfig.getSchemaDataSources());
         SchemaContextsBuilder schemaContextsBuilder = new SchemaContextsBuilder(
-                createDataSourcesMap(schemaDataSources), schemaDataSources, proxyConfig.getAuthentication(), databaseType, proxyConfig.getSchemaRules(), proxyConfig.getProps());
+                dataSourceContext.getDataSourcesMap(), dataSourceContext.getDatabaseType(), proxyConfig.getSchemaRules(), proxyConfig.getAuthentication(), proxyConfig.getProps());
         ProxySchemaContexts.getInstance().init(createSchemaContextsAware(schemaContextsBuilder, orchestrationEnabled));
     }
     
-    private static String getDatabaseTypeName(final Map<String, Map<String, DataSourceParameter>> schemaDataSources) {
-        return JDBCDriverURLRecognizerEngine.getJDBCDriverURLRecognizer(schemaDataSources.values().iterator().next().values().iterator().next().getUrl()).getDatabaseType();
-    }
-    
-    private static Map<String, Map<String, DataSource>> createDataSourcesMap(final Map<String, Map<String, DataSourceParameter>> schemaDataSources) {
-        return schemaDataSources.entrySet().stream().collect(Collectors.toMap(Entry::getKey, entry -> createDataSources(entry.getValue()), (oldValue, currentValue) -> oldValue, LinkedHashMap::new));
-    }
-    
-    private static Map<String, DataSource> createDataSources(final Map<String, DataSourceParameter> dataSourceParameters) {
-        Map<String, DataSource> result = new LinkedHashMap<>(dataSourceParameters.size(), 1);
-        for (Entry<String, DataSourceParameter> entry : dataSourceParameters.entrySet()) {
-            try {
-                result.put(entry.getKey(), JDBCRawBackendDataSourceFactory.getInstance().build(entry.getKey(), entry.getValue()));
-                // CHECKSTYLE:OFF
-            } catch (final Exception ex) {
-                // CHECKSTYLE:ON
-                throw new ShardingSphereException(String.format("Can not build data source, name is `%s`.", entry.getKey()), ex);
-            }
-        }
-        return result;
-    }
-    
     private static SchemaContextsAware createSchemaContextsAware(final SchemaContextsBuilder schemaContextsBuilder, final boolean orchestrationEnabled) throws SQLException {
         return orchestrationEnabled ? new ProxyOrchestrationSchemaContexts(schemaContextsBuilder.build()) : schemaContextsBuilder.build();
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/util/DataSourceConverter.java b/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/util/DataSourceConverter.java
index 63971fd..c187d4f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/util/DataSourceConverter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/util/DataSourceConverter.java
@@ -24,6 +24,7 @@ import org.apache.shardingsphere.infra.config.DataSourceConfiguration;
 import org.apache.shardingsphere.kernel.context.schema.DataSourceParameter;
 import org.apache.shardingsphere.proxy.config.yaml.YamlDataSourceParameter;
 
+import javax.sql.DataSource;
 import java.lang.reflect.Field;
 import java.util.LinkedHashMap;
 import java.util.Map;
@@ -121,4 +122,26 @@ public final class DataSourceConverter {
         result.getProps().put("readOnly", dataSourceParameter.isReadOnly());
         return result;
     }
+    
+    /**
+     * Get data source parameter.
+     *
+     * @param dataSource data source
+     * @return data source parameter
+     */
+    public static DataSourceParameter getDataSourceParameter(final DataSource dataSource) {
+        DataSourceParameter result = new DataSourceParameter();
+        HikariDataSource hikariDataSource = (HikariDataSource) dataSource;
+        result.setUrl(hikariDataSource.getJdbcUrl());
+        result.setUsername(hikariDataSource.getUsername());
+        result.setPassword(hikariDataSource.getPassword());
+        result.setConnectionTimeoutMilliseconds(hikariDataSource.getConnectionTimeout());
+        result.setIdleTimeoutMilliseconds(hikariDataSource.getIdleTimeout());
+        result.setMaxLifetimeMilliseconds(hikariDataSource.getMaxLifetime());
+        result.setMaxPoolSize(hikariDataSource.getMaximumPoolSize());
+        result.setMinPoolSize(hikariDataSource.getMinimumIdle());
+        // TODO setMaintenanceIntervalMilliseconds
+        result.setReadOnly(hikariDataSource.isReadOnly());
+        return result;
+    }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-orchestration/src/main/java/org/apache/shardingsphere/proxy/orchestration/schema/ProxyOrchestrationSchemaContexts.java b/shardingsphere-proxy/shardingsphere-proxy-orchestration/src/main/java/org/apache/shardingsphere/proxy/orchestration/schema/ProxyOrchestrationSchemaContexts.java
index 55a5036..2dc3700 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-orchestration/src/main/java/org/apache/shardingsphere/proxy/orchestration/schema/ProxyOrchestrationSchemaContexts.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-orchestration/src/main/java/org/apache/shardingsphere/proxy/orchestration/schema/ProxyOrchestrationSchemaContexts.java
@@ -48,8 +48,7 @@ public final class ProxyOrchestrationSchemaContexts extends OrchestrationSchemaC
     @Override
     public Map<String, DataSource> getAddedDataSources(final SchemaContext oldSchemaContext, final Map<String, DataSourceConfiguration> newDataSources) throws Exception {
         Map<String, DataSourceParameter> newDataSourceParameters = DataSourceConverter.getDataSourceParameterMap(newDataSources);
-        Map<String, DataSourceParameter> parameters = 
-                Maps.filterEntries(newDataSourceParameters, input -> !oldSchemaContext.getSchema().getDataSourceParameters().containsKey(input.getKey()));
+        Map<String, DataSourceParameter> parameters = Maps.filterKeys(newDataSourceParameters, each -> !oldSchemaContext.getSchema().getDataSources().containsKey(each));
         return createDataSources(parameters);
     }
     
@@ -58,15 +57,15 @@ public final class ProxyOrchestrationSchemaContexts extends OrchestrationSchemaC
         Map<String, DataSourceParameter> newDataSourceParameters = DataSourceConverter.getDataSourceParameterMap(newDataSources);
         Map<String, DataSourceParameter> parameters = new LinkedHashMap<>();
         for (Entry<String, DataSourceParameter> entry : newDataSourceParameters.entrySet()) {
-            if (isModifiedDataSource(oldSchemaContext.getSchema().getDataSourceParameters(), entry)) {
+            if (isModifiedDataSource(oldSchemaContext.getSchema().getDataSources(), entry)) {
                 parameters.put(entry.getKey(), entry.getValue());
             }
         }
         return createDataSources(parameters);
     }
     
-    private synchronized boolean isModifiedDataSource(final Map<String, DataSourceParameter> oldDataSourceParameters, final Entry<String, DataSourceParameter> target) {
-        return oldDataSourceParameters.containsKey(target.getKey()) && !oldDataSourceParameters.get(target.getKey()).equals(target.getValue());
+    private synchronized boolean isModifiedDataSource(final Map<String, DataSource> oldDataSources, final Entry<String, DataSourceParameter> target) {
+        return oldDataSources.containsKey(target.getKey()) && !DataSourceConverter.getDataSourceParameter(oldDataSources.get(target.getKey())).equals(target.getValue());
     }
     
     @Override
@@ -78,6 +77,14 @@ public final class ProxyOrchestrationSchemaContexts extends OrchestrationSchemaC
         return result;
     }
     
+    private Map<String, DataSource> createDataSources(final Map<String, DataSourceParameter> parameters) throws Exception {
+        Map<String, DataSource> result = new LinkedHashMap<>();
+        for (Entry<String, DataSourceParameter> entry: parameters.entrySet()) {
+            result.put(entry.getKey(), backendDataSourceFactory.build(entry.getKey(), entry.getValue()));
+        }
+        return result;
+    }
+    
     @Override
     public Map<String, Map<String, DataSourceParameter>> createDataSourceParametersMap(final Map<String, Map<String, DataSourceConfiguration>> dataSourcesMap) {
         Map<String, Map<String, DataSourceParameter>> result = new LinkedHashMap<>();
@@ -86,12 +93,4 @@ public final class ProxyOrchestrationSchemaContexts extends OrchestrationSchemaC
         }
         return result;
     }
-    
-    private Map<String, DataSource> createDataSources(final Map<String, DataSourceParameter> parameters) throws Exception {
-        Map<String, DataSource> result = new LinkedHashMap<>();
-        for (Entry<String, DataSourceParameter> entry: parameters.entrySet()) {
-            result.put(entry.getKey(), backendDataSourceFactory.build(entry.getKey(), entry.getValue()));
-        }
-        return result;
-    }
 }