You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2022/09/05 04:29:08 UTC

[shardingsphere] branch 5.2.0-release updated: Fix add miss original data source when add new resource. (#20747)

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

duanzhengqiang pushed a commit to branch 5.2.0-release
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/5.2.0-release by this push:
     new 8dee990098f Fix add miss original data source when add new resource. (#20747)
8dee990098f is described below

commit 8dee990098fd2cb000079ce1a933c530e8d0a704
Author: zhaojinchao <zh...@apache.org>
AuthorDate: Mon Sep 5 12:29:02 2022 +0800

    Fix add miss original data source when add new resource. (#20747)
    
    * Refactor ResourceSwitchManager.create() and ContextManager.dropResources()
    
    * Fix unit test
    
    * Fix unit test
---
 .../mode/manager/ContextManager.java               | 36 ++++++++++++++++------
 .../manager/switcher/ResourceSwitchManager.java    | 17 +++++++++-
 .../mode/manager/switcher/SwitchingResource.java   |  1 +
 .../mode/manager/ContextManagerTest.java           | 23 ++++++++++----
 .../switcher/ResourceSwitchManagerTest.java        |  2 --
 .../ClusterContextManagerCoordinatorTest.java      |  7 +++--
 6 files changed, 64 insertions(+), 22 deletions(-)

diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
index b63102521f1..c59384d47ac 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
@@ -243,21 +243,36 @@ public final class ContextManager implements AutoCloseable {
      */
     public synchronized void dropResources(final String databaseName, final Collection<String> toBeDroppedResourceNames) throws SQLException {
         // TODO should check to be dropped resources are unused here. ContextManager is atomic domain to maintain metadata, not Dist SQL handler
-        Map<String, DataSourceProperties> toBeReservedDataSourcePropsMap = getToBeReservedDataSourcePropsMap(databaseName, toBeDroppedResourceNames);
-        SwitchingResource switchingResource = new ResourceSwitchManager().create(metaDataContexts.getMetaData().getDatabase(databaseName).getResource(), toBeReservedDataSourcePropsMap);
+        Map<String, DataSourceProperties> dataSourcePropsMap = metaDataContexts.getPersistService().getDataSourceService().load(metaDataContexts.getMetaData().getActualDatabaseName(databaseName));
+        Map<String, DataSourceProperties> toBeDeletedDataSourcePropsMap = getToBeDeletedDataSourcePropsMap(dataSourcePropsMap, toBeDroppedResourceNames);
+        SwitchingResource switchingResource = new ResourceSwitchManager().createByDropResource(metaDataContexts.getMetaData().getDatabase(databaseName).getResource(), toBeDeletedDataSourcePropsMap);
         Map<String, ShardingSphereDatabase> reloadDatabases = createChangedDatabases(databaseName, switchingResource, null);
         deleteTableMetaData(metaDataContexts.getMetaData().getDatabase(databaseName), reloadDatabases.get(databaseName.toLowerCase()));
         metaDataContexts.getMetaData().getDatabases().putAll(reloadDatabases);
-        metaDataContexts.getPersistService().getDataSourceService().persist(metaDataContexts.getMetaData().getActualDatabaseName(databaseName), toBeReservedDataSourcePropsMap);
-        toBeDroppedResourceNames.forEach(each -> metaDataContexts.getMetaData().getDatabase(databaseName).getResource().getDataSources().remove(each));
+        metaDataContexts.getMetaData().getDatabases().putAll(renewDatabase(metaDataContexts.getMetaData().getDatabase(databaseName), switchingResource));
+        Map<String, DataSourceProperties> toBeReversedDataSourcePropsMap = getToBeReversedDataSourcePropsMap(dataSourcePropsMap, toBeDroppedResourceNames);
+        metaDataContexts.getPersistService().getDataSourceService().persist(metaDataContexts.getMetaData().getActualDatabaseName(databaseName), toBeReversedDataSourcePropsMap);
         switchingResource.closeStaleDataSources();
     }
     
-    private Map<String, DataSourceProperties> getToBeReservedDataSourcePropsMap(final String databaseName, final Collection<String> toBeDroppedResourceNames) {
-        Map<String, DataSourceProperties> dataSourcePropsMap = metaDataContexts.getPersistService().getDataSourceService().load(metaDataContexts.getMetaData().getActualDatabaseName(databaseName));
+    private Map<String, DataSourceProperties> getToBeReversedDataSourcePropsMap(final Map<String, DataSourceProperties> dataSourcePropsMap, final Collection<String> toBeDroppedResourceNames) {
         return dataSourcePropsMap.entrySet().stream().filter(entry -> !toBeDroppedResourceNames.contains(entry.getKey())).collect(Collectors.toMap(Entry::getKey, Entry::getValue));
     }
     
+    private Map<String, DataSourceProperties> getToBeDeletedDataSourcePropsMap(final Map<String, DataSourceProperties> dataSourcePropsMap, final Collection<String> toBeDroppedResourceNames) {
+        return dataSourcePropsMap.entrySet().stream().filter(entry -> toBeDroppedResourceNames.contains(entry.getKey())).collect(Collectors.toMap(Entry::getKey, Entry::getValue));
+    }
+    
+    private Map<String, ShardingSphereDatabase> renewDatabase(final ShardingSphereDatabase database, final SwitchingResource resource) {
+        Map<String, ShardingSphereDatabase> result = new LinkedHashMap<>(1, 1);
+        Map<String, DataSource> newDataSource =
+                database.getResource().getDataSources().entrySet().stream().filter(entry -> !resource.getStaleDataSources().containsKey(entry.getKey()))
+                        .collect(Collectors.toMap(Entry::getKey, Entry::getValue));
+        result.put(database.getName().toLowerCase(),
+                new ShardingSphereDatabase(database.getName(), database.getProtocolType(), new ShardingSphereResource(newDataSource), database.getRuleMetaData(), database.getSchemas()));
+        return result;
+    }
+    
     /**
      * Alter rule configuration.
      *
@@ -335,13 +350,14 @@ public final class ContextManager implements AutoCloseable {
     
     private Map<String, ShardingSphereDatabase> createChangedDatabases(final String databaseName,
                                                                        final SwitchingResource switchingResource, final Collection<RuleConfiguration> ruleConfigs) throws SQLException {
-        Map<String, DataSource> toBeCreatedDataSources = null == switchingResource
-                ? metaDataContexts.getMetaData().getDatabase(databaseName).getResource().getDataSources()
-                : switchingResource.getNewDataSources();
+        if (null != switchingResource && !switchingResource.getNewDataSources().isEmpty()) {
+            metaDataContexts.getMetaData().getDatabase(databaseName).getResource().getDataSources().putAll(switchingResource.getNewDataSources());
+        }
         Collection<RuleConfiguration> toBeCreatedRuleConfigs = null == ruleConfigs
                 ? metaDataContexts.getMetaData().getDatabase(databaseName).getRuleMetaData().getConfigurations()
                 : ruleConfigs;
-        DatabaseConfiguration toBeCreatedDatabaseConfig = new DataSourceProvidedDatabaseConfiguration(toBeCreatedDataSources, toBeCreatedRuleConfigs);
+        DatabaseConfiguration toBeCreatedDatabaseConfig =
+                new DataSourceProvidedDatabaseConfiguration(metaDataContexts.getMetaData().getDatabase(databaseName).getResource().getDataSources(), toBeCreatedRuleConfigs);
         ShardingSphereDatabase changedDatabase = ShardingSphereDatabasesFactory.create(metaDataContexts.getMetaData().getActualDatabaseName(databaseName),
                 toBeCreatedDatabaseConfig, metaDataContexts.getMetaData().getProps(), instanceContext);
         Map<String, ShardingSphereDatabase> result = new LinkedHashMap<>(metaDataContexts.getMetaData().getDatabases());
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/switcher/ResourceSwitchManager.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/switcher/ResourceSwitchManager.java
index 92e7aa8ea96..07a0c8087a8 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/switcher/ResourceSwitchManager.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/switcher/ResourceSwitchManager.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.datasource.props.DataSourcePropertiesCrea
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 
 import javax.sql.DataSource;
+import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -44,6 +45,17 @@ public final class ResourceSwitchManager {
         return new SwitchingResource(resource, createNewDataSources(resource, toBeChangedDataSourceProps), getStaleDataSources(resource, toBeChangedDataSourceProps));
     }
     
+    /**
+     * Create switching resource.
+     *
+     * @param resource resource
+     * @param toBeDeletedDataSourceProps to be deleted data source properties map
+     * @return created switching resource
+     */
+    public SwitchingResource createByDropResource(final ShardingSphereResource resource, final Map<String, DataSourceProperties> toBeDeletedDataSourceProps) {
+        return new SwitchingResource(resource, Collections.emptyMap(), getStaleDataSources(resource.getDataSources(), toBeDeletedDataSourceProps));
+    }
+    
     private Map<String, DataSource> createNewDataSources(final ShardingSphereResource resource, final Map<String, DataSourceProperties> toBeChangedDataSourceProps) {
         Map<String, DataSource> result = new LinkedHashMap<>(resource.getDataSources());
         result.keySet().removeAll(getToBeDeletedDataSources(resource, toBeChangedDataSourceProps).keySet());
@@ -73,11 +85,14 @@ public final class ResourceSwitchManager {
     
     private Map<String, DataSource> getStaleDataSources(final ShardingSphereResource resource, final Map<String, DataSourceProperties> toBeChangedDataSourceProps) {
         Map<String, DataSource> result = new LinkedHashMap<>(resource.getDataSources().size(), 1);
-        result.putAll(getToBeDeletedDataSources(resource, toBeChangedDataSourceProps));
         result.putAll(getToBeChangedDataSources(resource, toBeChangedDataSourceProps));
         return result;
     }
     
+    private Map<String, DataSource> getStaleDataSources(final Map<String, DataSource> dataSources, final Map<String, DataSourceProperties> toBeDeletedDataSourceProps) {
+        return dataSources.entrySet().stream().filter(entry -> toBeDeletedDataSourceProps.containsKey(entry.getKey())).collect(Collectors.toMap(Entry::getKey, Entry::getValue));
+    }
+    
     private Map<String, DataSource> getToBeDeletedDataSources(final ShardingSphereResource resource, final Map<String, DataSourceProperties> toBeChangedDataSourceProps) {
         return resource.getDataSources().entrySet().stream().filter(entry -> !toBeChangedDataSourceProps.containsKey(entry.getKey())).collect(Collectors.toMap(Entry::getKey, Entry::getValue));
     }
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/switcher/SwitchingResource.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/switcher/SwitchingResource.java
index 93e585ee46a..5d4d9bea68f 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/switcher/SwitchingResource.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/switcher/SwitchingResource.java
@@ -35,6 +35,7 @@ public final class SwitchingResource {
     @Getter
     private final Map<String, DataSource> newDataSources;
     
+    @Getter
     private final Map<String, DataSource> staleDataSources;
     
     /**
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/ContextManagerTest.java b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/ContextManagerTest.java
index a26b016e2d9..22710a6f5ec 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/ContextManagerTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/ContextManagerTest.java
@@ -45,6 +45,7 @@ import org.junit.Test;
 import javax.sql.DataSource;
 import java.sql.SQLException;
 import java.sql.Types;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
@@ -205,12 +206,20 @@ public final class ContextManagerTest {
     
     @Test
     public void assertDropResources() throws SQLException {
+        ShardingSphereDatabase database = new ShardingSphereDatabase(
+                "foo_db", new MySQLDatabaseType(), createOriginalResource(), createOriginalRuleMetaData(), Collections.emptyMap());
+        when(metaDataContexts.getMetaData().getDatabase("foo_db")).thenReturn(database);
+        Map<String, ShardingSphereDatabase> databases = new LinkedHashMap<>(1, 1);
+        databases.put("foo_db", database);
+        when(metaDataContexts.getMetaData().getDatabases()).thenReturn(databases);
         when(metaDataContexts.getMetaData().getActualDatabaseName("foo_db")).thenReturn("foo_db");
-        when(metaDataContexts.getMetaData().getDatabase("foo_db").getResource().getDataSources()).thenReturn(new HashMap<>(Collections.singletonMap("foo_ds", new MockedDataSource())));
         when(metaDataContexts.getPersistService()).thenReturn(mock(MetaDataPersistService.class, RETURNS_DEEP_STUBS));
-        when(metaDataContexts.getPersistService().getDataSourceService().load("foo_db")).thenReturn(Collections.singletonMap("foo_ds", mock(DataSourceProperties.class)));
-        contextManager.dropResources("foo_db", Collections.singleton("foo_ds"));
-        assertTrue(contextManager.getMetaDataContexts().getMetaData().getDatabase("foo_db").getResource().getDataSources().isEmpty());
+        Map<String, DataSourceProperties> dataSourcePropertiesMap = new LinkedHashMap<>(1, 1);
+        dataSourcePropertiesMap.put("ds_1", mock(DataSourceProperties.class));
+        dataSourcePropertiesMap.put("ds_2", mock(DataSourceProperties.class));
+        when(metaDataContexts.getPersistService().getDataSourceService().load("foo_db")).thenReturn(dataSourcePropertiesMap);
+        contextManager.dropResources("foo_db", Arrays.asList("ds_1", "ds_2"));
+        assertTrue(contextManager.getMetaDataContexts().getMetaData().getDatabases().get("foo_db").getResource().getDataSources().isEmpty());
     }
     
     @Test
@@ -237,7 +246,7 @@ public final class ContextManagerTest {
         when(metaDataContexts.getMetaData().getActualDatabaseName("foo_db")).thenReturn("foo_db");
         when(metaDataContexts.getMetaData().getGlobalRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.emptyList()));
         contextManager.alterDataSourceConfiguration("foo_db", Collections.singletonMap("foo_ds", new DataSourceProperties(MockedDataSource.class.getName(), createProperties("test", "test"))));
-        assertThat(contextManager.getMetaDataContexts().getMetaData().getDatabase("foo_db").getResource().getDataSources().size(), is(1));
+        assertThat(contextManager.getMetaDataContexts().getMetaData().getDatabase("foo_db").getResource().getDataSources().size(), is(3));
         assertAlteredDataSource((MockedDataSource) contextManager.getMetaDataContexts().getMetaData().getDatabase("foo_db").getResource().getDataSources().get("foo_ds"));
     }
     
@@ -273,7 +282,9 @@ public final class ContextManagerTest {
     
     @Test
     public void assertReloadDatabase() {
-        when(metaDataContexts.getMetaData().getDatabase("foo_db").getResource().getDataSources()).thenReturn(Collections.singletonMap("foo_ds", new MockedDataSource()));
+        Map<String, DataSource> dataSourceMap = new LinkedHashMap<>(1, 1);
+        dataSourceMap.put("foo_ds", new MockedDataSource());
+        when(metaDataContexts.getMetaData().getDatabase("foo_db").getResource().getDataSources()).thenReturn(dataSourceMap);
         when(metaDataContexts.getMetaData().getActualDatabaseName("foo_db")).thenReturn("foo_db");
         DatabaseMetaDataPersistService databaseMetaDataPersistService = mock(DatabaseMetaDataPersistService.class, RETURNS_DEEP_STUBS);
         MetaDataPersistService persistService = mock(MetaDataPersistService.class);
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/switcher/ResourceSwitchManagerTest.java b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/switcher/ResourceSwitchManagerTest.java
index e4a4c3c6df4..21edc8dc02f 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/switcher/ResourceSwitchManagerTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/switcher/ResourceSwitchManagerTest.java
@@ -47,7 +47,6 @@ public final class ResourceSwitchManagerTest {
         Map<String, DataSource> result = new HashMap<>(3, 1);
         result.put("not_change", new MockedDataSource());
         result.put("replace", new MockedDataSource());
-        result.put("delete", new MockedDataSource());
         return result;
     }
     
@@ -67,7 +66,6 @@ public final class ResourceSwitchManagerTest {
     }
     
     private void assertStaleDataSources(final Map<String, DataSource> originalDataSourceMap) throws InterruptedException {
-        assertStaleDataSource((MockedDataSource) originalDataSourceMap.get("delete"));
         assertStaleDataSource((MockedDataSource) originalDataSourceMap.get("replace"));
         assertNotStaleDataSource((MockedDataSource) originalDataSourceMap.get("not_change"));
     }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
index c52ad1607fb..1828b1b6e19 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
@@ -92,7 +92,6 @@ import javax.sql.DataSource;
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.Map;
@@ -146,14 +145,16 @@ public final class ClusterContextManagerCoordinatorTest {
     
     private Map<String, ShardingSphereDatabase> createDatabases() {
         when(database.getName()).thenReturn("db");
-        when(database.getResource().getDataSources()).thenReturn(Collections.emptyMap());
+        when(database.getResource().getDataSources()).thenReturn(new LinkedHashMap<>());
         when(database.getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
         when(database.getProtocolType()).thenReturn(new MySQLDatabaseType());
         when(database.getSchema("foo_schema")).thenReturn(mock(ShardingSphereSchema.class));
         when(database.getRuleMetaData().getRules()).thenReturn(new LinkedList<>());
         when(database.getRuleMetaData().getConfigurations()).thenReturn(Collections.emptyList());
         when(database.getRuleMetaData().findRules(ResourceHeldRule.class)).thenReturn(Collections.emptyList());
-        return new HashMap<>(Collections.singletonMap("db", database));
+        Map<String, ShardingSphereDatabase> result = new LinkedHashMap<>(1, 1);
+        result.put("db", database);
+        return result;
     }
     
     @Test