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/05/31 12:30:47 UTC

[shardingsphere] branch master updated: Add actualTablePrefix to support scaling create new sharding rule with different actual table (#18104)

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

zhonghongsheng 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 13bf2e54292 Add actualTablePrefix to support scaling create new sharding rule with different actual table  (#18104)
13bf2e54292 is described below

commit 13bf2e5429231cc402a22a9191cdbaebb23a5322
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Tue May 31 20:30:41 2022 +0800

    Add actualTablePrefix to support scaling create new sharding rule with different actual table  (#18104)
---
 .../rule/ShardingAutoTableRuleConfiguration.java   |  2 ++
 .../rule/ShardingTableRuleConfiguration.java       |  2 ++
 .../shardingsphere/sharding/rule/ShardingRule.java | 36 ++++++++++++++++++--
 .../shardingsphere/sharding/rule/TableRule.java    | 38 +++++++++++++++-------
 .../YamlShardingAutoTableRuleConfiguration.java    |  2 ++
 .../config/rule/YamlTableRuleConfiguration.java    |  2 ++
 ...rdingAutoTableRuleConfigurationYamlSwapper.java |  2 ++
 .../ShardingTableRuleConfigurationYamlSwapper.java |  4 ++-
 .../rewrite/token/pojo/CursorTokenTest.java        |  2 +-
 .../sharding/rule/TableRuleTest.java               | 36 ++++++++++++++++++++
 ...gAutoTableRuleConfigurationYamlSwapperTest.java |  2 ++
 11 files changed, 113 insertions(+), 15 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/api/config/rule/ShardingAutoTableRuleConfiguration.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/api/config/rule/ShardingAutoTableRuleConfiguration.java
index d081b0f829c..5fda87e8fb1 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/api/config/rule/ShardingAutoTableRuleConfiguration.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/api/config/rule/ShardingAutoTableRuleConfiguration.java
@@ -35,6 +35,8 @@ public final class ShardingAutoTableRuleConfiguration {
     
     private final String actualDataSources;
     
+    private String actualTablePrefix;
+    
     private ShardingStrategyConfiguration shardingStrategy;
     
     private KeyGenerateStrategyConfiguration keyGenerateStrategy;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/api/config/rule/ShardingTableRuleConfiguration.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/api/config/rule/ShardingTableRuleConfiguration.java
index 641fbd06cf8..690e8fdda4d 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/api/config/rule/ShardingTableRuleConfiguration.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/api/config/rule/ShardingTableRuleConfiguration.java
@@ -35,6 +35,8 @@ public final class ShardingTableRuleConfiguration {
     
     private final String actualDataNodes;
     
+    private String actualTablePrefix;
+    
     private ShardingStrategyConfiguration databaseShardingStrategy;
     
     private ShardingStrategyConfiguration tableShardingStrategy;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
index 27886791b3a..fcc0f4a26e4 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
@@ -19,11 +19,13 @@ package org.apache.shardingsphere.sharding.rule;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Splitter;
+import com.google.common.base.Strings;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.InstanceAwareAlgorithm;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.exception.ShardingSphereConfigurationException;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
 import org.apache.shardingsphere.infra.datanode.DataNode;
@@ -84,6 +86,8 @@ public final class ShardingRule implements SchemaRule, DataNodeContainedRule, Ta
     
     private static final String EQUAL = "=";
     
+    private static final String ALGORITHM_EXPRESSION_KEY = "algorithm-expression";
+    
     private final Collection<String> dataSourceNames;
     
     private final Map<String, ShardingAlgorithm> shardingAlgorithms = new LinkedHashMap<>();
@@ -108,7 +112,7 @@ public final class ShardingRule implements SchemaRule, DataNodeContainedRule, Ta
     
     public ShardingRule(final ShardingRuleConfiguration config, final Collection<String> dataSourceNames) {
         this.dataSourceNames = getDataSourceNames(config.getTables(), config.getAutoTables(), dataSourceNames);
-        config.getShardingAlgorithms().forEach((key, value) -> shardingAlgorithms.put(key, ShardingAlgorithmFactory.newInstance(value)));
+        config.getShardingAlgorithms().forEach((key, value) -> shardingAlgorithms.put(key, createShardingAlgorithm(key, value, config.getTables(), config.getAutoTables())));
         config.getKeyGenerators().forEach((key, value) -> keyGenerators.put(key, KeyGenerateAlgorithmFactory.newInstance(value)));
         tableRules.putAll(createTableRules(config.getTables(), config.getDefaultKeyGenerateStrategy()));
         tableRules.putAll(createAutoTableRules(config.getAutoTables(), config.getDefaultKeyGenerateStrategy()));
@@ -148,7 +152,7 @@ public final class ShardingRule implements SchemaRule, DataNodeContainedRule, Ta
     
     @Override
     public boolean check(final RuleConfiguration ruleConfig, final Collection<String> dataSourceNames) {
-        if (null == ruleConfig || !(ruleConfig instanceof ShardingRuleConfiguration)) {
+        if (!(ruleConfig instanceof ShardingRuleConfiguration)) {
             return true;
         }
         ShardingRuleConfiguration config = (ShardingRuleConfiguration) ruleConfig;
@@ -807,4 +811,32 @@ public final class ShardingRule implements SchemaRule, DataNodeContainedRule, Ta
             ((InstanceAwareAlgorithm) defaultKeyGenerateAlgorithm).setInstanceContext(instanceContext);
         }
     }
+    
+    private ShardingAlgorithm createShardingAlgorithm(final String name, final ShardingSphereAlgorithmConfiguration config, final Collection<ShardingTableRuleConfiguration> tables,
+                                                      final Collection<ShardingAutoTableRuleConfiguration> autoTables) {
+        Map<String, String> algorithmTablePrefixMap = getAlgorithmTablePrefixMap(tables, autoTables);
+        if (algorithmTablePrefixMap.containsKey(name)) {
+            String algorithmExpression = config.getProps().getProperty(ALGORITHM_EXPRESSION_KEY);
+            String actualTablePrefix = algorithmTablePrefixMap.get(name);
+            if (!Strings.isNullOrEmpty(algorithmExpression) && !algorithmExpression.startsWith(actualTablePrefix)) {
+                config.getProps().setProperty(ALGORITHM_EXPRESSION_KEY, actualTablePrefix + algorithmExpression);
+            }
+        }
+        return ShardingAlgorithmFactory.newInstance(config);
+    }
+    
+    private Map<String, String> getAlgorithmTablePrefixMap(final Collection<ShardingTableRuleConfiguration> tables, final Collection<ShardingAutoTableRuleConfiguration> autoTables) {
+        Map<String, String> result = new LinkedHashMap<>(tables.size() + autoTables.size(), 1);
+        for (ShardingTableRuleConfiguration each : tables) {
+            if (null != each.getActualTablePrefix() && null != each.getTableShardingStrategy()) {
+                result.put(each.getTableShardingStrategy().getShardingAlgorithmName(), each.getActualTablePrefix());
+            }
+        }
+        for (ShardingAutoTableRuleConfiguration each : autoTables) {
+            if (null != each.getActualTablePrefix() && null != each.getShardingStrategy()) {
+                result.put(each.getShardingStrategy().getShardingAlgorithmName(), each.getActualTablePrefix());
+            }
+        }
+        return result;
+    }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/TableRule.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/TableRule.java
index 0ef697e0ffc..6a17a78c962 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/TableRule.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/TableRule.java
@@ -26,13 +26,13 @@ import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.datanode.DataNodeInfo;
 import org.apache.shardingsphere.infra.datanode.DataNodeUtil;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.expr.InlineExpressionParser;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.NoneShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.sharding.ShardingAutoTableAlgorithm;
-import org.apache.shardingsphere.infra.expr.InlineExpressionParser;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -53,7 +53,7 @@ import java.util.stream.Collectors;
  * Table rule.
  */
 @Getter
-@ToString(exclude = {"dataNodeIndexMap", "actualTables", "actualDatasourceNames", "datasourceToTablesMap", "dataSourceDataNode", "tableDataNode"})
+@ToString(exclude = {"dataNodeIndexMap", "actualTables", "actualTablePrefix", "actualDatasourceNames", "datasourceToTablesMap", "dataSourceDataNode", "tableDataNode"})
 public final class TableRule {
     
     private static final Pattern DATA_NODE_SUFFIX_PATTERN = Pattern.compile("\\d+$");
@@ -64,6 +64,8 @@ public final class TableRule {
     
     private final List<DataNode> actualDataNodes;
     
+    private final String actualTablePrefix;
+    
     @Getter(AccessLevel.NONE)
     private final Set<String> actualTables;
     
@@ -90,7 +92,8 @@ public final class TableRule {
     public TableRule(final Collection<String> dataSourceNames, final String logicTableName) {
         logicTable = logicTableName;
         dataNodeIndexMap = new HashMap<>(dataSourceNames.size(), 1);
-        actualDataNodes = generateDataNodes(logicTableName, dataSourceNames);
+        actualTablePrefix = null;
+        actualDataNodes = generateDataNodes(logicTableName, dataSourceNames, null);
         actualTables = getActualTables();
         databaseShardingStrategyConfig = null;
         tableShardingStrategyConfig = null;
@@ -104,7 +107,9 @@ public final class TableRule {
         logicTable = tableRuleConfig.getLogicTable();
         List<String> dataNodes = new InlineExpressionParser(tableRuleConfig.getActualDataNodes()).splitAndEvaluate();
         dataNodeIndexMap = new HashMap<>(dataNodes.size(), 1);
-        actualDataNodes = isEmptyDataNodes(dataNodes) ? generateDataNodes(tableRuleConfig.getLogicTable(), dataSourceNames) : generateDataNodes(dataNodes, dataSourceNames);
+        actualTablePrefix = tableRuleConfig.getActualTablePrefix();
+        actualDataNodes = isEmptyDataNodes(dataNodes) ? generateDataNodes(tableRuleConfig.getLogicTable(), dataSourceNames, actualTablePrefix)
+                : generateDataNodes(dataNodes, dataSourceNames, actualTablePrefix);
         actualTables = getActualTables();
         databaseShardingStrategyConfig = tableRuleConfig.getDatabaseShardingStrategy();
         tableShardingStrategyConfig = tableRuleConfig.getTableShardingStrategy();
@@ -116,14 +121,16 @@ public final class TableRule {
         checkRule(dataNodes);
     }
     
-    public TableRule(final ShardingAutoTableRuleConfiguration tableRuleConfig,
-                     final Collection<String> dataSourceNames, final ShardingAutoTableAlgorithm shardingAutoTableAlgorithm, final String defaultGenerateKeyColumn) {
+    public TableRule(final ShardingAutoTableRuleConfiguration tableRuleConfig, final Collection<String> dataSourceNames,
+                     final ShardingAutoTableAlgorithm shardingAutoTableAlgorithm, final String defaultGenerateKeyColumn) {
         logicTable = tableRuleConfig.getLogicTable();
         databaseShardingStrategyConfig = new NoneShardingStrategyConfiguration();
         tableShardingStrategyConfig = tableRuleConfig.getShardingStrategy();
         List<String> dataNodes = getDataNodes(tableRuleConfig, shardingAutoTableAlgorithm, dataSourceNames);
         dataNodeIndexMap = new HashMap<>(dataNodes.size(), 1);
-        actualDataNodes = isEmptyDataNodes(dataNodes) ? generateDataNodes(tableRuleConfig.getLogicTable(), dataSourceNames) : generateDataNodes(dataNodes, dataSourceNames);
+        actualTablePrefix = tableRuleConfig.getActualTablePrefix();
+        actualDataNodes = isEmptyDataNodes(dataNodes) ? generateDataNodes(tableRuleConfig.getLogicTable(), dataSourceNames, actualTablePrefix)
+                : generateDataNodes(dataNodes, dataSourceNames, actualTablePrefix);
         actualTables = getActualTables();
         KeyGenerateStrategyConfiguration keyGeneratorConfig = tableRuleConfig.getKeyGenerateStrategy();
         generateKeyColumn = null != keyGeneratorConfig && !Strings.isNullOrEmpty(keyGeneratorConfig.getColumn()) ? keyGeneratorConfig.getColumn() : defaultGenerateKeyColumn;
@@ -178,11 +185,12 @@ public final class TableRule {
         return null == dataNodes || dataNodes.isEmpty();
     }
     
-    private List<DataNode> generateDataNodes(final String logicTable, final Collection<String> dataSourceNames) {
+    private List<DataNode> generateDataNodes(final String logicTable, final Collection<String> dataSourceNames, final String actualTablePrefix) {
         List<DataNode> result = new LinkedList<>();
         int index = 0;
         for (String each : dataSourceNames) {
-            DataNode dataNode = new DataNode(each, logicTable);
+            String actualTable = Strings.isNullOrEmpty(actualTablePrefix) ? logicTable : actualTablePrefix + logicTable;
+            DataNode dataNode = new DataNode(each, actualTable);
             result.add(dataNode);
             dataNodeIndexMap.put(dataNode, index);
             actualDatasourceNames.add(each);
@@ -192,11 +200,11 @@ public final class TableRule {
         return result;
     }
     
-    private List<DataNode> generateDataNodes(final List<String> actualDataNodes, final Collection<String> dataSourceNames) {
+    private List<DataNode> generateDataNodes(final List<String> actualDataNodes, final Collection<String> dataSourceNames, final String actualTablePrefix) {
         List<DataNode> result = new LinkedList<>();
         int index = 0;
         for (String each : actualDataNodes) {
-            DataNode dataNode = new DataNode(each);
+            DataNode dataNode = generateDataNode(each, actualTablePrefix);
             if (!dataSourceNames.contains(dataNode.getDataSourceName())) {
                 throw new ShardingSphereException("Cannot find data source in sharding rule, invalid actual data node is: '%s'", each);
             }
@@ -209,6 +217,14 @@ public final class TableRule {
         return result;
     }
     
+    private DataNode generateDataNode(final String dataNode, final String actualTablePrefix) {
+        DataNode result = new DataNode(dataNode);
+        if (Strings.isNullOrEmpty(actualTablePrefix)) {
+            return result;
+        }
+        return new DataNode(result.getDataSourceName(), actualTablePrefix + result.getTableName());
+    }
+    
     /**
      * Get data node groups.
      *
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/config/rule/YamlShardingAutoTableRuleConfiguration.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/config/rule/YamlShardingAutoTableRuleConfiguration.java
index 76eaeaccc01..14d8ac3f5f4 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/config/rule/YamlShardingAutoTableRuleConfiguration.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/config/rule/YamlShardingAutoTableRuleConfiguration.java
@@ -34,6 +34,8 @@ public final class YamlShardingAutoTableRuleConfiguration implements YamlConfigu
     
     private String actualDataSources;
     
+    private String actualTablePrefix;
+    
     private YamlShardingStrategyConfiguration shardingStrategy;
     
     private YamlKeyGenerateStrategyConfiguration keyGenerateStrategy;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/config/rule/YamlTableRuleConfiguration.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/config/rule/YamlTableRuleConfiguration.java
index c1a86890df5..abeb4ac9a6d 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/config/rule/YamlTableRuleConfiguration.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/config/rule/YamlTableRuleConfiguration.java
@@ -34,6 +34,8 @@ public final class YamlTableRuleConfiguration implements YamlConfiguration {
     
     private String actualDataNodes;
     
+    private String actualTablePrefix;
+    
     private YamlShardingStrategyConfiguration databaseStrategy;
     
     private YamlShardingStrategyConfiguration tableStrategy;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/swapper/rule/ShardingAutoTableRuleConfigurationYamlSwapper.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/swapper/rule/ShardingAutoTableRuleConfigurationYamlSwapper.java
index 100be36a701..a6a903cc064 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/swapper/rule/ShardingAutoTableRuleConfigurationYamlSwapper.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/swapper/rule/ShardingAutoTableRuleConfigurationYamlSwapper.java
@@ -38,6 +38,7 @@ public final class ShardingAutoTableRuleConfigurationYamlSwapper implements Yaml
         YamlShardingAutoTableRuleConfiguration result = new YamlShardingAutoTableRuleConfiguration();
         result.setLogicTable(data.getLogicTable());
         result.setActualDataSources(data.getActualDataSources());
+        result.setActualTablePrefix(data.getActualTablePrefix());
         if (null != data.getShardingStrategy()) {
             result.setShardingStrategy(shardingStrategyYamlSwapper.swapToYamlConfiguration(data.getShardingStrategy()));
         }
@@ -51,6 +52,7 @@ public final class ShardingAutoTableRuleConfigurationYamlSwapper implements Yaml
     public ShardingAutoTableRuleConfiguration swapToObject(final YamlShardingAutoTableRuleConfiguration yamlConfig) {
         Preconditions.checkNotNull(yamlConfig.getLogicTable(), "Logic table cannot be null.");
         ShardingAutoTableRuleConfiguration result = new ShardingAutoTableRuleConfiguration(yamlConfig.getLogicTable(), yamlConfig.getActualDataSources());
+        result.setActualTablePrefix(yamlConfig.getActualTablePrefix());
         if (null != yamlConfig.getShardingStrategy()) {
             result.setShardingStrategy(shardingStrategyYamlSwapper.swapToObject(yamlConfig.getShardingStrategy()));
         }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/swapper/rule/ShardingTableRuleConfigurationYamlSwapper.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/swapper/rule/ShardingTableRuleConfigurationYamlSwapper.java
index ee023a65c44..bbc7731888e 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/swapper/rule/ShardingTableRuleConfigurationYamlSwapper.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/yaml/swapper/rule/ShardingTableRuleConfigurationYamlSwapper.java
@@ -18,9 +18,9 @@
 package org.apache.shardingsphere.sharding.yaml.swapper.rule;
 
 import com.google.common.base.Preconditions;
+import org.apache.shardingsphere.infra.yaml.config.swapper.YamlConfigurationSwapper;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.yaml.config.rule.YamlTableRuleConfiguration;
-import org.apache.shardingsphere.infra.yaml.config.swapper.YamlConfigurationSwapper;
 import org.apache.shardingsphere.sharding.yaml.swapper.strategy.KeyGenerateStrategyConfigurationYamlSwapper;
 import org.apache.shardingsphere.sharding.yaml.swapper.strategy.ShardingStrategyConfigurationYamlSwapper;
 
@@ -38,6 +38,7 @@ public final class ShardingTableRuleConfigurationYamlSwapper implements YamlConf
         YamlTableRuleConfiguration result = new YamlTableRuleConfiguration();
         result.setLogicTable(data.getLogicTable());
         result.setActualDataNodes(data.getActualDataNodes());
+        result.setActualTablePrefix(data.getActualTablePrefix());
         if (null != data.getDatabaseShardingStrategy()) {
             result.setDatabaseStrategy(shardingStrategyYamlSwapper.swapToYamlConfiguration(data.getDatabaseShardingStrategy()));
         }
@@ -54,6 +55,7 @@ public final class ShardingTableRuleConfigurationYamlSwapper implements YamlConf
     public ShardingTableRuleConfiguration swapToObject(final YamlTableRuleConfiguration yamlConfig) {
         Preconditions.checkNotNull(yamlConfig.getLogicTable(), "Logic table cannot be null.");
         ShardingTableRuleConfiguration result = new ShardingTableRuleConfiguration(yamlConfig.getLogicTable(), yamlConfig.getActualDataNodes());
+        result.setActualTablePrefix(yamlConfig.getActualTablePrefix());
         if (null != yamlConfig.getDatabaseStrategy()) {
             result.setDatabaseShardingStrategy(shardingStrategyYamlSwapper.swapToObject(yamlConfig.getDatabaseStrategy()));
         }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/CursorTokenTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/CursorTokenTest.java
index d5225ce4b38..6ccad6dedd9 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/CursorTokenTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/CursorTokenTest.java
@@ -37,7 +37,7 @@ public final class CursorTokenTest {
     
     @Test
     public void assertToString() {
-        CursorToken cursorToken = new CursorToken(0, 0, 
+        CursorToken cursorToken = new CursorToken(0, 0,
                 new IdentifierValue("t_order_cursor"), mock(CursorStatementContext.class, RETURNS_DEEP_STUBS), mock(ShardingRule.class));
         RouteUnit routeUnit = mock(RouteUnit.class);
         when(routeUnit.getTableMappers()).thenReturn(Collections.singletonList(new RouteMapper("t_order", "t_order_0")));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/TableRuleTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/TableRuleTest.java
index 2dcf32cf38c..6eace0e0b1a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/TableRuleTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/TableRuleTest.java
@@ -178,4 +178,40 @@ public final class TableRuleTest {
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds0", "table_0")));
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds1", "table_0")));
     }
+    
+    @Test
+    public void assertDatNodeGroupsWhenShardingTableConfigActualTablePrefix() {
+        ShardingTableRuleConfiguration shardingTableRuleConfig = new ShardingTableRuleConfiguration("t_order", "ds_${0..1}.t_order_${0..1}");
+        shardingTableRuleConfig.setActualTablePrefix("tmp_");
+        TableRule tableRule = new TableRule(shardingTableRuleConfig, Arrays.asList("ds_0", "ds_1"), "order_id");
+        Map<String, List<DataNode>> actual = tableRule.getDataNodeGroups();
+        assertThat(actual.size(), is(2));
+        assertTrue(actual.get("ds_0").contains(new DataNode("ds_0", "tmp_t_order_0")));
+        assertTrue(actual.get("ds_0").contains(new DataNode("ds_0", "tmp_t_order_1")));
+        assertTrue(actual.get("ds_1").contains(new DataNode("ds_1", "tmp_t_order_0")));
+        assertTrue(actual.get("ds_1").contains(new DataNode("ds_1", "tmp_t_order_1")));
+    }
+    
+    @Test
+    public void assertDatNodeGroupsWhenShardingAutoTableConfigActualTablePrefix() {
+        ShardingAutoTableRuleConfiguration shardingTableRuleConfig = new ShardingAutoTableRuleConfiguration("t_order", "ds_${0..1}");
+        shardingTableRuleConfig.setActualTablePrefix("tmp_");
+        shardingTableRuleConfig.setShardingStrategy(new StandardShardingStrategyConfiguration("order_id", "mod"));
+        ModShardingAlgorithm modShardingAlgorithm = createModShardingAlgorithm();
+        TableRule tableRule = new TableRule(shardingTableRuleConfig, Arrays.asList("ds_0", "ds_1"), modShardingAlgorithm, "order_id");
+        Map<String, List<DataNode>> actual = tableRule.getDataNodeGroups();
+        assertThat(actual.size(), is(2));
+        assertTrue(actual.get("ds_0").contains(new DataNode("ds_0", "tmp_t_order_0")));
+        assertTrue(actual.get("ds_0").contains(new DataNode("ds_0", "tmp_t_order_2")));
+        assertTrue(actual.get("ds_1").contains(new DataNode("ds_1", "tmp_t_order_1")));
+        assertTrue(actual.get("ds_1").contains(new DataNode("ds_1", "tmp_t_order_3")));
+    }
+    
+    private ModShardingAlgorithm createModShardingAlgorithm() {
+        ModShardingAlgorithm result = new ModShardingAlgorithm();
+        Properties props = new Properties();
+        props.setProperty("sharding-count", "4");
+        result.init(props);
+        return result;
+    }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/swapper/rule/ShardingAutoTableRuleConfigurationYamlSwapperTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/swapper/rule/ShardingAutoTableRuleConfigurationYamlSwapperTest.java
index 08d2790d3fa..6962c95eb46 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/swapper/rule/ShardingAutoTableRuleConfigurationYamlSwapperTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/swapper/rule/ShardingAutoTableRuleConfigurationYamlSwapperTest.java
@@ -80,11 +80,13 @@ public final class ShardingAutoTableRuleConfigurationYamlSwapperTest {
     @Test
     public void assertSwapToYamlWithMaxProperties() {
         ShardingAutoTableRuleConfiguration shardingTableRuleConfig = new ShardingAutoTableRuleConfiguration("tbl", "ds0,ds1");
+        shardingTableRuleConfig.setActualTablePrefix("tmp_");
         shardingTableRuleConfig.setShardingStrategy(mock(StandardShardingStrategyConfiguration.class));
         shardingTableRuleConfig.setKeyGenerateStrategy(mock(KeyGenerateStrategyConfiguration.class));
         YamlShardingAutoTableRuleConfiguration actual = tableYamlSwapper.swapToYamlConfiguration(shardingTableRuleConfig);
         assertThat(actual.getLogicTable(), is("tbl"));
         assertThat(actual.getActualDataSources(), is("ds0,ds1"));
+        assertThat(actual.getActualTablePrefix(), is("tmp_"));
         assertNotNull(actual.getShardingStrategy());
         assertNotNull(actual.getKeyGenerateStrategy());
     }