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 2023/04/28 08:48:55 UTC

[shardingsphere] branch master updated: Revert `DataSourceContainedRule#getDataSourceMapper` return role info (#25388)

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

duanzhengqiang 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 61e891af380 Revert `DataSourceContainedRule#getDataSourceMapper` return role info (#25388)
61e891af380 is described below

commit 61e891af3808acee04bab7d08f5d1b1f6a28f2d3
Author: ZhangCheng <ch...@apache.org>
AuthorDate: Fri Apr 28 16:48:48 2023 +0800

    Revert `DataSourceContainedRule#getDataSourceMapper` return role info (#25388)
    
    * Revert "Modify ResourceContainedRule.getDataSourceMapper return role info (#24609)"
    
    * Revert "Modify ResourceContainedRule.getDataSourceMapper return role info (#24609)"
    
    ---------
    
    Co-authored-by: Zhengqiang Duan <du...@apache.org>
---
 .../group/ReadwriteSplittingGroup.java             |  4 +-
 .../group/type/StaticReadwriteSplittingGroup.java  | 12 ++----
 .../rule/ReadwriteSplittingRule.java               |  5 +--
 .../rule/ReadwriteSplittingRuleTest.java           |  7 +---
 ...DropReadwriteSplittingRuleStatementUpdater.java |  9 ++---
 .../CountReadwriteSplittingRuleExecutorTest.java   |  8 +---
 ...ReadwriteSplittingRuleStatementUpdaterTest.java |  4 +-
 ...ReadwriteSplittingRuleStatementUpdaterTest.java |  8 ++--
 ...ReadwriteSplittingRuleStatementUpdaterTest.java |  6 +--
 .../shardingsphere/shadow/rule/ShadowRule.java     | 14 +++----
 .../distsql/query/CountShadowRuleExecutorTest.java |  8 ++--
 .../CreateShadowRuleStatementUpdaterTest.java      |  4 +-
 ...reateShardingTableRuleStatementUpdaterTest.java |  5 +--
 .../infra/datanode/DataNodeUtils.java              |  7 ++--
 .../infra/datasource/mapper/DataSourceRole.java    | 26 ------------
 .../datasource/mapper/DataSourceRoleInfo.java      | 35 ----------------
 .../identifier/type/DataSourceContainedRule.java   |  3 +-
 .../type/StaticDataSourceContainedRule.java        |  3 +-
 .../infra/datanode/DataNodeUtilsTest.java          | 10 +----
 .../infra/datanode/DataNodesTest.java              | 15 +++----
 .../shardingsphere/infra/fixture/FixtureRule.java  |  3 +-
 .../shardingsphere/single/rule/SingleRule.java     |  9 ++---
 .../subscriber/ConfigurationChangedSubscriber.java |  3 +-
 .../unit/UnregisterStorageUnitBackendHandler.java  |  9 +++--
 .../rql/storage/unit/ShowStorageUnitExecutor.java  |  9 +++--
 .../fixture/FixtureDataSourceContainedRule.java    | 47 ----------------------
 .../ImportDatabaseConfigurationUpdaterTest.java    |  4 +-
 .../UnregisterStorageUnitBackendHandlerTest.java   |  6 +--
 28 files changed, 66 insertions(+), 217 deletions(-)

diff --git a/features/readwrite-splitting/core/src/main/java/org/apache/shardingsphere/readwritesplitting/group/ReadwriteSplittingGroup.java b/features/readwrite-splitting/core/src/main/java/org/apache/shardingsphere/readwritesplitting/group/ReadwriteSplittingGroup.java
index 678cd8a4c0a..0714cc501ed 100644
--- a/features/readwrite-splitting/core/src/main/java/org/apache/shardingsphere/readwritesplitting/group/ReadwriteSplittingGroup.java
+++ b/features/readwrite-splitting/core/src/main/java/org/apache/shardingsphere/readwritesplitting/group/ReadwriteSplittingGroup.java
@@ -17,8 +17,6 @@
 
 package org.apache.shardingsphere.readwritesplitting.group;
 
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
-
 import java.util.Collection;
 import java.util.List;
 
@@ -45,5 +43,5 @@ public interface ReadwriteSplittingGroup {
      *
      * @return all data sources
      */
-    Collection<DataSourceRoleInfo> getAllDataSources();
+    Collection<String> getAllDataSources();
 }
diff --git a/features/readwrite-splitting/core/src/main/java/org/apache/shardingsphere/readwritesplitting/group/type/StaticReadwriteSplittingGroup.java b/features/readwrite-splitting/core/src/main/java/org/apache/shardingsphere/readwritesplitting/group/type/StaticReadwriteSplittingGroup.java
index b077c16084a..4d9665f1fc7 100644
--- a/features/readwrite-splitting/core/src/main/java/org/apache/shardingsphere/readwritesplitting/group/type/StaticReadwriteSplittingGroup.java
+++ b/features/readwrite-splitting/core/src/main/java/org/apache/shardingsphere/readwritesplitting/group/type/StaticReadwriteSplittingGroup.java
@@ -18,8 +18,6 @@
 package org.apache.shardingsphere.readwritesplitting.group.type;
 
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRole;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.readwritesplitting.group.ReadwriteSplittingGroup;
 
 import java.util.Collection;
@@ -47,12 +45,10 @@ public final class StaticReadwriteSplittingGroup implements ReadwriteSplittingGr
     }
     
     @Override
-    public Collection<DataSourceRoleInfo> getAllDataSources() {
-        Collection<DataSourceRoleInfo> result = new LinkedList<>();
-        result.add(new DataSourceRoleInfo(writeDataSourceName, DataSourceRole.PRIMARY));
-        readDataSourceNames.forEach(each -> {
-            result.add(new DataSourceRoleInfo(each, DataSourceRole.MEMBER));
-        });
+    public Collection<String> getAllDataSources() {
+        Collection<String> result = new LinkedList<>();
+        result.add(writeDataSourceName);
+        result.addAll(readDataSourceNames);
         return result;
     }
 }
diff --git a/features/readwrite-splitting/core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java b/features/readwrite-splitting/core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java
index b045d14eeb7..88170f2b6a8 100644
--- a/features/readwrite-splitting/core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java
+++ b/features/readwrite-splitting/core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java
@@ -21,7 +21,6 @@ import com.google.common.base.Preconditions;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.datasource.state.DataSourceState;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.metadata.database.schema.QualifiedDatabase;
@@ -148,8 +147,8 @@ public final class ReadwriteSplittingRule implements DatabaseRule, DataSourceCon
     }
     
     @Override
-    public Map<String, Collection<DataSourceRoleInfo>> getDataSourceMapper() {
-        Map<String, Collection<DataSourceRoleInfo>> result = new LinkedHashMap<>();
+    public Map<String, Collection<String>> getDataSourceMapper() {
+        Map<String, Collection<String>> result = new HashMap<>();
         for (Entry<String, ReadwriteSplittingDataSourceRule> entry : dataSourceRules.entrySet()) {
             result.put(entry.getValue().getName(), entry.getValue().getReadwriteSplittingGroup().getAllDataSources());
         }
diff --git a/features/readwrite-splitting/core/src/test/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRuleTest.java b/features/readwrite-splitting/core/src/test/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRuleTest.java
index 7375d130c87..901f8511c42 100644
--- a/features/readwrite-splitting/core/src/test/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRuleTest.java
+++ b/features/readwrite-splitting/core/src/test/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRuleTest.java
@@ -18,8 +18,6 @@
 package org.apache.shardingsphere.readwritesplitting.rule;
 
 import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRole;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.datasource.state.DataSourceState;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.metadata.database.schema.QualifiedDatabase;
@@ -100,9 +98,8 @@ class ReadwriteSplittingRuleTest {
     @Test
     void assertGetDataSourceMapper() {
         ReadwriteSplittingRule readwriteSplittingRule = createReadwriteSplittingRule();
-        Map<String, Collection<DataSourceRoleInfo>> actual = readwriteSplittingRule.getDataSourceMapper();
-        Map<String, Collection<DataSourceRoleInfo>> expected = Collections.singletonMap("readwrite", Arrays.asList(new DataSourceRoleInfo("write_ds", DataSourceRole.PRIMARY),
-                new DataSourceRoleInfo("read_ds_0", DataSourceRole.MEMBER), new DataSourceRoleInfo("read_ds_1", DataSourceRole.MEMBER)));
+        Map<String, Collection<String>> actual = readwriteSplittingRule.getDataSourceMapper();
+        Map<String, Collection<String>> expected = Collections.singletonMap("readwrite", Arrays.asList("write_ds", "read_ds_0", "read_ds_1"));
         assertThat(actual, is(expected));
     }
 }
diff --git a/features/readwrite-splitting/distsql/handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleStatementUpdater.java b/features/readwrite-splitting/distsql/handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleStatementUpdater.java
index 9c7edd2f5e2..cbe92673c2f 100644
--- a/features/readwrite-splitting/distsql/handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleStatementUpdater.java
+++ b/features/readwrite-splitting/distsql/handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleStatementUpdater.java
@@ -17,11 +17,10 @@
 
 package org.apache.shardingsphere.readwritesplitting.distsql.handler.update;
 
+import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.distsql.handler.exception.rule.MissingRequiredRuleException;
 import org.apache.shardingsphere.distsql.handler.exception.rule.RuleInUsedException;
 import org.apache.shardingsphere.distsql.handler.update.RuleDefinitionDropUpdater;
-import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
@@ -34,7 +33,6 @@ import org.apache.shardingsphere.readwritesplitting.rule.ReadwriteSplittingRule;
 
 import java.util.Collection;
 import java.util.HashSet;
-import java.util.LinkedHashSet;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.stream.Collectors;
@@ -76,8 +74,9 @@ public final class DropReadwriteSplittingRuleStatementUpdater implements RuleDef
             if (each instanceof ReadwriteSplittingRule) {
                 continue;
             }
-            result.addAll(each.getDataSourceMapper().values().stream().flatMap(Collection::stream)
-                    .map(DataSourceRoleInfo::getName).collect(Collectors.toCollection(LinkedHashSet::new)));
+            Collection<String> actualDataSources = new HashSet<>();
+            each.getDataSourceMapper().values().forEach(actualDataSources::addAll);
+            result.addAll(actualDataSources);
         }
         for (DataNodeContainedRule each : database.getRuleMetaData().findRules(DataNodeContainedRule.class)) {
             Collection<DataNode> actualDataNodes = new HashSet<>();
diff --git a/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/query/CountReadwriteSplittingRuleExecutorTest.java b/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/query/CountReadwriteSplittingRuleExecutorTest.java
index 4cd2373ca90..fa227362f4a 100644
--- a/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/query/CountReadwriteSplittingRuleExecutorTest.java
+++ b/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/query/CountReadwriteSplittingRuleExecutorTest.java
@@ -17,8 +17,6 @@
 
 package org.apache.shardingsphere.readwritesplitting.distsql.handler.query;
 
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRole;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -30,7 +28,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
-import java.util.Map;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
@@ -71,10 +68,7 @@ class CountReadwriteSplittingRuleExecutorTest {
     
     private ReadwriteSplittingRule mockReadwriteSplittingRule() {
         ReadwriteSplittingRule result = mock(ReadwriteSplittingRule.class);
-        Map<String, Collection<DataSourceRoleInfo>> dataSourceMapper = Collections.singletonMap("readwrite_splitting",
-                Arrays.asList(new DataSourceRoleInfo("write_ds", DataSourceRole.PRIMARY),
-                        new DataSourceRoleInfo("read_ds", DataSourceRole.MEMBER)));
-        when(result.getDataSourceMapper()).thenReturn(dataSourceMapper);
+        when(result.getDataSourceMapper()).thenReturn(Collections.singletonMap("readwrite_splitting", Arrays.asList("write_ds", "read_ds")));
         return result;
     }
 }
diff --git a/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/AlterReadwriteSplittingRuleStatementUpdaterTest.java b/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/AlterReadwriteSplittingRuleStatementUpdaterTest.java
index c0a586ef571..acdd3640fbc 100644
--- a/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/AlterReadwriteSplittingRuleStatementUpdaterTest.java
+++ b/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/AlterReadwriteSplittingRuleStatementUpdaterTest.java
@@ -21,8 +21,6 @@ import org.apache.shardingsphere.distsql.handler.exception.rule.InvalidRuleConfi
 import org.apache.shardingsphere.distsql.handler.exception.rule.MissingRequiredRuleException;
 import org.apache.shardingsphere.distsql.handler.exception.storageunit.MissingRequiredStorageUnitsException;
 import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRole;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResourceMetaData;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
@@ -86,7 +84,7 @@ class AlterReadwriteSplittingRuleStatementUpdaterTest {
     @Test
     void assertCheckSQLStatementWithoutExistedAutoAwareResources() {
         DataSourceContainedRule dataSourceContainedRule = mock(DataSourceContainedRule.class);
-        when(dataSourceContainedRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("ms_group", Collections.singleton(new DataSourceRoleInfo("ds_0", DataSourceRole.PRIMARY))));
+        when(dataSourceContainedRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("ms_group", Collections.singleton("ds_0")));
         when(database.getRuleMetaData().findRules(DataSourceContainedRule.class)).thenReturn(Collections.singleton(dataSourceContainedRule));
         ReadwriteSplittingRuleSegment ruleSegment = new ReadwriteSplittingRuleSegment("readwrite_ds", "ha_group", new AlgorithmSegment("TEST", new Properties()));
         assertThrows(MissingRequiredStorageUnitsException.class,
diff --git a/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/CreateReadwriteSplittingRuleStatementUpdaterTest.java b/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/CreateReadwriteSplittingRuleStatementUpdaterTest.java
index f6fee2e5577..aac2a9d86b7 100644
--- a/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/CreateReadwriteSplittingRuleStatementUpdaterTest.java
+++ b/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/CreateReadwriteSplittingRuleStatementUpdaterTest.java
@@ -21,8 +21,6 @@ import org.apache.shardingsphere.distsql.handler.exception.rule.DuplicateRuleExc
 import org.apache.shardingsphere.distsql.handler.exception.rule.InvalidRuleConfigurationException;
 import org.apache.shardingsphere.distsql.handler.exception.storageunit.MissingRequiredStorageUnitsException;
 import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRole;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResourceMetaData;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
@@ -95,7 +93,7 @@ class CreateReadwriteSplittingRuleStatementUpdaterTest {
     @Test
     void assertCheckSQLStatementWithDuplicateLogicResource() {
         DataSourceContainedRule dataSourceContainedRule = mock(DataSourceContainedRule.class);
-        when(dataSourceContainedRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("duplicate_ds", Collections.singleton(new DataSourceRoleInfo("ds_0", DataSourceRole.PRIMARY))));
+        when(dataSourceContainedRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("duplicate_ds", Collections.singleton("ds_0")));
         when(database.getRuleMetaData().findRules(DataSourceContainedRule.class)).thenReturn(Collections.singleton(dataSourceContainedRule));
         ReadwriteSplittingRuleSegment ruleSegment = new ReadwriteSplittingRuleSegment("duplicate_ds", "write_ds_0", Arrays.asList("read_ds_0", "read_ds_1"),
                 new AlgorithmSegment(null, new Properties()));
@@ -105,7 +103,7 @@ class CreateReadwriteSplittingRuleStatementUpdaterTest {
     @Test
     void assertCheckSQLStatementWithoutExistedAutoAwareResources() {
         DataSourceContainedRule dataSourceContainedRule = mock(DataSourceContainedRule.class);
-        when(dataSourceContainedRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("ms_group", Collections.singleton(new DataSourceRoleInfo("ds_0", DataSourceRole.PRIMARY))));
+        when(dataSourceContainedRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("ms_group", Collections.singleton("ds_0")));
         when(database.getRuleMetaData().findRules(DataSourceContainedRule.class)).thenReturn(Collections.singleton(dataSourceContainedRule));
         ReadwriteSplittingRuleSegment ruleSegment = new ReadwriteSplittingRuleSegment("dynamic_rule", "ha_group", new AlgorithmSegment("TEST", new Properties()));
         assertThrows(MissingRequiredStorageUnitsException.class, () -> updater.checkSQLStatement(database, createSQLStatement(false, ruleSegment), null));
@@ -151,7 +149,7 @@ class CreateReadwriteSplittingRuleStatementUpdaterTest {
     @Test
     void assertUpdateSuccess() {
         DataSourceContainedRule dataSourceContainedRule = mock(DataSourceContainedRule.class);
-        when(dataSourceContainedRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("ms_group", Collections.singleton(new DataSourceRoleInfo("ds_0", DataSourceRole.PRIMARY))));
+        when(dataSourceContainedRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("ms_group", Collections.singleton("ds_0")));
         when(database.getRuleMetaData().findRules(DataSourceContainedRule.class)).thenReturn(Collections.singleton(dataSourceContainedRule));
         
         when(TypedSPILoader.contains(ReadQueryLoadBalanceAlgorithm.class, "TEST")).thenReturn(true);
diff --git a/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleStatementUpdaterTest.java b/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleStatementUpdaterTest.java
index 15bd6518b3e..27b97b8551e 100644
--- a/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleStatementUpdaterTest.java
+++ b/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleStatementUpdaterTest.java
@@ -22,8 +22,6 @@ import org.apache.shardingsphere.distsql.handler.exception.rule.RuleDefinitionVi
 import org.apache.shardingsphere.distsql.handler.exception.rule.RuleInUsedException;
 import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
 import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRole;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
@@ -37,7 +35,6 @@ import org.mockito.Mock;
 import org.mockito.junit.jupiter.MockitoExtension;
 
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedList;
 import java.util.Map;
@@ -80,8 +77,7 @@ class DropReadwriteSplittingRuleStatementUpdaterTest {
     @Test
     void assertCheckSQLStatementWithInUsed() throws RuleDefinitionViolationException {
         DataSourceContainedRule dataSourceContainedRule = mock(DataSourceContainedRule.class);
-        Map<String, Collection<DataSourceRoleInfo>> dataSourceMapper = Collections.singletonMap("foo_ds", Collections.singleton(new DataSourceRoleInfo("readwrite_ds", DataSourceRole.PRIMARY)));
-        when(dataSourceContainedRule.getDataSourceMapper()).thenReturn(dataSourceMapper);
+        when(dataSourceContainedRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("foo_ds", Collections.singleton("readwrite_ds")));
         when(database.getRuleMetaData().findRules(DataSourceContainedRule.class)).thenReturn(Collections.singleton(dataSourceContainedRule));
         DataNodeContainedRule dataNodeContainedRule = mock(DataNodeContainedRule.class);
         when(dataNodeContainedRule.getAllDataNodes()).thenReturn(Collections.singletonMap("foo_ds", Collections.singleton(new DataNode("readwrite_ds.tbl"))));
diff --git a/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java b/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java
index a3013271f7f..3717a86e333 100644
--- a/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java
+++ b/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java
@@ -20,8 +20,6 @@ package org.apache.shardingsphere.shadow.rule;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRole;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
@@ -238,16 +236,16 @@ public final class ShadowRule implements DatabaseRule, DataSourceContainedRule {
     }
     
     @Override
-    public Map<String, Collection<DataSourceRoleInfo>> getDataSourceMapper() {
-        Map<String, Collection<DataSourceRoleInfo>> result = new LinkedHashMap<>();
+    public Map<String, Collection<String>> getDataSourceMapper() {
+        Map<String, Collection<String>> result = new LinkedHashMap<>();
         shadowDataSourceMappings.forEach((key, value) -> result.put(key, createShadowDataSources(value)));
         return result;
     }
     
-    private Collection<DataSourceRoleInfo> createShadowDataSources(final ShadowDataSourceRule shadowDataSourceRule) {
-        Collection<DataSourceRoleInfo> result = new LinkedList<>();
-        result.add(new DataSourceRoleInfo(shadowDataSourceRule.getProductionDataSource(), DataSourceRole.PRODUCTION));
-        result.add(new DataSourceRoleInfo(shadowDataSourceRule.getShadowDataSource(), DataSourceRole.SHADOW));
+    private Collection<String> createShadowDataSources(final ShadowDataSourceRule shadowDataSourceRule) {
+        Collection<String> result = new LinkedList<>();
+        result.add(shadowDataSourceRule.getProductionDataSource());
+        result.add(shadowDataSourceRule.getShadowDataSource());
         return result;
     }
     
diff --git a/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/query/CountShadowRuleExecutorTest.java b/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/query/CountShadowRuleExecutorTest.java
index 1ba4be5c2d5..3e935b258c3 100644
--- a/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/query/CountShadowRuleExecutorTest.java
+++ b/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/query/CountShadowRuleExecutorTest.java
@@ -17,8 +17,6 @@
 
 package org.apache.shardingsphere.shadow.distsql.query;
 
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRole;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -73,9 +71,9 @@ class CountShadowRuleExecutorTest {
     
     private ShadowRule mockShadowRule() {
         ShadowRule result = mock(ShadowRule.class);
-        Map<String, Collection<DataSourceRoleInfo>> dataSourceMapper = new LinkedHashMap<>();
-        dataSourceMapper.put("shadow-data-source-0", Arrays.asList(new DataSourceRoleInfo("ds", DataSourceRole.PRODUCTION), new DataSourceRoleInfo("ds_shadow", DataSourceRole.SHADOW)));
-        dataSourceMapper.put("shadow-data-source-1", Arrays.asList(new DataSourceRoleInfo("ds1", DataSourceRole.PRODUCTION), new DataSourceRoleInfo("ds1_shadow", DataSourceRole.SHADOW)));
+        Map<String, Collection<String>> dataSourceMapper = new LinkedHashMap<>();
+        dataSourceMapper.put("shadow-data-source-0", Arrays.asList("ds", "ds_shadow"));
+        dataSourceMapper.put("shadow-data-source-1", Arrays.asList("ds1", "ds1_shadow"));
         when(result.getDataSourceMapper()).thenReturn(dataSourceMapper);
         return result;
     }
diff --git a/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateShadowRuleStatementUpdaterTest.java b/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateShadowRuleStatementUpdaterTest.java
index ce51c0ee553..440e2acb9ae 100644
--- a/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateShadowRuleStatementUpdaterTest.java
+++ b/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateShadowRuleStatementUpdaterTest.java
@@ -21,8 +21,6 @@ import org.apache.shardingsphere.distsql.handler.exception.rule.DuplicateRuleExc
 import org.apache.shardingsphere.distsql.handler.exception.rule.InvalidRuleConfigurationException;
 import org.apache.shardingsphere.distsql.handler.exception.storageunit.MissingRequiredStorageUnitsException;
 import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRole;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResourceMetaData;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
@@ -91,7 +89,7 @@ class CreateShadowRuleStatementUpdaterTest {
     @Test
     void assertExecuteWithDuplicateLogicResource() {
         DataSourceContainedRule dataSourceContainedRule = mock(DataSourceContainedRule.class);
-        when(dataSourceContainedRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("duplicate_ds", Collections.singleton(new DataSourceRoleInfo("ds_0", DataSourceRole.PRIMARY))));
+        when(dataSourceContainedRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("duplicate_ds", Collections.singleton("ds_0")));
         when(database.getRuleMetaData().findRules(DataSourceContainedRule.class)).thenReturn(Collections.singleton(dataSourceContainedRule));
         ShadowRuleSegment ruleSegment = new ShadowRuleSegment("duplicate_ds", null, null, null);
         assertThrows(InvalidRuleConfigurationException.class, () -> updater.checkSQLStatement(database, new CreateShadowRuleStatement(false, Collections.singleton(ruleSegment)), null));
diff --git a/features/sharding/distsql/handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java b/features/sharding/distsql/handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java
index 683de397c93..277e16a63fc 100644
--- a/features/sharding/distsql/handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java
+++ b/features/sharding/distsql/handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java
@@ -26,7 +26,6 @@ import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
 import org.apache.shardingsphere.distsql.parser.statement.DistSQLStatement;
 import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResourceMetaData;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -304,8 +303,8 @@ class CreateShardingTableRuleStatementUpdaterTest {
         }
         
         @Override
-        public Map<String, Collection<DataSourceRoleInfo>> getDataSourceMapper() {
-            return Collections.singletonMap("logic_ds", Collections.emptyList());
+        public Map<String, Collection<String>> getDataSourceMapper() {
+            return Collections.singletonMap("logic_ds", null);
         }
         
         @Override
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodeUtils.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodeUtils.java
index fa521f2141f..0de98de2bf9 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodeUtils.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodeUtils.java
@@ -19,7 +19,6 @@ package org.apache.shardingsphere.infra.datanode;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -60,13 +59,13 @@ public final class DataNodeUtils {
      * @param dataSources dataSource map
      * @return data node collection
      */
-    public static Collection<DataNode> buildDataNode(final DataNode dataNode, final Map<String, Collection<DataSourceRoleInfo>> dataSources) {
+    public static Collection<DataNode> buildDataNode(final DataNode dataNode, final Map<String, Collection<String>> dataSources) {
         if (!dataSources.containsKey(dataNode.getDataSourceName())) {
             return Collections.singletonList(dataNode);
         }
         Collection<DataNode> result = new LinkedList<>();
-        for (DataSourceRoleInfo each : dataSources.get(dataNode.getDataSourceName())) {
-            result.add(new DataNode(each.getName(), dataNode.getTableName()));
+        for (String each : dataSources.get(dataNode.getDataSourceName())) {
+            result.add(new DataNode(each, dataNode.getTableName()));
         }
         return result;
     }
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/datasource/mapper/DataSourceRole.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/datasource/mapper/DataSourceRole.java
deleted file mode 100644
index b95a2ae48a1..00000000000
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/datasource/mapper/DataSourceRole.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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.infra.datasource.mapper;
-
-/**
- * Data source role.
- */
-public enum DataSourceRole {
-    
-    PRIMARY, MEMBER, PRODUCTION, SHADOW
-}
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/datasource/mapper/DataSourceRoleInfo.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/datasource/mapper/DataSourceRoleInfo.java
deleted file mode 100644
index 5d3c827f29c..00000000000
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/datasource/mapper/DataSourceRoleInfo.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.infra.datasource.mapper;
-
-import lombok.EqualsAndHashCode;
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
-
-/**
- * Data source role info.
- */
-@RequiredArgsConstructor
-@Getter
-@EqualsAndHashCode
-public final class DataSourceRoleInfo {
-    
-    private final String name;
-    
-    private final DataSourceRole role;
-}
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/DataSourceContainedRule.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/DataSourceContainedRule.java
index 99e403ab5ce..ca5302c233d 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/DataSourceContainedRule.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/DataSourceContainedRule.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.infra.rule.identifier.type;
 
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
 import java.util.Collection;
@@ -33,5 +32,5 @@ public interface DataSourceContainedRule extends ShardingSphereRule {
      *
      * @return data source mapper
      */
-    Map<String, Collection<DataSourceRoleInfo>> getDataSourceMapper();
+    Map<String, Collection<String>> getDataSourceMapper();
 }
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/StaticDataSourceContainedRule.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/StaticDataSourceContainedRule.java
index a1b771c34bc..53111980ffd 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/StaticDataSourceContainedRule.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/StaticDataSourceContainedRule.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.infra.rule.identifier.type;
 
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.event.DataSourceStatusChangedEvent;
 
@@ -34,7 +33,7 @@ public interface StaticDataSourceContainedRule extends ShardingSphereRule {
      *
      * @return data source mapper
      */
-    Map<String, Collection<DataSourceRoleInfo>> getDataSourceMapper();
+    Map<String, Collection<String>> getDataSourceMapper();
     
     /**
      * Update data source status.
diff --git a/infra/common/src/test/java/org/apache/shardingsphere/infra/datanode/DataNodeUtilsTest.java b/infra/common/src/test/java/org/apache/shardingsphere/infra/datanode/DataNodeUtilsTest.java
index e0522dbc50a..3c709be1ed7 100644
--- a/infra/common/src/test/java/org/apache/shardingsphere/infra/datanode/DataNodeUtilsTest.java
+++ b/infra/common/src/test/java/org/apache/shardingsphere/infra/datanode/DataNodeUtilsTest.java
@@ -17,8 +17,6 @@
 
 package org.apache.shardingsphere.infra.datanode;
 
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRole;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.junit.jupiter.api.Test;
 
 import java.util.Arrays;
@@ -49,9 +47,7 @@ class DataNodeUtilsTest {
     @Test
     void assertBuildDataNodeWithSameDataSource() {
         DataNode dataNode = new DataNode("readwrite_ds.t_order");
-        Map<String, Collection<DataSourceRoleInfo>> dataSourceMapper = Collections.singletonMap("readwrite_ds",
-                Arrays.asList(new DataSourceRoleInfo("ds_0", DataSourceRole.PRIMARY), new DataSourceRoleInfo("shadow_ds_0", DataSourceRole.MEMBER)));
-        Collection<DataNode> dataNodes = DataNodeUtils.buildDataNode(dataNode, dataSourceMapper);
+        Collection<DataNode> dataNodes = DataNodeUtils.buildDataNode(dataNode, Collections.singletonMap("readwrite_ds", Arrays.asList("ds_0", "shadow_ds_0")));
         assertThat(dataNodes.size(), is(2));
         Iterator<DataNode> iterator = dataNodes.iterator();
         assertThat(iterator.next().getDataSourceName(), is("ds_0"));
@@ -61,9 +57,7 @@ class DataNodeUtilsTest {
     @Test
     void assertBuildDataNodeWithoutSameDataSource() {
         DataNode dataNode = new DataNode("read_ds.t_order");
-        Map<String, Collection<DataSourceRoleInfo>> dataSourceMapper = Collections.singletonMap("readwrite_ds",
-                Arrays.asList(new DataSourceRoleInfo("ds_0", DataSourceRole.PRIMARY), new DataSourceRoleInfo("shadow_ds_0", DataSourceRole.MEMBER)));
-        Collection<DataNode> dataNodes = DataNodeUtils.buildDataNode(dataNode, dataSourceMapper);
+        Collection<DataNode> dataNodes = DataNodeUtils.buildDataNode(dataNode, Collections.singletonMap("readwrite_ds", Arrays.asList("ds_0", "shadow_ds_0")));
         assertThat(dataNodes.size(), is(1));
         assertThat(dataNodes.iterator().next().getDataSourceName(), is("read_ds"));
     }
diff --git a/infra/common/src/test/java/org/apache/shardingsphere/infra/datanode/DataNodesTest.java b/infra/common/src/test/java/org/apache/shardingsphere/infra/datanode/DataNodesTest.java
index 57eb4fcf2fb..e8caf1d4719 100644
--- a/infra/common/src/test/java/org/apache/shardingsphere/infra/datanode/DataNodesTest.java
+++ b/infra/common/src/test/java/org/apache/shardingsphere/infra/datanode/DataNodesTest.java
@@ -17,8 +17,6 @@
 
 package org.apache.shardingsphere.infra.datanode;
 
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRole;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.fixture.FixtureRule;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
@@ -28,8 +26,8 @@ import org.junit.jupiter.api.Test;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
-import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.Map;
 
@@ -40,6 +38,12 @@ import static org.mockito.Mockito.when;
 
 class DataNodesTest {
     
+    private static final Map<String, Collection<String>> READ_WRITE_SPLITTING_DATASOURCE_MAP = new HashMap<>();
+    
+    static {
+        READ_WRITE_SPLITTING_DATASOURCE_MAP.putIfAbsent("readwrite_ds", Arrays.asList("primary_ds", "replica_ds_0", "replica_ds_1"));
+    }
+    
     @Test
     void assertGetDataNodesForShardingTableWithoutDataNodeContainedRule() {
         DataNodes dataNodes = new DataNodes(Collections.singletonList(mockDataSourceContainedRule()));
@@ -131,10 +135,7 @@ class DataNodesTest {
     
     private ShardingSphereRule mockDataSourceContainedRule() {
         DataSourceContainedRule result = mock(FixtureRule.class);
-        Map<String, Collection<DataSourceRoleInfo>> dataSourceMapper = new LinkedHashMap<>();
-        dataSourceMapper.put("readwrite_ds", Arrays.asList(new DataSourceRoleInfo("primary_ds", DataSourceRole.PRIMARY),
-                new DataSourceRoleInfo("replica_ds_0", DataSourceRole.MEMBER), new DataSourceRoleInfo("replica_ds_1", DataSourceRole.MEMBER)));
-        when(result.getDataSourceMapper()).thenReturn(dataSourceMapper);
+        when(result.getDataSourceMapper()).thenReturn(READ_WRITE_SPLITTING_DATASOURCE_MAP);
         return result;
     }
     
diff --git a/infra/common/src/test/java/org/apache/shardingsphere/infra/fixture/FixtureRule.java b/infra/common/src/test/java/org/apache/shardingsphere/infra/fixture/FixtureRule.java
index 0b27496fdfe..e50265e9287 100644
--- a/infra/common/src/test/java/org/apache/shardingsphere/infra/fixture/FixtureRule.java
+++ b/infra/common/src/test/java/org/apache/shardingsphere/infra/fixture/FixtureRule.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.infra.fixture;
 
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
 
@@ -36,7 +35,7 @@ public final class FixtureRule implements DatabaseRule, DataSourceContainedRule
     }
     
     @Override
-    public Map<String, Collection<DataSourceRoleInfo>> getDataSourceMapper() {
+    public Map<String, Collection<String>> getDataSourceMapper() {
         return Collections.emptyMap();
     }
     
diff --git a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleRule.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleRule.java
index 860c05c7f2a..2307d7500ec 100644
--- a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleRule.java
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleRule.java
@@ -21,7 +21,6 @@ import lombok.Getter;
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
 import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.datasource.state.DataSourceStateManager;
 import org.apache.shardingsphere.infra.metadata.database.schema.QualifiedTable;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
@@ -92,10 +91,10 @@ public final class SingleRule implements DatabaseRule, DataNodeContainedRule, Ta
     
     private Map<String, DataSource> getAggregateDataSourceMap(final Map<String, DataSource> dataSourceMap, final DataSourceContainedRule builtRule) {
         Map<String, DataSource> result = new LinkedHashMap<>();
-        for (Entry<String, Collection<DataSourceRoleInfo>> entry : builtRule.getDataSourceMapper().entrySet()) {
-            for (DataSourceRoleInfo each : entry.getValue()) {
-                if (dataSourceMap.containsKey(each.getName())) {
-                    result.putIfAbsent(entry.getKey(), dataSourceMap.remove(each.getName()));
+        for (Entry<String, Collection<String>> entry : builtRule.getDataSourceMapper().entrySet()) {
+            for (String each : entry.getValue()) {
+                if (dataSourceMap.containsKey(each)) {
+                    result.putIfAbsent(entry.getKey(), dataSourceMap.remove(each));
                 }
             }
         }
diff --git a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/subscriber/ConfigurationChangedSubscriber.java b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/subscriber/ConfigurationChangedSubscriber.java
index a91a6942c17..d657ba02769 100644
--- a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/subscriber/ConfigurationChangedSubscriber.java
+++ b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/subscriber/ConfigurationChangedSubscriber.java
@@ -19,7 +19,6 @@ package org.apache.shardingsphere.mode.manager.cluster.coordinator.subscriber;
 
 import com.google.common.eventbus.Subscribe;
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.datasource.props.DataSourceProperties;
 import org.apache.shardingsphere.infra.datasource.state.DataSourceState;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
@@ -139,7 +138,7 @@ public final class ConfigurationChangedSubscriber {
     }
     
     private void disableDataSources(final StorageNodeDataSource storageNodeDataSource, final StaticDataSourceContainedRule rule, final QualifiedDatabase database) {
-        for (Entry<String, Collection<DataSourceRoleInfo>> entry : rule.getDataSourceMapper().entrySet()) {
+        for (Entry<String, Collection<String>> entry : rule.getDataSourceMapper().entrySet()) {
             if (!database.getGroupName().equals(entry.getKey())) {
                 continue;
             }
diff --git a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rdl/storage/unit/UnregisterStorageUnitBackendHandler.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rdl/storage/unit/UnregisterStorageUnitBackendHandler.java
index 69a5f2116ea..f092af84448 100644
--- a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rdl/storage/unit/UnregisterStorageUnitBackendHandler.java
+++ b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rdl/storage/unit/UnregisterStorageUnitBackendHandler.java
@@ -25,7 +25,6 @@ import org.apache.shardingsphere.distsql.handler.exception.storageunit.MissingRe
 import org.apache.shardingsphere.distsql.handler.exception.storageunit.StorageUnitInUsedException;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.UnregisterStorageUnitStatement;
 import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
 import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
@@ -41,7 +40,6 @@ import java.sql.SQLException;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.LinkedHashSet;
 import java.util.Map;
 import java.util.stream.Collectors;
 
@@ -107,8 +105,11 @@ public final class UnregisterStorageUnitBackendHandler extends StorageUnitDefini
     }
     
     private Collection<String> getInUsedResourceNames(final DataSourceContainedRule rule) {
-        return rule.getDataSourceMapper().values().stream().flatMap(Collection::stream)
-                .map(DataSourceRoleInfo::getName).collect(Collectors.toCollection(LinkedHashSet::new));
+        Collection<String> result = new HashSet<>();
+        for (Collection<String> each : rule.getDataSourceMapper().values()) {
+            result.addAll(each);
+        }
+        return result;
     }
     
     private Collection<String> getInUsedResourceNames(final DataNodeContainedRule rule) {
diff --git a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rql/storage/unit/ShowStorageUnitExecutor.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rql/storage/unit/ShowStorageUnitExecutor.java
index 4977aa81c0a..a90bf4a0242 100644
--- a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rql/storage/unit/ShowStorageUnitExecutor.java
+++ b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rql/storage/unit/ShowStorageUnitExecutor.java
@@ -24,7 +24,6 @@ import org.apache.shardingsphere.distsql.handler.query.RQLExecutor;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowStorageUnitsStatement;
 import org.apache.shardingsphere.infra.database.metadata.DataSourceMetaData;
 import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.datasource.props.DataSourceProperties;
 import org.apache.shardingsphere.infra.datasource.props.DataSourcePropertiesCreator;
 import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
@@ -39,7 +38,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
 import java.util.LinkedList;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -128,8 +126,11 @@ public final class ShowStorageUnitExecutor implements RQLExecutor<ShowStorageUni
     }
     
     private Collection<String> getInUsedResourceNames(final DataSourceContainedRule rule) {
-        return rule.getDataSourceMapper().values().stream().flatMap(Collection::stream)
-                .map(DataSourceRoleInfo::getName).collect(Collectors.toCollection(LinkedHashSet::new));
+        Set<String> result = new HashSet<>();
+        for (Collection<String> each : rule.getDataSourceMapper().values()) {
+            result.addAll(each);
+        }
+        return result;
     }
     
     private Collection<String> getInUsedResourceNames(final DataNodeContainedRule rule) {
diff --git a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/fixture/FixtureDataSourceContainedRule.java b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/fixture/FixtureDataSourceContainedRule.java
deleted file mode 100644
index 9c4c7425bd8..00000000000
--- a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/fixture/FixtureDataSourceContainedRule.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.handler.distsql.fixture;
-
-import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
-import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
-import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-
-import static org.mockito.Mockito.mock;
-
-public final class FixtureDataSourceContainedRule implements DatabaseRule, DataSourceContainedRule {
-    
-    @Override
-    public RuleConfiguration getConfiguration() {
-        return mock(RuleConfiguration.class);
-    }
-    
-    @Override
-    public Map<String, Collection<DataSourceRoleInfo>> getDataSourceMapper() {
-        return Collections.emptyMap();
-    }
-    
-    @Override
-    public String getType() {
-        return FixtureDataSourceContainedRule.class.getSimpleName();
-    }
-}
diff --git a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/ImportDatabaseConfigurationUpdaterTest.java b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/ImportDatabaseConfigurationUpdaterTest.java
index 0df6a7309f5..a9722ec5267 100644
--- a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/ImportDatabaseConfigurationUpdaterTest.java
+++ b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/ImportDatabaseConfigurationUpdaterTest.java
@@ -33,7 +33,6 @@ import org.apache.shardingsphere.infra.util.exception.external.sql.type.generic.
 import org.apache.shardingsphere.infra.util.spi.exception.ServiceProviderNotFoundServerException;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.proxy.backend.handler.distsql.fixture.FixtureDataSourceContainedRule;
 import org.apache.shardingsphere.proxy.backend.util.YamlDatabaseConfigurationImportExecutor;
 import org.apache.shardingsphere.test.mock.AutoMockExtension;
 import org.apache.shardingsphere.test.mock.StaticMockSettings;
@@ -130,7 +129,8 @@ class ImportDatabaseConfigurationUpdaterTest {
         when(database.getResourceMetaData()).thenReturn(resourceMetaData);
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
         when(database.getSchema(DefaultDatabase.LOGIC_NAME)).thenReturn(schema);
-        when(database.getRuleMetaData().findRules(DataSourceContainedRule.class)).thenReturn(Collections.singleton(new FixtureDataSourceContainedRule()));
+        DataSourceContainedRule dataSourceContainedRule = mock(DataSourceContainedRule.class);
+        when(database.getRuleMetaData().findRules(DataSourceContainedRule.class)).thenReturn(Collections.singleton(dataSourceContainedRule));
         when(result.getMetaDataContexts().getMetaData().getDatabases()).thenReturn(Collections.singletonMap(databaseName, database));
         when(result.getMetaDataContexts().getMetaData().getDatabase(databaseName)).thenReturn(database);
         when(result.getMetaDataContexts().getMetaData().getProps()).thenReturn(new ConfigurationProperties(createProperties()));
diff --git a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rdl/storage/unit/UnregisterStorageUnitBackendHandlerTest.java b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rdl/storage/unit/UnregisterStorageUnitBackendHandlerTest.java
index 3c48eb73ce9..f893d6384af 100644
--- a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rdl/storage/unit/UnregisterStorageUnitBackendHandlerTest.java
+++ b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rdl/storage/unit/UnregisterStorageUnitBackendHandlerTest.java
@@ -22,8 +22,6 @@ import org.apache.shardingsphere.distsql.handler.exception.storageunit.MissingRe
 import org.apache.shardingsphere.distsql.handler.exception.storageunit.StorageUnitInUsedException;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.UnregisterStorageUnitStatement;
 import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRole;
-import org.apache.shardingsphere.infra.datasource.mapper.DataSourceRoleInfo;
 import org.apache.shardingsphere.infra.instance.mode.ModeContextManager;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResourceMetaData;
@@ -130,7 +128,7 @@ class UnregisterStorageUnitBackendHandlerTest {
     void assertStorageUnitNameInUseExecute() {
         when(ruleMetaData.findRules(DataSourceContainedRule.class)).thenReturn(Collections.singleton(shadowRule));
         when(shadowRule.getType()).thenReturn("ShadowRule");
-        when(shadowRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("", Collections.singleton(new DataSourceRoleInfo("foo_ds", DataSourceRole.PRIMARY))));
+        when(shadowRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("", Collections.singleton("foo_ds")));
         when(resourceMetaData.getDataSources()).thenReturn(Collections.singletonMap("foo_ds", dataSource));
         when(database.getResourceMetaData()).thenReturn(resourceMetaData);
         when(contextManager.getMetaDataContexts().getMetaData().getDatabase("foo_db")).thenReturn(database);
@@ -178,7 +176,7 @@ class UnregisterStorageUnitBackendHandlerTest {
     void assertStorageUnitNameInUseWithIfExists() {
         when(ruleMetaData.findRules(DataSourceContainedRule.class)).thenReturn(Collections.singleton(shadowRule));
         when(shadowRule.getType()).thenReturn("ShadowRule");
-        when(shadowRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("", Collections.singleton(new DataSourceRoleInfo("foo_ds", DataSourceRole.PRIMARY))));
+        when(shadowRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("", Collections.singleton("foo_ds")));
         when(contextManager.getMetaDataContexts().getMetaData().getDatabase("foo_db")).thenReturn(database);
         UnregisterStorageUnitStatement unregisterStorageUnitStatement = new UnregisterStorageUnitStatement(true, Collections.singleton("foo_ds"), true);
         assertThrows(DistSQLException.class, () -> handler.execute("foo_db", unregisterStorageUnitStatement));