You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ki...@apache.org on 2020/08/25 11:58:32 UTC

[shardingsphere] branch master updated: Ass JDBCOrchestrationSchemaContexts (#7068)

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

kimmking 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 45271bf  Ass JDBCOrchestrationSchemaContexts (#7068)
45271bf is described below

commit 45271bf05fe9584ccb71344b1c273cc234935002
Author: Juan Pan(Trista) <pa...@apache.org>
AuthorDate: Tue Aug 25 19:57:43 2020 +0800

    Ass JDBCOrchestrationSchemaContexts (#7068)
---
 .../core/schema/OrchestrationSchemaContexts.java   |  22 ++---
 .../fixture/TestOrchestrationSchemaContexts.java   |   3 +-
 .../shardingsphere-jdbc-orchestration/pom.xml      |   5 +
 .../schema/JDBCOrchestrationSchemaContexts.java    |  71 ++++++++++++++
 .../JDBCOrchestrationSchemaContextsTest.java       | 102 +++++++++++++++++++++
 .../schema/ProxyOrchestrationSchemaContexts.java   |   3 +-
 6 files changed, 188 insertions(+), 18 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 b4cba98..cfce01e 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
@@ -39,7 +39,6 @@ import org.apache.shardingsphere.kernel.context.SchemaContexts;
 import org.apache.shardingsphere.kernel.context.SchemaContextsBuilder;
 import org.apache.shardingsphere.kernel.context.StandardSchemaContexts;
 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.metrics.configuration.config.MetricsConfiguration;
 import org.apache.shardingsphere.metrics.facade.MetricsTrackerManagerFacade;
@@ -59,6 +58,7 @@ import org.apache.shardingsphere.orchestration.core.registry.event.DisabledState
 import org.apache.shardingsphere.orchestration.core.registry.schema.OrchestrationSchema;
 
 import javax.sql.DataSource;
+import java.sql.Connection;
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.Collections;
@@ -113,11 +113,14 @@ public abstract class OrchestrationSchemaContexts implements SchemaContexts {
         return schemaContexts.getDatabaseType();
     }
     
-    private DatabaseType getDatabaseType(final Map<String, Map<String, DataSourceParameter>> dataSourceParametersMap) {
-        if (dataSourceParametersMap.isEmpty() || dataSourceParametersMap.values().iterator().next().isEmpty()) {
+    private DatabaseType getDatabaseType(final Map<String, Map<String, DataSource>> dataSourcesMap) throws SQLException {
+        if (dataSourcesMap.isEmpty() || dataSourcesMap.values().iterator().next().isEmpty()) {
             return schemaContexts.getDatabaseType();
         }
-        return DatabaseTypes.getDatabaseTypeByURL(dataSourceParametersMap.values().iterator().next().values().iterator().next().getUrl());
+        DataSource dataSource = dataSourcesMap.values().iterator().next().values().iterator().next();
+        try (Connection connection = dataSource.getConnection()) {
+            return DatabaseTypes.getDatabaseTypeByURL(connection.getMetaData().getURL());
+        }
     }
     
     @Override
@@ -335,8 +338,7 @@ public abstract class OrchestrationSchemaContexts implements SchemaContexts {
     private SchemaContext getAddedSchemaContext(final SchemaAddedEvent schemaAddedEvent) throws Exception {
         String schemaName = schemaAddedEvent.getShardingSchemaName();
         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);
+        DatabaseType databaseType = getDatabaseType(dataSourcesMap);
         SchemaContextsBuilder schemaContextsBuilder = new SchemaContextsBuilder(databaseType, dataSourcesMap,  
                 Collections.singletonMap(schemaName, schemaAddedEvent.getRuleConfigurations()), schemaContexts.getAuthentication(), schemaContexts.getProps().getProps());
         return schemaContextsBuilder.build().getSchemaContexts().get(schemaName);
@@ -403,12 +405,4 @@ public abstract class OrchestrationSchemaContexts implements SchemaContexts {
     protected abstract Map<String, DataSource> getModifiedDataSources(SchemaContext oldSchemaContext, Map<String, DataSourceConfiguration> newDataSources) throws Exception;
     
     protected abstract Map<String, Map<String, DataSource>> createDataSourcesMap(Map<String, Map<String, DataSourceConfiguration>> dataSourcesMap) throws Exception;
-    
-    /**
-     * Create data source parameters map.
-     * 
-     * @param dataSourcesMap data source map
-     * @return data source parameters map
-     */
-    public abstract Map<String, Map<String, DataSourceParameter>> createDataSourceParametersMap(Map<String, Map<String, DataSourceConfiguration>> dataSourcesMap);
 }
diff --git a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-schema/src/test/java/org/apache/shardingsphere/orchestration/core/schema/fixture/TestOrchestrationSchemaContexts.java b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-schema/src/test/java/org/apache/shardingsphere/orchestration/core/schema/fixture/TestOrchestrationSchemaCont [...]
index 91eed66..3e8bf98 100644
--- a/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-schema/src/test/java/org/apache/shardingsphere/orchestration/core/schema/fixture/TestOrchestrationSchemaContexts.java
+++ b/shardingsphere-control-panel/shardingsphere-orchestration/shardingsphere-orchestration-core/shardingsphere-orchestration-core-schema/src/test/java/org/apache/shardingsphere/orchestration/core/schema/fixture/TestOrchestrationSchemaContexts.java
@@ -74,8 +74,7 @@ public final class TestOrchestrationSchemaContexts extends OrchestrationSchemaCo
         return result;
     }
     
-    @Override
-    public Map<String, Map<String, DataSourceParameter>> createDataSourceParametersMap(final Map<String, Map<String, DataSourceConfiguration>> dataSourcesMap) {
+    private Map<String, Map<String, DataSourceParameter>> createDataSourceParametersMap(final Map<String, Map<String, DataSourceConfiguration>> dataSourcesMap) {
         Map<String, Map<String, DataSourceParameter>> result = new LinkedHashMap<>();
         for (Map.Entry<String, Map<String, DataSourceConfiguration>> entry : dataSourcesMap.entrySet()) {
             result.put(entry.getKey(), getDataSourceParameterMap(entry.getValue()));
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/pom.xml b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/pom.xml
index b583a73..523b702 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/pom.xml
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/pom.xml
@@ -44,6 +44,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-orchestration-core-schema</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
             <artifactId>shardingsphere-orchestration-repository-api</artifactId>
             <version>${project.version}</version>
         </dependency>
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/main/java/org/apache/shardingsphere/driver/orchestration/internal/schema/JDBCOrchestrationSchemaContexts.java b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/main/java/org/apache/shardingsphere/driver/orchestration/internal/schema/JDBCOrchestrationSchemaContexts.java
new file mode 100644
index 0000000..11578f0
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/main/java/org/apache/shardingsphere/driver/orchestration/internal/schema/JDBCOrchestrationSchemaContexts.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.driver.orchestration.internal.schema;
+
+import com.google.common.collect.Maps;
+import org.apache.commons.collections4.map.LinkedMap;
+import org.apache.shardingsphere.driver.orchestration.internal.util.DataSourceConverter;
+import org.apache.shardingsphere.infra.config.DataSourceConfiguration;
+import org.apache.shardingsphere.infra.database.DefaultSchema;
+import org.apache.shardingsphere.kernel.context.SchemaContext;
+import org.apache.shardingsphere.kernel.context.SchemaContexts;
+import org.apache.shardingsphere.orchestration.core.facade.OrchestrationFacade;
+import org.apache.shardingsphere.orchestration.core.schema.OrchestrationSchemaContexts;
+
+import javax.sql.DataSource;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+
+/**
+ * JDBC orchestration schemaContexts.
+ */
+public final class JDBCOrchestrationSchemaContexts extends OrchestrationSchemaContexts {
+    
+    protected JDBCOrchestrationSchemaContexts(final SchemaContexts schemaContexts, final OrchestrationFacade orchestrationFacade) {
+        super(schemaContexts, orchestrationFacade);
+    }
+    
+    @Override
+    protected Map<String, DataSource> getAddedDataSources(final SchemaContext oldSchemaContext, final Map<String, DataSourceConfiguration> newDataSources) throws Exception {
+        Map<String, DataSourceConfiguration> newDataSourceConfigs = Maps.filterKeys(newDataSources, each -> !oldSchemaContext.getSchema().getDataSources().containsKey(each));
+        return DataSourceConverter.getDataSourceMap(newDataSourceConfigs);
+    }
+    
+    @Override
+    protected Map<String, DataSource> getModifiedDataSources(final SchemaContext oldSchemaContext, final Map<String, DataSourceConfiguration> newDataSources) throws Exception {
+        Map<String, DataSourceConfiguration> modifiedDataSourceConfigs =
+                newDataSources.entrySet().stream().filter(this::isModifiedDataSource).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue, (key, repeatKey) -> key, LinkedHashMap::new));
+        return DataSourceConverter.getDataSourceMap(modifiedDataSourceConfigs);
+    }
+    
+    private synchronized boolean isModifiedDataSource(final Entry<String, DataSourceConfiguration> dataSourceConfigs) {
+        Map<String, DataSourceConfiguration> oldDataSourceConfigs = DataSourceConverter.getDataSourceConfigurationMap(getSchemaContexts().get(DefaultSchema.LOGIC_NAME).getSchema().getDataSources());
+        return oldDataSourceConfigs.containsKey(dataSourceConfigs.getKey()) && !oldDataSourceConfigs.get(dataSourceConfigs.getKey()).equals(dataSourceConfigs.getValue());
+    }
+    
+    @Override
+    protected Map<String, Map<String, DataSource>> createDataSourcesMap(final Map<String, Map<String, DataSourceConfiguration>> dataSourcesMap) throws Exception {
+        Map<String, Map<String, DataSource>> result = new LinkedMap<>(dataSourcesMap.size());
+        for (Entry<String, Map<String, DataSourceConfiguration>> entry : dataSourcesMap.entrySet()) {
+            result.put(entry.getKey(), DataSourceConverter.getDataSourceMap(entry.getValue()));
+        }
+        return result;
+    }
+}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/java/org/apache/shardingsphere/driver/orchestration/internal/schema/JDBCOrchestrationSchemaContextsTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/java/org/apache/shardingsphere/driver/orchestration/internal/schema/JDBCOrchestrationSchemaContextsTest.java
new file mode 100644
index 0000000..3adb45e
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/java/org/apache/shardingsphere/driver/orchestration/internal/schema/JDBCOrchestrationSchemaContextsTest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.driver.orchestration.internal.schema;
+
+import org.apache.commons.collections4.map.LinkedMap;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.shardingsphere.driver.orchestration.internal.util.DataSourceConverter;
+import org.apache.shardingsphere.infra.auth.Authentication;
+import org.apache.shardingsphere.infra.config.DataSourceConfiguration;
+import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
+import org.apache.shardingsphere.infra.database.DefaultSchema;
+import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
+import org.apache.shardingsphere.kernel.context.SchemaContext;
+import org.apache.shardingsphere.kernel.context.StandardSchemaContexts;
+import org.apache.shardingsphere.kernel.context.runtime.RuntimeContext;
+import org.apache.shardingsphere.kernel.context.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.orchestration.core.common.event.DataSourceChangedEvent;
+import org.apache.shardingsphere.orchestration.core.facade.OrchestrationFacade;
+import org.apache.shardingsphere.orchestration.core.registry.RegistryCenter;
+import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import javax.sql.DataSource;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class JDBCOrchestrationSchemaContextsTest {
+    
+    @Mock
+    private RegistryCenter registryCenter;
+    
+    @Mock
+    private OrchestrationFacade facade;
+    
+    private JDBCOrchestrationSchemaContexts schemaContexts;
+    
+    @Before
+    public void setUp() {
+        when(registryCenter.loadDisabledDataSources()).thenReturn(Collections.emptyList());
+        when(facade.getRegistryCenter()).thenReturn(registryCenter);
+        schemaContexts = new JDBCOrchestrationSchemaContexts(new StandardSchemaContexts(new LinkedMap<>(), new Authentication(),
+                new ConfigurationProperties(new Properties()), new H2DatabaseType()), facade);
+    }
+    
+    private Map<String, SchemaContext> getSchemaContextMap() {
+        ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
+        when(schema.getDataSources()).thenReturn(getDataSources());
+        RuntimeContext runtimeContext = mock(RuntimeContext.class);
+        when(runtimeContext.getTransactionManagerEngine()).thenReturn(new ShardingTransactionManagerEngine());
+        SchemaContext result = new SchemaContext(DefaultSchema.LOGIC_NAME, schema, runtimeContext);
+        return Collections.singletonMap(DefaultSchema.LOGIC_NAME, result);
+    }
+    
+    private Map<String, DataSource> getDataSources() {
+        BasicDataSource result = new BasicDataSource();
+        result.setDriverClassName("org.h2.Driver");
+        result.setUrl("jdbc:h2:mem:test;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MySQL");
+        result.setUsername("sa");
+        result.setPassword("");
+        return Collections.singletonMap("db0", result);
+    }
+    
+    @Test
+    public void assertRenewDataSourceChangedEvent() throws Exception {
+        schemaContexts.getSchemaContexts().putAll(getSchemaContextMap());
+        DataSourceChangedEvent event = new DataSourceChangedEvent(DefaultSchema.LOGIC_NAME, getDataSourceConfigurations());
+        schemaContexts.renew(event);
+    }
+    
+    private Map<String, DataSourceConfiguration> getDataSourceConfigurations() {
+        BasicDataSource result = new BasicDataSource();
+        result.setDriverClassName("org.h2.Driver");
+        result.setUrl("jdbc:h2:mem:test1;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MySQL");
+        result.setUsername("sa");
+        result.setPassword("");
+        return DataSourceConverter.getDataSourceConfigurationMap(Collections.singletonMap("db0", 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 a2f5010..a4a9276 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
@@ -85,8 +85,7 @@ public final class ProxyOrchestrationSchemaContexts extends OrchestrationSchemaC
         return result;
     }
     
-    @Override
-    public Map<String, Map<String, DataSourceParameter>> createDataSourceParametersMap(final Map<String, Map<String, DataSourceConfiguration>> dataSources) {
+    private Map<String, Map<String, DataSourceParameter>> createDataSourceParametersMap(final Map<String, Map<String, DataSourceConfiguration>> dataSources) {
         Map<String, Map<String, DataSourceParameter>> result = new LinkedHashMap<>(dataSources.size(), 1);
         for (Entry<String, Map<String, DataSourceConfiguration>> entry : dataSources.entrySet()) {
             result.put(entry.getKey(), DataSourceConverter.getDataSourceParameterMap(entry.getValue()));