This is an automated email from the ASF dual-hosted git repository.
jianglongtao 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 65e8eab5df9 Refactor DataNodeRule (#30307)
65e8eab5df9 is described below
commit 65e8eab5df97b2fa31159d676135d4fbeb7e9227
Author: Liang Zhang <[email protected]>
AuthorDate: Tue Feb 27 11:36:46 2024 +0800
Refactor DataNodeRule (#30307)
* Refactor DataNodeRule
* Refactor DataNodeRule
* Refactor DataNodeRule
* Refactor DataNodeRule
---
.../broadcast/rule/BroadcastDataNodeRule.java | 82 +++++++++++++++
.../broadcast/rule/BroadcastRule.java | 64 +-----------
.../broadcast/route/BroadcastSqlRouterTest.java | 2 +-
.../update/DropReadwriteSplittingRuleExecutor.java | 4 +-
.../DropReadwriteSplittingRuleExecutorTest.java | 7 +-
.../constraint/ShardingConstraintReviser.java | 2 +-
.../sharding/rule/ShardingDataNodeRule.java | 112 +++++++++++++++++++++
.../shardingsphere/sharding/rule/ShardingRule.java | 51 +---------
.../metadata/ShardingMetaDataReviseEngineTest.java | 5 +-
.../sharding/rule/ShardingRuleTest.java | 12 +--
.../shardingsphere/infra/datanode/DataNodes.java | 6 +-
.../infra/metadata/database/rule/RuleMetaData.java | 5 +-
.../collector/tables/PgClassTableCollector.java | 6 +-
.../type/datanode/DataNodeContainedRule.java | 33 ++++++
.../DataNodeRule.java} | 7 +-
.../infra/datanode/DataNodesTest.java | 16 ++-
.../schema/util/SchemaMetaDataUtilsTest.java | 15 +--
.../handler/fixture/DistSQLHandlerFixtureRule.java | 40 ++------
.../driver/executor/DriverJDBCExecutor.java | 4 +-
.../batch/BatchPreparedStatementExecutor.java | 4 +-
.../metadata/ShardingSphereDatabaseMetaData.java | 7 +-
.../core/resultset/DatabaseMetaDataResultSet.java | 4 +-
.../resultset/ShardingSphereResultSetMetaData.java | 6 +-
.../statement/ShardingSpherePreparedStatement.java | 4 +-
.../core/statement/ShardingSphereStatement.java | 4 +-
.../batch/BatchPreparedStatementExecutorTest.java | 4 +-
.../driver/jdbc/adapter/StatementAdapterTest.java | 6 +-
.../resultset/DatabaseMetaDataResultSetTest.java | 5 +-
.../data/pipeline/cdc/util/CDCDataNodeUtils.java | 8 +-
.../pipeline/cdc/util/CDCDataNodeUtilsTest.java | 13 +--
.../single/rule/SingleDataNodeRule.java | 27 ++---
.../shardingsphere/single/rule/SingleRule.java | 42 ++------
.../single/route/SingleSQLRouterTest.java | 4 +-
.../engine/SingleStandardRouteEngineTest.java | 4 +-
.../shardingsphere/single/rule/SingleRuleTest.java | 24 ++---
.../handler/query/ShowSingleTableExecutor.java | 2 +-
.../handler/update/UnloadSingleTableExecutor.java | 2 +-
.../handler/query/ShowSingleTableExecutorTest.java | 4 +-
.../proxy/backend/connector/DatabaseConnector.java | 4 +-
.../header/query/MySQLQueryHeaderBuilder.java | 4 +-
.../header/query/MySQLQueryHeaderBuilderTest.java | 10 +-
41 files changed, 379 insertions(+), 286 deletions(-)
diff --git
a/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastDataNodeRule.java
b/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastDataNodeRule.java
new file mode 100644
index 00000000000..508068d293c
--- /dev/null
+++
b/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastDataNodeRule.java
@@ -0,0 +1,82 @@
+/*
+ * 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.broadcast.rule;
+
+import org.apache.shardingsphere.infra.datanode.DataNode;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Broadcast data node rule.
+ */
+public final class BroadcastDataNodeRule implements DataNodeRule {
+
+ private final Collection<String> tables;
+
+ private final Map<String, Collection<DataNode>> tableDataNodes;
+
+ public BroadcastDataNodeRule(final Collection<String> dataSourceNames,
final Collection<String> tables) {
+ this.tables = tables;
+ tableDataNodes =
tables.stream().collect(Collectors.toMap(String::toLowerCase, each ->
generateDataNodes(each, dataSourceNames)));
+ }
+
+ private Collection<DataNode> generateDataNodes(final String logicTable,
final Collection<String> dataSourceNames) {
+ return dataSourceNames.stream().map(each -> new DataNode(each,
logicTable)).collect(Collectors.toList());
+ }
+
+ @Override
+ public Map<String, Collection<DataNode>> getAllDataNodes() {
+ return tableDataNodes;
+ }
+
+ @Override
+ public Collection<DataNode> getDataNodesByTableName(final String
tableName) {
+ return tableDataNodes.getOrDefault(tableName, Collections.emptyList());
+ }
+
+ @Override
+ public Optional<String> findFirstActualTable(final String logicTable) {
+ return tableDataNodes.containsKey(logicTable.toLowerCase()) ?
Optional.of(logicTable) : Optional.empty();
+ }
+
+ @Override
+ public boolean isNeedAccumulate(final Collection<String> tables) {
+ return tables.isEmpty() || !this.tables.containsAll(tables);
+ }
+
+ @Override
+ public Optional<String> findLogicTableByActualTable(final String
actualTable) {
+ return tableDataNodes.containsKey(actualTable.toLowerCase()) ?
Optional.of(actualTable) : Optional.empty();
+ }
+
+ @Override
+ public Optional<String> findActualTableByCatalog(final String catalog,
final String logicTable) {
+ if (!tableDataNodes.containsKey(logicTable.toLowerCase())) {
+ return Optional.empty();
+ }
+ if
(tableDataNodes.get(logicTable.toLowerCase()).stream().noneMatch(each ->
each.getDataSourceName().equalsIgnoreCase(catalog))) {
+ return Optional.empty();
+ }
+ return Optional.of(logicTable);
+ }
+}
diff --git
a/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastRule.java
b/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastRule.java
index 35454ee182e..cfca4a67c10 100644
---
a/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastRule.java
+++
b/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastRule.java
@@ -19,22 +19,19 @@ package org.apache.shardingsphere.broadcast.rule;
import lombok.Getter;
import
org.apache.shardingsphere.broadcast.api.config.BroadcastRuleConfiguration;
-import org.apache.shardingsphere.infra.datanode.DataNode;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.table.TableMapperContainedRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.table.TableMapperRule;
import javax.sql.DataSource;
import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
import java.util.LinkedList;
import java.util.Map;
import java.util.Map.Entry;
-import java.util.Optional;
import java.util.TreeSet;
import java.util.stream.Collectors;
@@ -52,7 +49,7 @@ public final class BroadcastRule implements DatabaseRule,
DataNodeContainedRule,
private final Collection<String> dataSourceNames;
- private final Map<String, Collection<DataNode>> tableDataNodes;
+ private final DataNodeRule dataNodeRule;
private final TableMapperRule tableMapperRule;
@@ -61,9 +58,8 @@ public final class BroadcastRule implements DatabaseRule,
DataNodeContainedRule,
this.databaseName = databaseName;
dataSourceNames = getAggregatedDataSourceNames(dataSources,
builtRules);
tables = createBroadcastTables(config.getTables());
- tableDataNodes = createShardingTableDataNodes(dataSourceNames, tables);
+ dataNodeRule = new BroadcastDataNodeRule(dataSourceNames, tables);
tableMapperRule = new BroadcastTableMapperRule(tables);
-
}
private Collection<String> getAggregatedDataSourceNames(final Map<String,
DataSource> dataSources, final Collection<ShardingSphereRule> builtRules) {
@@ -98,58 +94,6 @@ public final class BroadcastRule implements DatabaseRule,
DataNodeContainedRule,
return result;
}
- private Map<String, Collection<DataNode>>
createShardingTableDataNodes(final Collection<String> dataSourceNames, final
Collection<String> tables) {
- Map<String, Collection<DataNode>> result = new
HashMap<>(tables.size(), 1F);
- for (String each : tables) {
- result.put(each.toLowerCase(), generateDataNodes(each,
dataSourceNames));
- }
- return result;
- }
-
- private Collection<DataNode> generateDataNodes(final String logicTable,
final Collection<String> dataSourceNames) {
- Collection<DataNode> result = new LinkedList<>();
- for (String each : dataSourceNames) {
- result.add(new DataNode(each, logicTable));
- }
- return result;
- }
-
- @Override
- public Map<String, Collection<DataNode>> getAllDataNodes() {
- return tableDataNodes;
- }
-
- @Override
- public Collection<DataNode> getDataNodesByTableName(final String
tableName) {
- return tableDataNodes.getOrDefault(tableName, Collections.emptyList());
- }
-
- @Override
- public Optional<String> findFirstActualTable(final String logicTable) {
- return tableDataNodes.containsKey(logicTable.toLowerCase()) ?
Optional.of(logicTable) : Optional.empty();
- }
-
- @Override
- public boolean isNeedAccumulate(final Collection<String> tables) {
- return !isAllBroadcastTables(tables);
- }
-
- @Override
- public Optional<String> findLogicTableByActualTable(final String
actualTable) {
- return tableDataNodes.containsKey(actualTable.toLowerCase()) ?
Optional.of(actualTable) : Optional.empty();
- }
-
- @Override
- public Optional<String> findActualTableByCatalog(final String catalog,
final String logicTable) {
- if (!tableDataNodes.containsKey(logicTable.toLowerCase())) {
- return Optional.empty();
- }
- if
(tableDataNodes.get(logicTable.toLowerCase()).stream().noneMatch(each ->
each.getDataSourceName().equalsIgnoreCase(catalog))) {
- return Optional.empty();
- }
- return Optional.of(logicTable);
- }
-
/**
* Get broadcast rule table names.
*
diff --git
a/features/broadcast/core/src/test/java/org/apache/shardingsphere/broadcast/route/BroadcastSqlRouterTest.java
b/features/broadcast/core/src/test/java/org/apache/shardingsphere/broadcast/route/BroadcastSqlRouterTest.java
index 3f640ad4f72..64c594e81ce 100644
---
a/features/broadcast/core/src/test/java/org/apache/shardingsphere/broadcast/route/BroadcastSqlRouterTest.java
+++
b/features/broadcast/core/src/test/java/org/apache/shardingsphere/broadcast/route/BroadcastSqlRouterTest.java
@@ -88,7 +88,7 @@ class BroadcastSqlRouterTest {
BroadcastRuleConfiguration currentConfig =
mock(BroadcastRuleConfiguration.class);
when(currentConfig.getTables()).thenReturn(Collections.singleton("t_order"));
BroadcastRule broadcastRule = new BroadcastRule(currentConfig,
DefaultDatabase.LOGIC_NAME, Collections.singletonMap("tmp_ds", new
MockedDataSource(mockConnection())), Collections.emptyList());
- broadcastRule.getTableDataNodes().put("t_order",
Collections.singletonList(createDataNode("tmp_ds")));
+ broadcastRule.getDataNodeRule().getAllDataNodes().put("t_order",
Collections.singletonList(createDataNode("tmp_ds")));
ShardingSphereDatabase database = mockSingleDatabase();
RouteContext routeContext = new
BroadcastSQLRouter().createRouteContext(createQueryContext(),
mock(RuleMetaData.class), database, broadcastRule, new
ConfigurationProperties(new Properties()),
new ConnectionContext());
diff --git
a/features/readwrite-splitting/distsql/handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutor.java
b/features/readwrite-splitting/distsql/handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutor.java
index 7dd5f479ecd..bef6caca96f 100644
---
a/features/readwrite-splitting/distsql/handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutor.java
+++
b/features/readwrite-splitting/distsql/handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutor.java
@@ -26,7 +26,7 @@ import
org.apache.shardingsphere.infra.algorithm.core.config.AlgorithmConfigurat
import org.apache.shardingsphere.infra.datanode.DataNode;
import
org.apache.shardingsphere.infra.exception.core.ShardingSpherePreconditions;
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.datanode.DataNodeContainedRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.StaticDataSourceContainedRule;
import
org.apache.shardingsphere.readwritesplitting.api.ReadwriteSplittingRuleConfiguration;
@@ -90,7 +90,7 @@ public final class DropReadwriteSplittingRuleExecutor
implements DatabaseRuleDro
continue;
}
Collection<DataNode> actualDataNodes = new HashSet<>();
- each.getAllDataNodes().values().forEach(actualDataNodes::addAll);
+
each.getDataNodeRule().getAllDataNodes().values().forEach(actualDataNodes::addAll);
result.addAll(actualDataNodes.stream().map(DataNode::getDataSourceName).collect(Collectors.toSet()));
}
return result;
diff --git
a/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutorTest.java
b/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutorTest.java
index 8d563c4e9e0..ae1f283f9a9 100644
---
a/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutorTest.java
+++
b/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutorTest.java
@@ -23,7 +23,7 @@ import
org.apache.shardingsphere.distsql.handler.exception.rule.RuleInUsedExcept
import
org.apache.shardingsphere.infra.algorithm.core.config.AlgorithmConfiguration;
import org.apache.shardingsphere.infra.datanode.DataNode;
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.datanode.DataNodeContainedRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
import
org.apache.shardingsphere.readwritesplitting.api.ReadwriteSplittingRuleConfiguration;
import
org.apache.shardingsphere.readwritesplitting.api.rule.ReadwriteSplittingDataSourceRuleConfiguration;
@@ -48,6 +48,7 @@ import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -82,8 +83,8 @@ class DropReadwriteSplittingRuleExecutorTest {
DataSourceContainedRule dataSourceContainedRule =
mock(DataSourceContainedRule.class);
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"))));
+ DataNodeContainedRule dataNodeContainedRule =
mock(DataNodeContainedRule.class, RETURNS_DEEP_STUBS);
+
when(dataNodeContainedRule.getDataNodeRule().getAllDataNodes()).thenReturn(Collections.singletonMap("foo_ds",
Collections.singleton(new DataNode("readwrite_ds.tbl"))));
when(database.getRuleMetaData().findRules(DataNodeContainedRule.class)).thenReturn(Collections.singleton(dataNodeContainedRule));
executor.setDatabase(database);
ReadwriteSplittingRule rule = mock(ReadwriteSplittingRule.class);
diff --git
a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/reviser/constraint/ShardingConstraintReviser.java
b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/reviser/constraint/ShardingConstraintReviser.java
index b422f4b163f..51c602663d7 100644
---
a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/reviser/constraint/ShardingConstraintReviser.java
+++
b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/reviser/constraint/ShardingConstraintReviser.java
@@ -40,7 +40,7 @@ public final class ShardingConstraintReviser implements
ConstraintReviser<Shardi
String referencedTableName =
originalMetaData.getReferencedTableName();
Optional<String> logicIndexName =
getLogicIndex(originalMetaData.getName(), each.getTableName());
if (logicIndexName.isPresent()) {
- return Optional.of(new
ConstraintMetaData(logicIndexName.get(),
rule.findLogicTableByActualTable(referencedTableName).orElse(referencedTableName)));
+ return Optional.of(new
ConstraintMetaData(logicIndexName.get(),
rule.getDataNodeRule().findLogicTableByActualTable(referencedTableName).orElse(referencedTableName)));
}
}
return Optional.empty();
diff --git
a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingDataNodeRule.java
b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingDataNodeRule.java
new file mode 100644
index 00000000000..fe19f0a1bdb
--- /dev/null
+++
b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingDataNodeRule.java
@@ -0,0 +1,112 @@
+/*
+ * 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.sharding.rule;
+
+import com.cedarsoftware.util.CaseInsensitiveMap;
+import com.google.common.base.Strings;
+import org.apache.shardingsphere.infra.datanode.DataNode;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Sharding data node rule.
+ */
+public final class ShardingDataNodeRule implements DataNodeRule {
+
+ private final Map<String, ShardingTable> shardingTables;
+
+ private final Map<String, Collection<DataNode>> tableDataNodes;
+
+ public ShardingDataNodeRule(final Map<String, ShardingTable>
shardingTables) {
+ this.shardingTables = shardingTables;
+ tableDataNodes = createShardingTableDataNodes(shardingTables);
+ }
+
+ private Map<String, Collection<DataNode>>
createShardingTableDataNodes(final Map<String, ShardingTable> shardingTables) {
+ Map<String, Collection<DataNode>> result = new
CaseInsensitiveMap<>(shardingTables.size(), 1F);
+ for (ShardingTable each : shardingTables.values()) {
+ result.put(each.getLogicTable(), each.getActualDataNodes());
+ }
+ return result;
+ }
+
+ @Override
+ public Map<String, Collection<DataNode>> getAllDataNodes() {
+ return tableDataNodes;
+ }
+
+ @Override
+ public Collection<DataNode> getDataNodesByTableName(final String
tableName) {
+ return tableDataNodes.getOrDefault(tableName, Collections.emptyList());
+ }
+
+ @Override
+ public Optional<String> findFirstActualTable(final String logicTable) {
+ return findShardingTable(logicTable).map(optional ->
optional.getActualDataNodes().get(0).getTableName());
+ }
+
+ private Optional<ShardingTable> findShardingTable(final String
logicTableName) {
+ return Strings.isNullOrEmpty(logicTableName) ||
!shardingTables.containsKey(logicTableName) ? Optional.empty() :
Optional.of(shardingTables.get(logicTableName));
+ }
+
+ @Override
+ public boolean isNeedAccumulate(final Collection<String> tables) {
+ return containsShardingTable(tables);
+ }
+
+ private boolean containsShardingTable(final Collection<String>
logicTableNames) {
+ for (String each : logicTableNames) {
+ if (isShardingTable(each)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private boolean isShardingTable(final String logicTableName) {
+ return shardingTables.containsKey(logicTableName);
+ }
+
+ @Override
+ public Optional<String> findLogicTableByActualTable(final String
actualTable) {
+ return
findShardingTableByActualTable(actualTable).map(ShardingTable::getLogicTable);
+ }
+
+ private Optional<ShardingTable> findShardingTableByActualTable(final
String actualTableName) {
+ for (ShardingTable each : shardingTables.values()) {
+ if (each.isExisted(actualTableName)) {
+ return Optional.of(each);
+ }
+ }
+ return Optional.empty();
+ }
+
+ @Override
+ public Optional<String> findActualTableByCatalog(final String catalog,
final String logicTable) {
+ return findShardingTable(logicTable).flatMap(optional ->
findActualTableFromActualDataNode(catalog, optional.getActualDataNodes()));
+ }
+
+ private Optional<String> findActualTableFromActualDataNode(final String
catalog, final List<DataNode> actualDataNodes) {
+ return actualDataNodes.stream().filter(each ->
each.getDataSourceName().equalsIgnoreCase(catalog)).findFirst().map(DataNode::getTableName);
+ }
+}
diff --git
a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
index a536b7a2a86..a177dd2bc0d 100644
---
a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
+++
b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
@@ -35,7 +35,8 @@ import
org.apache.shardingsphere.infra.instance.InstanceContextAware;
import
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
import
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.table.TableMapperContainedRule;
import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
@@ -113,10 +114,10 @@ public final class ShardingRule implements DatabaseRule,
DataNodeContainedRule,
private final String defaultShardingColumn;
- private final Map<String, Collection<DataNode>> shardingTableDataNodes;
-
private final ShardingCache shardingCache;
+ private final DataNodeRule dataNodeRule;
+
private final ShardingTableMapperRule tableMapperRule;
public ShardingRule(final ShardingRuleConfiguration ruleConfig, final
Map<String, DataSource> dataSources, final InstanceContext instanceContext) {
@@ -136,7 +137,6 @@ public final class ShardingRule implements DatabaseRule,
DataNodeContainedRule,
? TypedSPILoader.getService(KeyGenerateAlgorithm.class, null)
:
keyGenerators.get(ruleConfig.getDefaultKeyGenerateStrategy().getKeyGeneratorName());
defaultShardingColumn = ruleConfig.getDefaultShardingColumn();
- shardingTableDataNodes = createShardingTableDataNodes(shardingTables);
ShardingSpherePreconditions.checkState(isValidBindingTableConfiguration(shardingTables,
new BindingTableCheckedConfiguration(this.dataSourceNames, shardingAlgorithms,
ruleConfig.getBindingTableGroups(),
defaultDatabaseShardingStrategyConfig, defaultTableShardingStrategyConfig,
defaultShardingColumn)),
InvalidBindingTablesException::new);
@@ -145,6 +145,7 @@ public final class ShardingRule implements DatabaseRule,
DataNodeContainedRule,
((InstanceContextAware)
defaultKeyGenerateAlgorithm).setInstanceContext(instanceContext);
}
shardingCache = null == ruleConfig.getShardingCache() ? null : new
ShardingCache(ruleConfig.getShardingCache(), this);
+ dataNodeRule = new ShardingDataNodeRule(shardingTables);
tableMapperRule = new ShardingTableMapperRule(shardingTables);
}
@@ -168,14 +169,6 @@ public final class ShardingRule implements DatabaseRule,
DataNodeContainedRule,
return null == ruleConfig.getDefaultTableShardingStrategy() ? new
NoneShardingStrategyConfiguration() :
ruleConfig.getDefaultTableShardingStrategy();
}
- private Map<String, Collection<DataNode>>
createShardingTableDataNodes(final Map<String, ShardingTable> shardingTables) {
- Map<String, Collection<DataNode>> result = new
CaseInsensitiveMap<>(shardingTables.size(), 1F);
- for (ShardingTable each : shardingTables.values()) {
- result.put(each.getLogicTable(), each.getActualDataNodes());
- }
- return result;
- }
-
private Collection<String> getDataSourceNames(final
Collection<ShardingTableRuleConfiguration> tableRuleConfigs,
final
Collection<ShardingAutoTableRuleConfiguration> autoTableRuleConfigs, final
Collection<String> dataSourceNames) {
if (tableRuleConfigs.isEmpty() && autoTableRuleConfigs.isEmpty()) {
@@ -679,40 +672,6 @@ public final class ShardingRule implements DatabaseRule,
DataNodeContainedRule,
return null != shardingCache;
}
- @Override
- public Map<String, Collection<DataNode>> getAllDataNodes() {
- return shardingTableDataNodes;
- }
-
- @Override
- public Collection<DataNode> getDataNodesByTableName(final String
tableName) {
- return shardingTableDataNodes.getOrDefault(tableName,
Collections.emptyList());
- }
-
- @Override
- public Optional<String> findFirstActualTable(final String logicTable) {
- return findShardingTable(logicTable).map(optional ->
optional.getActualDataNodes().get(0).getTableName());
- }
-
- @Override
- public boolean isNeedAccumulate(final Collection<String> tables) {
- return containsShardingTable(tables);
- }
-
- @Override
- public Optional<String> findLogicTableByActualTable(final String
actualTable) {
- return
findShardingTableByActualTable(actualTable).map(ShardingTable::getLogicTable);
- }
-
- @Override
- public Optional<String> findActualTableByCatalog(final String catalog,
final String logicTable) {
- return findShardingTable(logicTable).flatMap(optional ->
findActualTableFromActualDataNode(catalog, optional.getActualDataNodes()));
- }
-
- private Optional<String> findActualTableFromActualDataNode(final String
catalog, final List<DataNode> actualDataNodes) {
- return actualDataNodes.stream().filter(each ->
each.getDataSourceName().equalsIgnoreCase(catalog)).findFirst().map(DataNode::getTableName);
- }
-
private boolean isJoinConditionContainsShardingColumns(final
ShardingSphereSchema schema, final SelectStatementContext select,
final
Collection<String> tableNames, final Collection<WhereSegment> whereSegments) {
Collection<String> databaseJoinConditionTables = new
HashSet<>(tableNames.size(), 1F);
diff --git
a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataReviseEngineTest.java
b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataReviseEngineTest.java
index 4724af365f6..b7daa23b04a 100644
---
a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataReviseEngineTest.java
+++
b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataReviseEngineTest.java
@@ -23,6 +23,7 @@ import
org.apache.shardingsphere.infra.database.core.metadata.data.model.SchemaM
import
org.apache.shardingsphere.infra.database.core.metadata.data.model.TableMetaData;
import
org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterial;
import
org.apache.shardingsphere.infra.metadata.database.schema.reviser.MetaDataReviseEngine;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
import org.apache.shardingsphere.sharding.rule.ShardingRule;
import org.junit.jupiter.api.Test;
@@ -56,7 +57,9 @@ class ShardingMetaDataReviseEngineTest {
private ShardingRule mockShardingRule() {
ShardingRule result = mock(ShardingRule.class);
-
when(result.findLogicTableByActualTable("t_order")).thenReturn(Optional.of("t_order"));
+ DataNodeRule dataNodeRule = mock(DataNodeRule.class);
+
when(dataNodeRule.findLogicTableByActualTable("t_order")).thenReturn(Optional.of("t_order"));
+ when(result.getDataNodeRule()).thenReturn(dataNodeRule);
return result;
}
diff --git
a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
index c6e21ab543d..8376026a7d5 100644
---
a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
+++
b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
@@ -202,12 +202,12 @@ class ShardingRuleTest {
@Test
void assertFindLogicTableByActualTable() {
-
assertTrue(createMaximumShardingRule().findLogicTableByActualTable("table_0").isPresent());
+
assertTrue(createMaximumShardingRule().getDataNodeRule().findLogicTableByActualTable("table_0").isPresent());
}
@Test
void assertNotFindLogicTableByActualTable() {
-
assertFalse(createMaximumShardingRule().findLogicTableByActualTable("table_3").isPresent());
+
assertFalse(createMaximumShardingRule().getDataNodeRule().findLogicTableByActualTable("table_3").isPresent());
}
@Test
@@ -711,7 +711,7 @@ class ShardingRuleTest {
@Test
void assertGetDataNodesByTableName() {
ShardingRule shardingRule = createMinimumShardingRule();
- Collection<DataNode> actual =
shardingRule.getDataNodesByTableName("logic_table");
+ Collection<DataNode> actual =
shardingRule.getDataNodeRule().getDataNodesByTableName("logic_table");
assertThat(actual.size(), is(6));
Iterator<DataNode> iterator = actual.iterator();
DataNode firstDataNode = iterator.next();
@@ -773,7 +773,7 @@ class ShardingRuleTest {
@Test
void assertGetAllDataNodes() {
ShardingRule actual = createMaximumShardingRule();
- Map<String, Collection<DataNode>> allDataNodes =
actual.getAllDataNodes();
+ Map<String, Collection<DataNode>> allDataNodes =
actual.getDataNodeRule().getAllDataNodes();
assertTrue(allDataNodes.containsKey("logic_table"));
assertTrue(allDataNodes.containsKey("sub_logic_table"));
Collection<DataNode> logicTableDataNodes =
allDataNodes.get("logic_table");
@@ -798,14 +798,14 @@ class ShardingRuleTest {
@Test
void assertFindFirstActualTable() {
ShardingRule actual = createMaximumShardingRule();
- Optional<String> logicTable =
actual.findFirstActualTable("logic_table");
+ Optional<String> logicTable =
actual.getDataNodeRule().findFirstActualTable("logic_table");
assertThat(logicTable.orElse(""), is("table_0"));
}
@Test
void assertFindActualTableByCatalog() {
ShardingRule actual = createMaximumShardingRule();
- Optional<String> actualTableByCatalog =
actual.findActualTableByCatalog("ds_0", "logic_table");
+ Optional<String> actualTableByCatalog =
actual.getDataNodeRule().findActualTableByCatalog("ds_0", "logic_table");
assertThat(actualTableByCatalog.orElse(""), is("table_0"));
}
diff --git
a/infra/common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodes.java
b/infra/common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodes.java
index 4b593121e9d..0daadd06c8c 100644
---
a/infra/common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodes.java
+++
b/infra/common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodes.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.infra.datanode;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import org.apache.shardingsphere.infra.spi.type.ordered.OrderedSPILoader;
import java.util.Collection;
@@ -57,7 +57,7 @@ public final class DataNodes {
if (!dataNodeContainedRule.isPresent()) {
return Collections.emptyList();
}
- Collection<DataNode> result = new
LinkedList<>(dataNodeContainedRule.get().getDataNodesByTableName(tableName));
+ Collection<DataNode> result = new
LinkedList<>(dataNodeContainedRule.get().getDataNodeRule().getDataNodesByTableName(tableName));
for (Entry<ShardingSphereRule, DataNodeBuilder> entry :
dataNodeBuilders.entrySet()) {
result = entry.getValue().build(result, entry.getKey());
}
@@ -69,6 +69,6 @@ public final class DataNodes {
}
private boolean isDataNodeContainedRuleContainsTable(final
ShardingSphereRule each, final String tableName) {
- return each instanceof DataNodeContainedRule &&
!((DataNodeContainedRule) each).getDataNodesByTableName(tableName).isEmpty();
+ return each instanceof DataNodeContainedRule &&
!((DataNodeContainedRule)
each).getDataNodeRule().getDataNodesByTableName(tableName).isEmpty();
}
}
diff --git
a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/RuleMetaData.java
b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/RuleMetaData.java
index 5d56abca1a0..0742ffe6baa 100644
---
a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/RuleMetaData.java
+++
b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/RuleMetaData.java
@@ -22,7 +22,7 @@ import lombok.Getter;
import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
import org.apache.shardingsphere.infra.datanode.DataNode;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
import java.util.Collection;
@@ -131,7 +131,8 @@ public final class RuleMetaData {
}
private Collection<String> getInUsedStorageUnitNames(final
DataNodeContainedRule rule) {
- return rule.getAllDataNodes().values().stream().flatMap(each ->
each.stream().map(DataNode::getDataSourceName).collect(Collectors.toSet()).stream()).collect(Collectors.toSet());
+ return rule.getDataNodeRule().getAllDataNodes().values().stream()
+ .flatMap(each ->
each.stream().map(DataNode::getDataSourceName).collect(Collectors.toSet()).stream()).collect(Collectors.toSet());
}
private void mergeInUsedStorageUnitNameAndRules(final Map<String,
Collection<Class<? extends ShardingSphereRule>>> storageUnitNameAndRules,
diff --git
a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/statistics/collector/tables/PgClassTableCollector.java
b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/statistics/collector/tables/PgClassTableCollector.java
index 5aa7efeedfc..f2846f1f9dd 100644
---
a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/statistics/collector/tables/PgClassTableCollector.java
+++
b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/statistics/collector/tables/PgClassTableCollector.java
@@ -24,7 +24,7 @@ import
org.apache.shardingsphere.infra.metadata.statistics.collector.ShardingSph
import
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
import
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import java.sql.SQLException;
import java.util.ArrayList;
@@ -77,8 +77,8 @@ public final class PgClassTableCollector implements
ShardingSphereStatisticsColl
private String decorateTableName(final Collection<DataNodeContainedRule>
dataNodeContainedRules, final String actualTableName) {
for (DataNodeContainedRule each : dataNodeContainedRules) {
- if (each.findLogicTableByActualTable(actualTableName).isPresent())
{
- return each.findLogicTableByActualTable(actualTableName).get();
+ if
(each.getDataNodeRule().findLogicTableByActualTable(actualTableName).isPresent())
{
+ return
each.getDataNodeRule().findLogicTableByActualTable(actualTableName).get();
}
}
return actualTableName;
diff --git
a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/datanode/DataNodeContainedRule.java
b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/datanode/DataNodeContainedRule.java
new file mode 100644
index 00000000000..9403edbd6b1
--- /dev/null
+++
b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/datanode/DataNodeContainedRule.java
@@ -0,0 +1,33 @@
+/*
+ * 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.rule.identifier.type.datanode;
+
+import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+
+/**
+ * ShardingSphere rule which contains data node.
+ */
+public interface DataNodeContainedRule extends ShardingSphereRule {
+
+ /**
+ * Get data node rule.
+ *
+ * @return data node rule
+ */
+ DataNodeRule getDataNodeRule();
+}
diff --git
a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/DataNodeContainedRule.java
b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/datanode/DataNodeRule.java
similarity index 90%
rename from
infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/DataNodeContainedRule.java
rename to
infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/datanode/DataNodeRule.java
index 3526434cc1f..aa2537e14e1 100644
---
a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/DataNodeContainedRule.java
+++
b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/datanode/DataNodeRule.java
@@ -15,19 +15,18 @@
* limitations under the License.
*/
-package org.apache.shardingsphere.infra.rule.identifier.type;
+package org.apache.shardingsphere.infra.rule.identifier.type.datanode;
import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
import java.util.Collection;
import java.util.Map;
import java.util.Optional;
/**
- * ShardingSphere rule which contains data node.
+ * Data node rule.
*/
-public interface DataNodeContainedRule extends ShardingSphereRule {
+public interface DataNodeRule {
/**
* Get all data nodes.
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 e8caf1d4719..7f784f6ceb7 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
@@ -19,8 +19,9 @@ package org.apache.shardingsphere.infra.datanode;
import org.apache.shardingsphere.infra.fixture.FixtureRule;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
@@ -33,6 +34,7 @@ import java.util.Map;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -147,17 +149,21 @@ class DataNodesTest {
}
private ShardingSphereRule mockSingleRule() {
- DataNodeContainedRule result = mock(DataNodeContainedRule.class);
-
when(result.getDataNodesByTableName("t_single")).thenReturn(Collections.singletonList(new
DataNode("readwrite_ds", "t_single")));
+ DataNodeRule dataNodeRule = mock(DataNodeRule.class);
+
when(dataNodeRule.getDataNodesByTableName("t_single")).thenReturn(Collections.singleton(new
DataNode("readwrite_ds", "t_single")));
+ DataNodeContainedRule result = mock(DataNodeContainedRule.class,
RETURNS_DEEP_STUBS);
+ when(result.getDataNodeRule()).thenReturn(dataNodeRule);
return result;
}
private ShardingSphereRule mockShardingRule() {
- DataNodeContainedRule result = mock(DataNodeContainedRule.class);
Collection<DataNode> dataNodes = new LinkedList<>();
dataNodes.add(new DataNode("readwrite_ds", "t_order_0"));
dataNodes.add(new DataNode("readwrite_ds", "t_order_1"));
- when(result.getDataNodesByTableName("t_order")).thenReturn(dataNodes);
+ DataNodeRule dataNodeRule = mock(DataNodeRule.class);
+
when(dataNodeRule.getDataNodesByTableName("t_order")).thenReturn(dataNodes);
+ DataNodeContainedRule result = mock(DataNodeContainedRule.class);
+ when(result.getDataNodeRule()).thenReturn(dataNodeRule);
return result;
}
}
diff --git
a/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/util/SchemaMetaDataUtilsTest.java
b/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/util/SchemaMetaDataUtilsTest.java
index f08a63688ba..dfb22e76ee4 100644
---
a/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/util/SchemaMetaDataUtilsTest.java
+++
b/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/util/SchemaMetaDataUtilsTest.java
@@ -22,7 +22,7 @@ import
org.apache.shardingsphere.infra.database.core.metadata.data.loader.MetaDa
import org.apache.shardingsphere.infra.database.core.type.DatabaseType;
import org.apache.shardingsphere.infra.datanode.DataNode;
import
org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterial;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
import org.apache.shardingsphere.test.fixture.jdbc.MockedDataSource;
@@ -39,6 +39,7 @@ import java.util.Map;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -46,8 +47,8 @@ class SchemaMetaDataUtilsTest {
@Test
void assertGetSchemaMetaDataLoaderMaterialsWhenConfigCheckMetaDataEnable()
{
- DataNodeContainedRule dataNodeContainedRule =
mock(DataNodeContainedRule.class);
-
when(dataNodeContainedRule.getDataNodesByTableName("t_order")).thenReturn(mockShardingDataNodes());
+ DataNodeContainedRule dataNodeContainedRule =
mock(DataNodeContainedRule.class, RETURNS_DEEP_STUBS);
+
when(dataNodeContainedRule.getDataNodeRule().getDataNodesByTableName("t_order")).thenReturn(mockShardingDataNodes());
GenericSchemaBuilderMaterial material = new
GenericSchemaBuilderMaterial(mock(DatabaseType.class), mockStorageTypes(),
mockDataSourceMap(),
Arrays.asList(dataNodeContainedRule,
mock(DataSourceContainedRule.class)), mock(ConfigurationProperties.class),
"sharding_db");
Collection<MetaDataLoaderMaterial> actual =
SchemaMetaDataUtils.getMetaDataLoaderMaterials(Collections.singleton("t_order"),
material, true);
@@ -63,8 +64,8 @@ class SchemaMetaDataUtilsTest {
@Test
void
assertGetSchemaMetaDataLoaderMaterialsWhenNotConfigCheckMetaDataEnable() {
- DataNodeContainedRule dataNodeContainedRule =
mock(DataNodeContainedRule.class);
-
when(dataNodeContainedRule.getDataNodesByTableName("t_order")).thenReturn(mockShardingDataNodes());
+ DataNodeContainedRule dataNodeContainedRule =
mock(DataNodeContainedRule.class, RETURNS_DEEP_STUBS);
+
when(dataNodeContainedRule.getDataNodeRule().getDataNodesByTableName("t_order")).thenReturn(mockShardingDataNodes());
GenericSchemaBuilderMaterial material = new
GenericSchemaBuilderMaterial(mock(DatabaseType.class), mockStorageTypes(),
mockDataSourceMap(),
Arrays.asList(dataNodeContainedRule,
mock(DataSourceContainedRule.class)), mock(ConfigurationProperties.class),
"sharding_db");
Collection<MetaDataLoaderMaterial> actual =
SchemaMetaDataUtils.getMetaDataLoaderMaterials(Collections.singleton("t_order"),
material, false);
@@ -77,8 +78,8 @@ class SchemaMetaDataUtilsTest {
@Test
void
assertGetSchemaMetaDataLoaderMaterialsWhenNotConfigCheckMetaDataEnableForSingleTableDataNode()
{
- DataNodeContainedRule dataNodeContainedRule =
mock(DataNodeContainedRule.class);
-
when(dataNodeContainedRule.getDataNodesByTableName("t_single")).thenReturn(mockSingleTableDataNodes());
+ DataNodeContainedRule dataNodeContainedRule =
mock(DataNodeContainedRule.class, RETURNS_DEEP_STUBS);
+
when(dataNodeContainedRule.getDataNodeRule().getDataNodesByTableName("t_single")).thenReturn(mockSingleTableDataNodes());
GenericSchemaBuilderMaterial material = new
GenericSchemaBuilderMaterial(mock(DatabaseType.class), mockStorageTypes(),
mockDataSourceMap(),
Arrays.asList(dataNodeContainedRule,
mock(DataSourceContainedRule.class)), mock(ConfigurationProperties.class),
"public");
Collection<MetaDataLoaderMaterial> actual =
SchemaMetaDataUtils.getMetaDataLoaderMaterials(Collections.singleton("t_single"),
material, false);
diff --git
a/infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
b/infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
index cde6be16d0c..f43a56109a1 100644
---
a/infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
+++
b/infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
@@ -20,13 +20,13 @@ package org.apache.shardingsphere.distsql.handler.fixture;
import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
import org.apache.shardingsphere.infra.datanode.DataNode;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
-import java.util.Collection;
import java.util.Collections;
-import java.util.Map;
import java.util.Optional;
+import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -38,34 +38,14 @@ public final class DistSQLHandlerFixtureRule implements
ShardingSphereRule, Data
}
@Override
- public Map<String, Collection<DataNode>> getAllDataNodes() {
+ public DataNodeRule getDataNodeRule() {
+ DataNodeRule result = mock(DataNodeRule.class);
DataNode dataNode = mock(DataNode.class);
when(dataNode.getDataSourceName()).thenReturn("foo_ds");
- return Collections.singletonMap("", Collections.singleton(dataNode));
- }
-
- @Override
- public Collection<DataNode> getDataNodesByTableName(final String
tableName) {
- return null;
- }
-
- @Override
- public Optional<String> findFirstActualTable(final String logicTable) {
- return Optional.empty();
- }
-
- @Override
- public boolean isNeedAccumulate(final Collection<String> tables) {
- return false;
- }
-
- @Override
- public Optional<String> findLogicTableByActualTable(final String
actualTable) {
- return Optional.empty();
- }
-
- @Override
- public Optional<String> findActualTableByCatalog(final String catalog,
final String logicTable) {
- return Optional.empty();
+ when(result.getAllDataNodes()).thenReturn(Collections.singletonMap("",
Collections.singleton(dataNode)));
+ when(result.findFirstActualTable(any())).thenReturn(Optional.empty());
+
when(result.findLogicTableByActualTable(any())).thenReturn(Optional.empty());
+ when(result.findActualTableByCatalog(any(),
any())).thenReturn(Optional.empty());
+ return result;
}
}
diff --git
a/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
b/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
index 2c5308a90c5..5e7fbaac352 100644
---
a/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
+++
b/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
@@ -30,7 +30,7 @@ import
org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryRe
import org.apache.shardingsphere.infra.executor.sql.process.ProcessEngine;
import org.apache.shardingsphere.infra.route.context.RouteUnit;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import org.apache.shardingsphere.mode.manager.ContextManager;
import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
@@ -105,7 +105,7 @@ public final class DriverJDBCExecutor {
private boolean isNeedAccumulate(final Collection<ShardingSphereRule>
rules, final SQLStatementContext sqlStatementContext) {
for (ShardingSphereRule each : rules) {
- if (each instanceof DataNodeContainedRule &&
((DataNodeContainedRule)
each).isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames()))
{
+ if (each instanceof DataNodeContainedRule &&
((DataNodeContainedRule)
each).getDataNodeRule().isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames()))
{
return true;
}
}
diff --git
a/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutor.java
b/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutor.java
index 94c18eccbef..7860c66094e 100644
---
a/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutor.java
+++
b/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutor.java
@@ -31,7 +31,7 @@ import
org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.J
import
org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutorCallback;
import org.apache.shardingsphere.infra.metadata.user.Grantee;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
@@ -159,7 +159,7 @@ public final class BatchPreparedStatementExecutor {
private boolean isNeedAccumulate(final SQLStatementContext
sqlStatementContext) {
for (ShardingSphereRule each :
metaDataContexts.getMetaData().getDatabase(databaseName).getRuleMetaData().getRules())
{
- if (each instanceof DataNodeContainedRule &&
((DataNodeContainedRule)
each).isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames()))
{
+ if (each instanceof DataNodeContainedRule &&
((DataNodeContainedRule)
each).getDataNodeRule().isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames()))
{
return true;
}
}
diff --git
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/metadata/ShardingSphereDatabaseMetaData.java
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/metadata/ShardingSphereDatabaseMetaData.java
index 29f37c764f3..4e71ffcd39a 100644
---
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/metadata/ShardingSphereDatabaseMetaData.java
+++
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/metadata/ShardingSphereDatabaseMetaData.java
@@ -23,7 +23,7 @@ import
org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConne
import
org.apache.shardingsphere.driver.jdbc.core.resultset.DatabaseMetaDataResultSet;
import
org.apache.shardingsphere.infra.database.core.connector.ConnectionProperties;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import java.sql.Connection;
import java.sql.DatabaseMetaData;
@@ -201,7 +201,8 @@ public final class ShardingSphereDatabaseMetaData extends
AdaptedDatabaseMetaDat
private String getActualTableNamePattern(final String tableNamePattern) {
return null == tableNamePattern
? null
- : findDataNodeContainedRule().filter(optional ->
optional.findFirstActualTable(tableNamePattern).isPresent()).map(optional ->
"%" + tableNamePattern + "%").orElse(tableNamePattern);
+ : findDataNodeContainedRule()
+ .filter(optional ->
optional.getDataNodeRule().findFirstActualTable(tableNamePattern).isPresent()).map(optional
-> "%" + tableNamePattern + "%").orElse(tableNamePattern);
}
private String getActualTable(final String catalog, final String table) {
@@ -209,7 +210,7 @@ public final class ShardingSphereDatabaseMetaData extends
AdaptedDatabaseMetaDat
}
private Optional<String> findActualTable(final DataNodeContainedRule
dataNodeContainedRule, final String catalog, final String table) {
- return Strings.isNullOrEmpty(catalog) ?
dataNodeContainedRule.findFirstActualTable(table) :
dataNodeContainedRule.findActualTableByCatalog(catalog, table);
+ return Strings.isNullOrEmpty(catalog) ?
dataNodeContainedRule.getDataNodeRule().findFirstActualTable(table) :
dataNodeContainedRule.getDataNodeRule().findActualTableByCatalog(catalog,
table);
}
private Optional<DataNodeContainedRule> findDataNodeContainedRule() {
diff --git
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSet.java
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSet.java
index 5c44c3f0483..321895a080a 100644
---
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSet.java
+++
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSet.java
@@ -25,7 +25,7 @@ import
org.apache.shardingsphere.driver.jdbc.exception.syntax.ColumnLabelNotFoun
import
org.apache.shardingsphere.driver.jdbc.unsupported.AbstractUnsupportedDatabaseMetaDataResultSet;
import
org.apache.shardingsphere.infra.executor.sql.execute.result.query.impl.driver.jdbc.type.util.ResultSetUtils;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import java.math.BigDecimal;
import java.net.URL;
@@ -111,7 +111,7 @@ public final class DatabaseMetaDataResultSet extends
AbstractUnsupportedDatabase
for (int i = 1; i <= columnLabelIndexMap.size(); i++) {
if (tableNameColumnIndex == i) {
String tableName = resultSet.getString(i);
- Optional<String> logicTableName =
dataNodeContainedRule.isPresent() ?
dataNodeContainedRule.get().findLogicTableByActualTable(tableName) :
Optional.empty();
+ Optional<String> logicTableName =
dataNodeContainedRule.isPresent() ?
dataNodeContainedRule.get().getDataNodeRule().findLogicTableByActualTable(tableName)
: Optional.empty();
result.addObject(logicTableName.orElse(tableName));
} else if (indexNameColumnIndex == i) {
String tableName = resultSet.getString(tableNameColumnIndex);
diff --git
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/ShardingSphereResultSetMetaData.java
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/ShardingSphereResultSetMetaData.java
index a936c0f7367..1e1a097623f 100644
---
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/ShardingSphereResultSetMetaData.java
+++
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/ShardingSphereResultSetMetaData.java
@@ -25,7 +25,7 @@ import
org.apache.shardingsphere.infra.binder.context.statement.SQLStatementCont
import
org.apache.shardingsphere.infra.binder.context.statement.dml.SelectStatementContext;
import org.apache.shardingsphere.infra.database.core.DefaultDatabase;
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.datanode.DataNodeContainedRule;
import java.sql.ResultSetMetaData;
import java.sql.SQLException;
@@ -147,8 +147,8 @@ public final class ShardingSphereResultSetMetaData extends
WrapperAdapter implem
private String decorateTableName(final Collection<DataNodeContainedRule>
dataNodeContainedRules, final String actualTableName) {
for (DataNodeContainedRule each : dataNodeContainedRules) {
- if (each.findLogicTableByActualTable(actualTableName).isPresent())
{
- return each.findLogicTableByActualTable(actualTableName).get();
+ if
(each.getDataNodeRule().findLogicTableByActualTable(actualTableName).isPresent())
{
+ return
each.getDataNodeRule().findLogicTableByActualTable(actualTableName).get();
}
}
return actualTableName;
diff --git
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
index 6f289b7aced..81c58612823 100644
---
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
+++
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
@@ -76,7 +76,7 @@ import org.apache.shardingsphere.infra.metadata.user.Grantee;
import org.apache.shardingsphere.infra.parser.SQLParserEngine;
import org.apache.shardingsphere.infra.route.context.RouteContext;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import org.apache.shardingsphere.infra.rule.identifier.type.RawExecutionRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.StorageConnectorReusableRule;
import org.apache.shardingsphere.infra.session.query.QueryContext;
@@ -761,7 +761,7 @@ public final class ShardingSpherePreparedStatement extends
AbstractPreparedState
@Override
public boolean isAccumulate() {
return
metaDataContexts.getMetaData().getDatabase(databaseName).getRuleMetaData().findRules(DataNodeContainedRule.class).stream()
- .anyMatch(each ->
each.isNeedAccumulate(executionContext.getSqlStatementContext().getTablesContext().getTableNames()));
+ .anyMatch(each ->
each.getDataNodeRule().isNeedAccumulate(executionContext.getSqlStatementContext().getTablesContext().getTableNames()));
}
@Override
diff --git
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
index 83de9d7a266..1bdc9d889ec 100644
---
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
+++
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
@@ -70,7 +70,7 @@ import
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
import org.apache.shardingsphere.infra.metadata.database.rule.RuleMetaData;
import org.apache.shardingsphere.infra.metadata.user.Grantee;
import org.apache.shardingsphere.infra.route.context.RouteContext;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import org.apache.shardingsphere.infra.rule.identifier.type.RawExecutionRule;
import org.apache.shardingsphere.infra.session.query.QueryContext;
import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
@@ -665,7 +665,7 @@ public final class ShardingSphereStatement extends
AbstractStatementAdapter {
@Override
public boolean isAccumulate() {
return
metaDataContexts.getMetaData().getDatabase(databaseName).getRuleMetaData().findRules(DataNodeContainedRule.class).stream()
- .anyMatch(each ->
each.isNeedAccumulate(executionContext.getSqlStatementContext().getTablesContext().getTableNames()));
+ .anyMatch(each ->
each.getDataNodeRule().isNeedAccumulate(executionContext.getSqlStatementContext().getTablesContext().getTableNames()));
}
@Override
diff --git
a/jdbc/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
b/jdbc/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
index 23a6fc48c01..09a15092caf 100644
---
a/jdbc/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
+++
b/jdbc/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
@@ -115,8 +115,8 @@ class BatchPreparedStatementExecutorTest {
}
private ShardingRule mockShardingRule() {
- ShardingRule result = mock(ShardingRule.class);
- when(result.isNeedAccumulate(any())).thenReturn(true);
+ ShardingRule result = mock(ShardingRule.class, RETURNS_DEEP_STUBS);
+
when(result.getDataNodeRule().isNeedAccumulate(any())).thenReturn(true);
return result;
}
diff --git
a/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/StatementAdapterTest.java
b/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/StatementAdapterTest.java
index e907a752eac..1622ce2f07a 100644
---
a/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/StatementAdapterTest.java
+++
b/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/StatementAdapterTest.java
@@ -25,7 +25,7 @@ import
org.apache.shardingsphere.infra.database.core.DefaultDatabase;
import org.apache.shardingsphere.infra.database.core.type.DatabaseType;
import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
import org.apache.shardingsphere.infra.metadata.database.rule.RuleMetaData;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
import org.apache.shardingsphere.parser.rule.SQLParserRule;
import
org.apache.shardingsphere.parser.rule.builder.DefaultSQLParserRuleConfigurationBuilder;
@@ -252,8 +252,8 @@ class StatementAdapterTest {
private ShardingSphereStatement
mockShardingSphereStatementWithNeedAccumulate(final Statement... statements) {
ShardingSphereConnection connection =
mock(ShardingSphereConnection.class, RETURNS_DEEP_STUBS);
- DataNodeContainedRule rule = mock(DataNodeContainedRule.class);
- when(rule.isNeedAccumulate(any())).thenReturn(true);
+ DataNodeContainedRule rule = mock(DataNodeContainedRule.class,
RETURNS_DEEP_STUBS);
+ when(rule.getDataNodeRule().isNeedAccumulate(any())).thenReturn(true);
when(connection.getContextManager().getMetaDataContexts().getMetaData().getDatabase(DefaultDatabase.LOGIC_NAME).getRuleMetaData().getRules()).thenReturn(Collections.singleton(rule));
when(connection.getDatabaseName()).thenReturn("db");
DatabaseType databaseType =
TypedSPILoader.getService(DatabaseType.class, "FIXTURE");
diff --git
a/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSetTest.java
b/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSetTest.java
index cedcecd0f1f..9cda437e6f2 100644
---
a/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSetTest.java
+++
b/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSetTest.java
@@ -44,6 +44,7 @@ import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -133,8 +134,8 @@ class DatabaseMetaDataResultSetTest {
}
private ShardingRule mockShardingRule() {
- ShardingRule result = mock(ShardingRule.class);
-
when(result.findLogicTableByActualTable(ACTUAL_TABLE_NAME)).thenReturn(Optional.of(LOGIC_TABLE_NAME));
+ ShardingRule result = mock(ShardingRule.class, RETURNS_DEEP_STUBS);
+
when(result.getDataNodeRule().findLogicTableByActualTable(ACTUAL_TABLE_NAME)).thenReturn(Optional.of(LOGIC_TABLE_NAME));
return result;
}
diff --git
a/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtils.java
b/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtils.java
index 614f00d4206..dfa06c1df98 100644
---
a/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtils.java
+++
b/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtils.java
@@ -56,8 +56,8 @@ public final class CDCDataNodeUtils {
Map<String, List<DataNode>> result = new HashMap<>();
// TODO support virtual data source name
for (String each : tableNames) {
- if (singleRule.isPresent() &&
singleRule.get().getAllDataNodes().containsKey(each)) {
- result.put(each, new
ArrayList<>(singleRule.get().getAllDataNodes().get(each)));
+ if (singleRule.isPresent() &&
singleRule.get().getDataNodeRule().getAllDataNodes().containsKey(each)) {
+ result.put(each, new
ArrayList<>(singleRule.get().getDataNodeRule().getAllDataNodes().get(each)));
continue;
}
if (shardingRule.isPresent() &&
shardingRule.get().findShardingTable(each).isPresent()) {
@@ -65,8 +65,8 @@ public final class CDCDataNodeUtils {
result.put(each, shardingTable.getActualDataNodes());
continue;
}
- if (broadcastRule.isPresent() &&
broadcastRule.get().findFirstActualTable(each).isPresent()) {
- result.put(each,
Collections.singletonList(broadcastRule.get().getTableDataNodes().get(each).iterator().next()));
+ if (broadcastRule.isPresent() &&
broadcastRule.get().getDataNodeRule().findFirstActualTable(each).isPresent()) {
+ result.put(each,
Collections.singletonList(broadcastRule.get().getDataNodeRule().getAllDataNodes().get(each).iterator().next()));
continue;
}
throw new PipelineInvalidParameterException(String.format("Not
find actual data nodes of `%s`", each));
diff --git
a/kernel/data-pipeline/scenario/cdc/core/src/test/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtilsTest.java
b/kernel/data-pipeline/scenario/cdc/core/src/test/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtilsTest.java
index 586d23cc208..ca65c6176ce 100644
---
a/kernel/data-pipeline/scenario/cdc/core/src/test/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtilsTest.java
+++
b/kernel/data-pipeline/scenario/cdc/core/src/test/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtilsTest.java
@@ -35,6 +35,7 @@ import java.util.Optional;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -50,14 +51,14 @@ class CDCDataNodeUtilsTest {
when(mockShardingRule.findShardingTable("t_order")).thenReturn(Optional.of(mockShardingTable));
when(mockShardingRule.getShardingTable("t_order")).thenReturn(mockShardingTable);
when(mockRuleMetaData.findSingleRule(ShardingRule.class)).thenReturn(Optional.of(mockShardingRule));
- SingleRule mockSingleRule = mock(SingleRule.class);
+ SingleRule mockSingleRule = mock(SingleRule.class, RETURNS_DEEP_STUBS);
when(mockRuleMetaData.findSingleRule(SingleRule.class)).thenReturn(Optional.of(mockSingleRule));
-
when(mockSingleRule.getAllDataNodes()).thenReturn(Collections.singletonMap("t_order_item",
Collections.singletonList(new DataNode("single.t_order_item"))));
+
when(mockSingleRule.getDataNodeRule().getAllDataNodes()).thenReturn(Collections.singletonMap("t_order_item",
Collections.singletonList(new DataNode("single.t_order_item"))));
when(mockDatabase.getRuleMetaData()).thenReturn(mockRuleMetaData);
- BroadcastRule mockBroadcastRule = mock(BroadcastRule.class);
-
when(mockRuleMetaData.findSingleRule(BroadcastRule.class)).thenReturn(Optional.of(mockBroadcastRule));
-
when(mockBroadcastRule.findFirstActualTable("t_address")).thenReturn(Optional.of("broadcast.t_address"));
-
when(mockBroadcastRule.getTableDataNodes()).thenReturn(Collections.singletonMap("t_address",
Collections.singletonList(new DataNode("broadcast.t_address"))));
+ BroadcastRule broadcastRule = mock(BroadcastRule.class,
RETURNS_DEEP_STUBS);
+
when(mockRuleMetaData.findSingleRule(BroadcastRule.class)).thenReturn(Optional.of(broadcastRule));
+
when(broadcastRule.getDataNodeRule().findFirstActualTable("t_address")).thenReturn(Optional.of("broadcast.t_address"));
+
when(broadcastRule.getDataNodeRule().getAllDataNodes()).thenReturn(Collections.singletonMap("t_address",
Collections.singletonList(new DataNode("broadcast.t_address"))));
Map<String, List<DataNode>> actual =
CDCDataNodeUtils.buildDataNodesMap(mockDatabase, Arrays.asList("t_order",
"t_order_item", "t_address"));
assertTrue(actual.containsKey("t_order"));
assertTrue(actual.containsKey("t_order_item"));
diff --git
a/infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleDataNodeRule.java
similarity index 68%
copy from
infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
copy to
kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleDataNodeRule.java
index cde6be16d0c..ab6807ad3a0 100644
---
a/infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
+++
b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleDataNodeRule.java
@@ -15,38 +15,33 @@
* limitations under the License.
*/
-package org.apache.shardingsphere.distsql.handler.fixture;
+package org.apache.shardingsphere.single.rule;
-import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
+import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import java.util.Optional;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public final class DistSQLHandlerFixtureRule implements ShardingSphereRule,
DataNodeContainedRule {
+/**
+ * Single data node rule.
+ */
+@RequiredArgsConstructor
+public final class SingleDataNodeRule implements DataNodeRule {
- @Override
- public RuleConfiguration getConfiguration() {
- return mock(RuleConfiguration.class);
- }
+ private final Map<String, Collection<DataNode>> tableDataNodes;
@Override
public Map<String, Collection<DataNode>> getAllDataNodes() {
- DataNode dataNode = mock(DataNode.class);
- when(dataNode.getDataSourceName()).thenReturn("foo_ds");
- return Collections.singletonMap("", Collections.singleton(dataNode));
+ return tableDataNodes;
}
@Override
public Collection<DataNode> getDataNodesByTableName(final String
tableName) {
- return null;
+ return tableDataNodes.getOrDefault(tableName.toLowerCase(),
Collections.emptyList());
}
@Override
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 06fef5b4b10..115146d8247 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
@@ -29,8 +29,9 @@ import
org.apache.shardingsphere.infra.metadata.database.schema.QualifiedTable;
import
org.apache.shardingsphere.infra.metadata.database.schema.util.IndexMetaDataUtils;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.exportable.ExportableRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.exportable.constant.ExportableConstants;
import
org.apache.shardingsphere.infra.rule.identifier.type.table.TableMapperContainedRule;
@@ -64,16 +65,18 @@ public final class SingleRule implements DatabaseRule,
DataNodeContainedRule, Ta
@Getter
private final Collection<String> dataSourceNames;
- @Getter
private final Map<String, Collection<DataNode>> singleTableDataNodes;
private final DatabaseType protocolType;
+ @Getter
+ private final DataNodeRule dataNodeRule;
+
@Getter
private final SingleTableMapperRule tableMapperRule;
- public SingleRule(final SingleRuleConfiguration ruleConfig, final String
databaseName, final DatabaseType protocolType, final Map<String, DataSource>
dataSourceMap,
- final Collection<ShardingSphereRule> builtRules) {
+ public SingleRule(final SingleRuleConfiguration ruleConfig, final String
databaseName,
+ final DatabaseType protocolType, final Map<String,
DataSource> dataSourceMap, final Collection<ShardingSphereRule> builtRules) {
configuration = ruleConfig;
defaultDataSource = ruleConfig.getDefaultDataSource().orElse(null);
Map<String, DataSource> enabledDataSources =
DataSourceStateManager.getInstance().getEnabledDataSources(databaseName,
dataSourceMap);
@@ -81,6 +84,7 @@ public final class SingleRule implements DatabaseRule,
DataNodeContainedRule, Ta
dataSourceNames = aggregateDataSourceMap.keySet();
this.protocolType = protocolType;
singleTableDataNodes = SingleTableDataNodeLoader.load(databaseName,
protocolType, aggregateDataSourceMap, builtRules, configuration.getTables());
+ dataNodeRule = new SingleDataNodeRule(singleTableDataNodes);
tableMapperRule = new SingleTableMapperRule(singleTableDataNodes);
}
@@ -255,36 +259,6 @@ public final class SingleRule implements DatabaseRule,
DataNodeContainedRule, Ta
return new SingleRule((SingleRuleConfiguration) config, databaseName,
protocolType, dataSourceMap, builtRules);
}
- @Override
- public Map<String, Collection<DataNode>> getAllDataNodes() {
- return singleTableDataNodes;
- }
-
- @Override
- public Collection<DataNode> getDataNodesByTableName(final String
tableName) {
- return singleTableDataNodes.getOrDefault(tableName.toLowerCase(),
Collections.emptyList());
- }
-
- @Override
- public Optional<String> findFirstActualTable(final String logicTable) {
- return Optional.empty();
- }
-
- @Override
- public boolean isNeedAccumulate(final Collection<String> tables) {
- return false;
- }
-
- @Override
- public Optional<String> findLogicTableByActualTable(final String
actualTable) {
- return Optional.empty();
- }
-
- @Override
- public Optional<String> findActualTableByCatalog(final String catalog,
final String logicTable) {
- return Optional.empty();
- }
-
@Override
public Map<String, Object> getExportData() {
return
Collections.singletonMap(ExportableConstants.EXPORT_SINGLE_TABLES,
tableMapperRule.getLogicTableMapper().getTableNames());
diff --git
a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/SingleSQLRouterTest.java
b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/SingleSQLRouterTest.java
index 233784700fb..4dd3ab5a348 100644
---
a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/SingleSQLRouterTest.java
+++
b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/SingleSQLRouterTest.java
@@ -73,7 +73,7 @@ class SingleSQLRouterTest {
void assertCreateRouteContextWithSingleDataSource() throws SQLException {
SingleRule rule = new SingleRule(new SingleRuleConfiguration(),
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(),
Collections.singletonMap("foo_ds", new MockedDataSource(mockConnection())),
Collections.emptyList());
- rule.getSingleTableDataNodes().put("t_order",
Collections.singletonList(createDataNode("foo_ds")));
+ rule.getDataNodeRule().getAllDataNodes().put("t_order",
Collections.singletonList(createDataNode("foo_ds")));
ShardingSphereDatabase database = mockSingleDatabase();
RouteContext actual = new
SingleSQLRouter().createRouteContext(createQueryContext(),
mock(RuleMetaData.class), database, rule, new
ConfigurationProperties(new Properties()), new ConnectionContext());
@@ -94,7 +94,7 @@ class SingleSQLRouterTest {
void assertCreateRouteContextWithReadwriteSplittingDataSource() throws
SQLException {
SingleRule rule = new SingleRule(new SingleRuleConfiguration(),
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(),
Collections.singletonMap("readwrite_ds", new
MockedDataSource(mockConnection())), Collections.emptyList());
- rule.getSingleTableDataNodes().put("t_order",
Collections.singletonList(createDataNode("write_ds")));
+ rule.getDataNodeRule().getAllDataNodes().put("t_order",
Collections.singletonList(createDataNode("write_ds")));
ShardingSphereDatabase database = mockReadwriteSplittingDatabase();
RouteContext actual = new
SingleSQLRouter().createRouteContext(createQueryContext(),
mock(RuleMetaData.class), database, rule, new
ConfigurationProperties(new Properties()), new ConnectionContext());
diff --git
a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/engine/SingleStandardRouteEngineTest.java
b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/engine/SingleStandardRouteEngineTest.java
index a6507575165..48ec99df1bb 100644
---
a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/engine/SingleStandardRouteEngineTest.java
+++
b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/engine/SingleStandardRouteEngineTest.java
@@ -62,8 +62,8 @@ class SingleStandardRouteEngineTest {
void assertRouteInSameDataSource() throws SQLException {
SingleStandardRouteEngine engine = new
SingleStandardRouteEngine(mockQualifiedTables(), null);
SingleRule singleRule = new SingleRule(new SingleRuleConfiguration(),
DefaultDatabase.LOGIC_NAME, new MySQLDatabaseType(), createDataSourceMap(),
Collections.emptyList());
- singleRule.getSingleTableDataNodes().put("t_order",
Collections.singleton(mockDataNode("t_order")));
- singleRule.getSingleTableDataNodes().put("t_order_item",
Collections.singleton(mockDataNode("t_order_item")));
+ singleRule.getDataNodeRule().getAllDataNodes().put("t_order",
Collections.singleton(mockDataNode("t_order")));
+ singleRule.getDataNodeRule().getAllDataNodes().put("t_order_item",
Collections.singleton(mockDataNode("t_order_item")));
RouteContext routeContext = new RouteContext();
engine.route(routeContext, singleRule);
List<RouteUnit> routeUnits = new
ArrayList<>(routeContext.getRouteUnits());
diff --git
a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/rule/SingleRuleTest.java
b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/rule/SingleRuleTest.java
index 01fd57e1cce..3ee1eb177b3 100644
---
a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/rule/SingleRuleTest.java
+++
b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/rule/SingleRuleTest.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.infra.datanode.DataNode;
import org.apache.shardingsphere.infra.metadata.database.schema.QualifiedTable;
import org.apache.shardingsphere.infra.route.context.RouteContext;
import org.apache.shardingsphere.infra.route.context.RouteMapper;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import
org.apache.shardingsphere.infra.rule.identifier.type.table.TableMapperContainedRule;
import org.apache.shardingsphere.single.api.config.SingleRuleConfiguration;
import org.apache.shardingsphere.test.fixture.jdbc.MockedDataSource;
@@ -104,7 +104,7 @@ class SingleRuleTest {
when(tableContainedRule.getTableMapperRule().getDistributedTableMapper().getTableNames()).thenReturn(Collections.singletonList("t_order"));
when(tableContainedRule.getTableMapperRule().getActualTableMapper().getTableNames()).thenReturn(Arrays.asList("t_order_0",
"t_order_1"));
SingleRule singleRule = new SingleRule(ruleConfig,
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap,
Collections.singleton(tableContainedRule));
- Map<String, Collection<DataNode>> actual =
singleRule.getSingleTableDataNodes();
+ Map<String, Collection<DataNode>> actual =
singleRule.getDataNodeRule().getAllDataNodes();
assertThat(actual.size(), is(2));
assertTrue(actual.containsKey("employee"));
assertTrue(actual.containsKey("student"));
@@ -116,7 +116,7 @@ class SingleRuleTest {
when(tableContainedRule.getTableMapperRule().getDistributedTableMapper().getTableNames()).thenReturn(Collections.singleton("T_ORDER"));
when(tableContainedRule.getTableMapperRule().getActualTableMapper().getTableNames()).thenReturn(Arrays.asList("T_ORDER_0",
"T_ORDER_1"));
SingleRule singleRule = new SingleRule(ruleConfig,
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap,
Collections.singleton(tableContainedRule));
- Map<String, Collection<DataNode>> actual =
singleRule.getSingleTableDataNodes();
+ Map<String, Collection<DataNode>> actual =
singleRule.getDataNodeRule().getAllDataNodes();
assertThat(actual.size(), is(2));
assertTrue(actual.containsKey("employee"));
assertTrue(actual.containsKey("student"));
@@ -211,17 +211,17 @@ class SingleRuleTest {
void assertGetAllDataNodes() {
DataNodeContainedRule dataNodeContainedRule =
mock(DataNodeContainedRule.class);
SingleRule singleRule = new SingleRule(ruleConfig,
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap,
Collections.singleton(dataNodeContainedRule));
- assertTrue(singleRule.getAllDataNodes().containsKey("employee"));
- assertTrue(singleRule.getAllDataNodes().containsKey("student"));
- assertTrue(singleRule.getAllDataNodes().containsKey("t_order_0"));
- assertTrue(singleRule.getAllDataNodes().containsKey("t_order_1"));
+
assertTrue(singleRule.getDataNodeRule().getAllDataNodes().containsKey("employee"));
+
assertTrue(singleRule.getDataNodeRule().getAllDataNodes().containsKey("student"));
+
assertTrue(singleRule.getDataNodeRule().getAllDataNodes().containsKey("t_order_0"));
+
assertTrue(singleRule.getDataNodeRule().getAllDataNodes().containsKey("t_order_1"));
}
@Test
void assertGetDataNodesByTableName() {
DataNodeContainedRule dataNodeContainedRule =
mock(DataNodeContainedRule.class);
SingleRule singleRule = new SingleRule(ruleConfig,
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap,
Collections.singleton(dataNodeContainedRule));
- Collection<DataNode> actual =
singleRule.getDataNodesByTableName("EMPLOYEE");
+ Collection<DataNode> actual =
singleRule.getDataNodeRule().getDataNodesByTableName("EMPLOYEE");
assertThat(actual.size(), is(1));
DataNode dataNode = actual.iterator().next();
assertThat(dataNode.getDataSourceName(), is("foo_ds"));
@@ -233,14 +233,14 @@ class SingleRuleTest {
DataNodeContainedRule dataNodeContainedRule =
mock(DataNodeContainedRule.class);
SingleRule singleRule = new SingleRule(ruleConfig,
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap,
Collections.singleton(dataNodeContainedRule));
String logicTable = "employee";
- assertFalse(singleRule.findFirstActualTable(logicTable).isPresent());
+
assertFalse(singleRule.getDataNodeRule().findFirstActualTable(logicTable).isPresent());
}
@Test
void assertIsNeedAccumulate() {
DataNodeContainedRule dataNodeContainedRule =
mock(DataNodeContainedRule.class);
SingleRule singleRule = new SingleRule(ruleConfig,
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap,
Collections.singleton(dataNodeContainedRule));
- assertFalse(singleRule.isNeedAccumulate(Collections.emptyList()));
+
assertFalse(singleRule.getDataNodeRule().isNeedAccumulate(Collections.emptyList()));
}
@Test
@@ -248,7 +248,7 @@ class SingleRuleTest {
DataNodeContainedRule dataNodeContainedRule =
mock(DataNodeContainedRule.class);
SingleRule singleRule = new SingleRule(ruleConfig,
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap,
Collections.singleton(dataNodeContainedRule));
String actualTable = "student";
-
assertFalse(singleRule.findLogicTableByActualTable(actualTable).isPresent());
+
assertFalse(singleRule.getDataNodeRule().findLogicTableByActualTable(actualTable).isPresent());
}
@Test
@@ -257,6 +257,6 @@ class SingleRuleTest {
SingleRule singleRule = new SingleRule(ruleConfig,
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap,
Collections.singleton(dataNodeContainedRule));
String catalog = "employee";
String logicTable = "t_order_0";
- assertFalse(singleRule.findActualTableByCatalog(catalog,
logicTable).isPresent());
+
assertFalse(singleRule.getDataNodeRule().findActualTableByCatalog(catalog,
logicTable).isPresent());
}
}
diff --git
a/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutor.java
b/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutor.java
index 4c3675def05..50a27be73e2 100644
---
a/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutor.java
+++
b/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutor.java
@@ -51,7 +51,7 @@ public final class ShowSingleTableExecutor implements
DistSQLQueryExecutor<ShowS
@Override
public Collection<LocalDataQueryResultRow> getRows(final
ShowSingleTableStatement sqlStatement, final ContextManager contextManager) {
Collection<DataNode> resultDataNodes = getPattern(sqlStatement)
- .map(optional ->
getDataNodesWithLikePattern(rule.getSingleTableDataNodes(),
optional)).orElseGet(() -> getDataNodes(rule.getSingleTableDataNodes(),
sqlStatement));
+ .map(optional ->
getDataNodesWithLikePattern(rule.getDataNodeRule().getAllDataNodes(),
optional)).orElseGet(() ->
getDataNodes(rule.getDataNodeRule().getAllDataNodes(), sqlStatement));
Collection<DataNode> sortedDataNodes =
resultDataNodes.stream().sorted(Comparator.comparing(DataNode::getTableName)).collect(Collectors.toList());
return sortedDataNodes.stream().map(each -> new
LocalDataQueryResultRow(each.getTableName(),
each.getDataSourceName())).collect(Collectors.toList());
}
diff --git
a/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/update/UnloadSingleTableExecutor.java
b/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/update/UnloadSingleTableExecutor.java
index b2299df06e8..6d92ed4a621 100644
---
a/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/update/UnloadSingleTableExecutor.java
+++
b/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/update/UnloadSingleTableExecutor.java
@@ -64,7 +64,7 @@ public final class UnloadSingleTableExecutor implements
DatabaseRuleAlterExecuto
for (String each : sqlStatement.getTables()) {
checkTableExist(allTables, each);
checkIsSingleTable(singleTables, each);
- checkTableRuleExist(database.getName(),
database.getProtocolType(), singleRule.getDataNodesByTableName(each), each);
+ checkTableRuleExist(database.getName(),
database.getProtocolType(),
singleRule.getDataNodeRule().getDataNodesByTableName(each), each);
}
}
diff --git
a/kernel/single/distsql/handler/src/test/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutorTest.java
b/kernel/single/distsql/handler/src/test/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutorTest.java
index fba8523ee86..fa95936e169 100644
---
a/kernel/single/distsql/handler/src/test/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutorTest.java
+++
b/kernel/single/distsql/handler/src/test/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutorTest.java
@@ -86,11 +86,11 @@ class ShowSingleTableExecutorTest {
}
private SingleRule mockSingleRule() {
- SingleRule result = mock(SingleRule.class);
+ SingleRule result = mock(SingleRule.class, RETURNS_DEEP_STUBS);
Map<String, Collection<DataNode>> singleTableDataNodeMap = new
HashMap<>();
singleTableDataNodeMap.put("t_order", Collections.singleton(new
DataNode("ds_1", "t_order")));
singleTableDataNodeMap.put("t_order_item", Collections.singleton(new
DataNode("ds_2", "t_order_item")));
-
when(result.getSingleTableDataNodes()).thenReturn(singleTableDataNodeMap);
+
when(result.getDataNodeRule().getAllDataNodes()).thenReturn(singleTableDataNodeMap);
return result;
}
}
diff --git
a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnector.java
b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnector.java
index ab30083ad4b..6bece73bea9 100644
---
a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnector.java
+++
b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnector.java
@@ -46,7 +46,7 @@ import
org.apache.shardingsphere.infra.merge.result.MergedResult;
import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
import
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
import
org.apache.shardingsphere.infra.metadata.database.schema.util.SystemSchemaUtils;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import org.apache.shardingsphere.infra.session.query.QueryContext;
import org.apache.shardingsphere.mode.manager.ContextManager;
import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
@@ -343,7 +343,7 @@ public final class DatabaseConnector implements
DatabaseBackendHandler {
private boolean isNeedAccumulate(final SQLStatementContext
sqlStatementContext) {
Optional<DataNodeContainedRule> dataNodeContainedRule =
database.getRuleMetaData().findSingleRule(DataNodeContainedRule.class);
- return dataNodeContainedRule.isPresent() &&
dataNodeContainedRule.get().isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames());
+ return dataNodeContainedRule.isPresent() &&
dataNodeContainedRule.get().getDataNodeRule().isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames());
}
/**
diff --git
a/proxy/backend/type/mysql/src/main/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilder.java
b/proxy/backend/type/mysql/src/main/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilder.java
index fe6b745a994..4e74c268947 100644
---
a/proxy/backend/type/mysql/src/main/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilder.java
+++
b/proxy/backend/type/mysql/src/main/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilder.java
@@ -21,7 +21,7 @@ import
org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryRe
import
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
import
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereColumn;
import
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import
org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader;
import
org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeaderBuilder;
@@ -61,7 +61,7 @@ public final class MySQLQueryHeaderBuilder implements
QueryHeaderBuilder {
private String getLogicTableName(final ShardingSphereDatabase database,
final String actualTableName) {
for (DataNodeContainedRule each :
database.getRuleMetaData().findRules(DataNodeContainedRule.class)) {
- Optional<String> logicTable =
each.findLogicTableByActualTable(actualTableName);
+ Optional<String> logicTable =
each.getDataNodeRule().findLogicTableByActualTable(actualTableName);
if (logicTable.isPresent()) {
return logicTable.get();
}
diff --git
a/proxy/backend/type/mysql/src/test/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilderTest.java
b/proxy/backend/type/mysql/src/test/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilderTest.java
index c6515658be0..44828a75227 100644
---
a/proxy/backend/type/mysql/src/test/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilderTest.java
+++
b/proxy/backend/type/mysql/src/test/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilderTest.java
@@ -24,7 +24,7 @@ import
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSp
import
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereIndex;
import
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
import
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
-import
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
import
org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader;
import org.apache.shardingsphere.sharding.rule.ShardingRule;
import org.junit.jupiter.api.Test;
@@ -79,8 +79,8 @@ class MySQLQueryHeaderBuilderTest {
void assertBuildWithNullSchema() throws SQLException {
ShardingSphereDatabase database = mock(ShardingSphereDatabase.class,
RETURNS_DEEP_STUBS);
when(database.getSchemas()).thenReturn(Collections.emptyMap());
- DataNodeContainedRule containedRule =
mock(DataNodeContainedRule.class);
-
when(containedRule.findLogicTableByActualTable("t_order")).thenReturn(Optional.of("t_order"));
+ DataNodeContainedRule containedRule =
mock(DataNodeContainedRule.class, RETURNS_DEEP_STUBS);
+
when(containedRule.getDataNodeRule().findLogicTableByActualTable("t_order")).thenReturn(Optional.of("t_order"));
when(database.getRuleMetaData().findRules(DataNodeContainedRule.class)).thenReturn(Collections.singletonList(containedRule));
QueryResultMetaData queryResultMetaData = createQueryResultMetaData();
QueryHeader actual = new
MySQLQueryHeaderBuilder().build(queryResultMetaData, database,
queryResultMetaData.getColumnName(1), queryResultMetaData.getColumnLabel(1), 1);
@@ -104,8 +104,8 @@ class MySQLQueryHeaderBuilderTest {
when(schema.getTable("t_logic_order")).thenReturn(
new ShardingSphereTable("t_logic_order",
Collections.singleton(column), Collections.singleton(new
ShardingSphereIndex("order_id")), Collections.emptyList()));
when(result.getSchema(DefaultDatabase.LOGIC_NAME)).thenReturn(schema);
- ShardingRule shardingRule = mock(ShardingRule.class);
-
when(shardingRule.findLogicTableByActualTable("t_order")).thenReturn(Optional.of("t_logic_order"));
+ ShardingRule shardingRule = mock(ShardingRule.class,
RETURNS_DEEP_STUBS);
+
when(shardingRule.getDataNodeRule().findLogicTableByActualTable("t_order")).thenReturn(Optional.of("t_logic_order"));
when(result.getRuleMetaData().findRules(DataNodeContainedRule.class)).thenReturn(Collections.singleton(shardingRule));
when(result.getName()).thenReturn(DefaultDatabase.LOGIC_NAME);
return result;