You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2022/06/25 16:58:55 UTC

[shardingsphere] branch master updated: Support force route db or table use hint (#18592)

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

zhangliang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new ed72b893588 Support force route db or table use hint (#18592)
ed72b893588 is described below

commit ed72b893588664ea3eef1a978a4c9cc23dab5bc8
Author: GavinPeng <ro...@163.com>
AuthorDate: Sun Jun 26 00:58:48 2022 +0800

    Support force route db or table use hint (#18592)
    
    * support forced rerouting by hint
    
    * delete import java.util.*
    
    * fix check style
    
    * add force route unit test
    
    * rollback pom.xml
    
    Co-authored-by: gavin.peng <ga...@ximalaya.com>
---
 .../standard/ShardingStandardRoutingEngine.java    | 32 +++++++++++++---
 .../ShardingStandardRoutingEngineTest.java         | 43 ++++++++++++++++++++++
 2 files changed, 70 insertions(+), 5 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/standard/ShardingStandardRoutingEngine.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/standard/ShardingStandardRoutingEngine.java
index 016fce7c290..a17dba726f4 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/standard/ShardingStandardRoutingEngine.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/standard/ShardingStandardRoutingEngine.java
@@ -40,7 +40,6 @@ import org.apache.shardingsphere.sharding.rule.BindingTableRule;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
 import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
-
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -48,6 +47,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.LinkedHashSet;
 
 /**
  * Sharding standard routing engine.
@@ -202,6 +202,16 @@ public final class ShardingStandardRoutingEngine implements ShardingRouteEngine
     }
     
     private Collection<String> routeDataSources(final TableRule tableRule, final ShardingStrategy databaseShardingStrategy, final List<ShardingConditionValue> databaseShardingValues) {
+        if (!(databaseShardingStrategy instanceof HintShardingStrategy)) {
+            Collection<Comparable<?>> databaseShardings = HintManager.getDatabaseShardingValues(tableRule.getLogicTable());
+            if (databaseShardings != null && databaseShardings.size() > 0) {
+                List<String> list = new ArrayList<>();
+                for (Comparable<?> databaseSharding : databaseShardings) {
+                    list.add(String.valueOf(databaseSharding));
+                }
+                return list;
+            }
+        }
         if (databaseShardingValues.isEmpty()) {
             return tableRule.getActualDatasourceNames();
         }
@@ -214,10 +224,22 @@ public final class ShardingStandardRoutingEngine implements ShardingRouteEngine
     
     private Collection<DataNode> routeTables(final TableRule tableRule, final String routedDataSource,
                                              final ShardingStrategy tableShardingStrategy, final List<ShardingConditionValue> tableShardingValues) {
-        Collection<String> availableTargetTables = tableRule.getActualTableNames(routedDataSource);
-        Collection<String> routedTables = tableShardingValues.isEmpty()
-                ? availableTargetTables
-                : tableShardingStrategy.doSharding(availableTargetTables, tableShardingValues, tableRule.getTableDataNode(), properties);
+        Collection<String> routedTables = null;
+        if (!(tableShardingStrategy instanceof HintShardingStrategy)) {
+            Collection<Comparable<?>> tableShardings = HintManager.getTableShardingValues(tableRule.getLogicTable());
+            if (tableShardings != null && tableShardings.size() > 0) {
+                routedTables = new ArrayList<>(2);
+                for (Comparable<?> tableSharding : tableShardings) {
+                    routedTables.add((String) tableSharding);
+                }
+            }
+        }
+        if (routedTables == null || routedTables.isEmpty()) {
+            Collection<String> availableTargetTables = tableRule.getActualTableNames(routedDataSource);
+            routedTables = new LinkedHashSet<>(tableShardingValues.isEmpty()
+                    ? availableTargetTables
+                    : tableShardingStrategy.doSharding(availableTargetTables, tableShardingValues, tableRule.getTableDataNode(), properties));
+        }
         Collection<DataNode> result = new LinkedList<>();
         for (String each : routedTables) {
             result.add(new DataNode(routedDataSource, each));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/ShardingStandardRoutingEngineTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/ShardingStandardRoutingEngineTest.java
index ee8746f88d1..176b04c3ed2 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/ShardingStandardRoutingEngineTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/ShardingStandardRoutingEngineTest.java
@@ -82,6 +82,49 @@ public final class ShardingStandardRoutingEngineTest extends AbstractRoutingEngi
         assertThat(routeUnits.get(0).getTableMappers().iterator().next().getLogicName(), is("t_order"));
     }
     
+    @Test
+    public void assertForceRouteDbUseHintByShardingConditions() {
+        ShardingStandardRoutingEngine standardRoutingEngine = createShardingStandardRoutingEngine("t_order", createShardingConditions("t_order"));
+        HintManager hintManager = HintManager.getInstance();
+        hintManager.addDatabaseShardingValue("t_order", "ds_0");
+        RouteContext routeContext = standardRoutingEngine.route(createBasedShardingRule());
+        List<RouteUnit> routeUnits = new ArrayList<>(routeContext.getRouteUnits());
+        assertThat(routeContext.getRouteUnits().size(), is(1));
+        assertThat(routeUnits.get(0).getDataSourceMapper().getActualName(), is("ds_0"));
+        assertThat(routeUnits.get(0).getTableMappers().size(), is(1));
+        assertThat(routeUnits.get(0).getTableMappers().iterator().next().getActualName(), is("t_order_1"));
+        assertThat(routeUnits.get(0).getTableMappers().iterator().next().getLogicName(), is("t_order"));
+    }
+    
+    @Test
+    public void assertForceRouteTableUseHintByShardingConditions() {
+        ShardingStandardRoutingEngine standardRoutingEngine = createShardingStandardRoutingEngine("t_order", createShardingConditions("t_order"));
+        HintManager hintManager = HintManager.getInstance();
+        hintManager.addTableShardingValue("t_order", "t_order_0");
+        RouteContext routeContext = standardRoutingEngine.route(createBasedShardingRule());
+        List<RouteUnit> routeUnits = new ArrayList<>(routeContext.getRouteUnits());
+        assertThat(routeContext.getRouteUnits().size(), is(1));
+        assertThat(routeUnits.get(0).getDataSourceMapper().getActualName(), is("ds_1"));
+        assertThat(routeUnits.get(0).getTableMappers().size(), is(1));
+        assertThat(routeUnits.get(0).getTableMappers().iterator().next().getActualName(), is("t_order_0"));
+        assertThat(routeUnits.get(0).getTableMappers().iterator().next().getLogicName(), is("t_order"));
+    }
+    
+    @Test
+    public void assertForceRouteDbAndTableUseHintByShardingConditions() {
+        ShardingStandardRoutingEngine standardRoutingEngine = createShardingStandardRoutingEngine("t_order", createShardingConditions("t_order"));
+        HintManager hintManager = HintManager.getInstance();
+        hintManager.addDatabaseShardingValue("t_order", "ds_0");
+        hintManager.addTableShardingValue("t_order", "t_order_0");
+        RouteContext routeContext = standardRoutingEngine.route(createBasedShardingRule());
+        List<RouteUnit> routeUnits = new ArrayList<>(routeContext.getRouteUnits());
+        assertThat(routeContext.getRouteUnits().size(), is(1));
+        assertThat(routeUnits.get(0).getDataSourceMapper().getActualName(), is("ds_0"));
+        assertThat(routeUnits.get(0).getTableMappers().size(), is(1));
+        assertThat(routeUnits.get(0).getTableMappers().iterator().next().getActualName(), is("t_order_0"));
+        assertThat(routeUnits.get(0).getTableMappers().iterator().next().getLogicName(), is("t_order"));
+    }
+    
     @Test(expected = ShardingSphereException.class)
     public void assertRouteByErrorShardingTableStrategy() {
         ShardingStandardRoutingEngine standardRoutingEngine = createShardingStandardRoutingEngine("t_order", createErrorShardingConditions("t_order"));