You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2021/09/01 11:12:28 UTC

[shardingsphere] branch master updated: Make `CREATE SHARDING TABLE RULE` synax support inline expression to config `RESOURCES`. (#12109)

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

panjuan 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 a2bc559  Make `CREATE SHARDING TABLE RULE` synax support inline expression to config `RESOURCES`. (#12109)
a2bc559 is described below

commit a2bc5595f15ba790260625bc16ed6116c2b793a6
Author: Raigor <ra...@gmail.com>
AuthorDate: Wed Sep 1 19:11:30 2021 +0800

    Make `CREATE SHARDING TABLE RULE` synax support inline expression to config `RESOURCES`. (#12109)
    
    * Fixes #12102, make `CREATE SHARDING TABLE RULE` synax support inline expression to config `RESOURCES`.
    
    * Fix AlterShardingTableRuleStatementUpdater.
    
    * fix checkStyle
    
    * add test case.
    
    * add test cases.
    
    * Simplify the parse logic of autoTable resources.
    
    * remove class InlineExpressionUtil.
---
 .../sharding/support}/InlineExpressionParser.java   | 12 +++++++++++-
 .../complex/ComplexInlineShardingAlgorithm.java     |  2 +-
 .../sharding/hint/HintInlineShardingAlgorithm.java  |  2 +-
 .../sharding/inline/InlineShardingAlgorithm.java    |  1 +
 .../shardingsphere/sharding/rule/ShardingRule.java  |  8 +++-----
 .../shardingsphere/sharding/rule/TableRule.java     |  2 +-
 .../sharding/inline/InlineExpressionParserTest.java |  1 +
 .../sharding/rule/ShardingRuleTest.java             | 13 +++++++++++++
 .../AlterShardingTableRuleStatementUpdater.java     |  9 ++++++++-
 .../CreateShardingTableRuleStatementUpdater.java    |  9 ++++++++-
 .../AlterShardingTableRuleStatementUpdaterTest.java | 10 ++++++++++
 ...CreateShardingTableRuleStatementUpdaterTest.java |  6 ++++++
 .../main/antlr4/imports/sharding/RDLStatement.g4    |  6 +++++-
 .../core/ShardingDistSQLStatementVisitor.java       | 21 ++++++++++-----------
 .../spring/boot/datasource/DataSourceMapSetter.java |  2 +-
 .../scaling/core/util/JobConfigurationUtil.java     |  2 +-
 .../test/integration/cases/dataset/DataSet.java     |  2 +-
 .../env/dataset/DataSetEnvironmentManager.java      |  2 +-
 .../test/integration/engine/it/BatchITCase.java     |  2 +-
 .../test/integration/engine/it/ddl/BaseDDLIT.java   |  2 +-
 .../test/integration/engine/it/dml/BaseDMLIT.java   |  2 +-
 .../src/main/resources/case/rdl/alter.xml           | 16 ++++++++++++++++
 .../src/main/resources/case/rdl/create.xml          | 16 ++++++++++++++++
 .../src/main/resources/sql/supported/rdl/alter.xml  |  1 +
 .../src/main/resources/sql/supported/rdl/create.xml |  1 +
 25 files changed, 120 insertions(+), 30 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineExpressionParser.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/support/InlineExpressionParser.java
similarity index 94%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineExpressionParser.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/support/InlineExpressionParser.java
index 590c362..19d4bfe 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineExpressionParser.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/support/InlineExpressionParser.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.algorithm.sharding.inline;
+package org.apache.shardingsphere.sharding.support;
 
 import com.google.common.base.Strings;
 import com.google.common.collect.Collections2;
@@ -49,6 +49,16 @@ public final class InlineExpressionParser {
     private final String inlineExpression;
     
     /**
+     * Judge whether the expression is an inline expression.
+     *
+     * @param expression expression to be judged
+     * @return whether the expression is an inline expression
+     */
+    public static boolean isInlineExpression(final String expression) {
+        return expression.contains("${") || expression.contains("$->{");
+    }
+    
+    /**
      * Replace all inline expression placeholders.
      * 
      * @param inlineExpression inline expression with {@code $->}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/complex/ComplexInlineShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/complex/ComplexInlineShardingAlgorithm.java
index a4ac2ae..9ce8b09 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/complex/ComplexInlineShardingAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/complex/ComplexInlineShardingAlgorithm.java
@@ -22,7 +22,7 @@ import groovy.lang.Closure;
 import groovy.util.Expando;
 import lombok.Getter;
 import lombok.Setter;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 import org.apache.shardingsphere.sharding.api.sharding.complex.ComplexKeysShardingAlgorithm;
 import org.apache.shardingsphere.sharding.api.sharding.complex.ComplexKeysShardingValue;
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/hint/HintInlineShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/hint/HintInlineShardingAlgorithm.java
index a354c8a..79d5d9b 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/hint/HintInlineShardingAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/hint/HintInlineShardingAlgorithm.java
@@ -22,7 +22,7 @@ import groovy.lang.Closure;
 import groovy.util.Expando;
 import lombok.Getter;
 import lombok.Setter;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 import org.apache.shardingsphere.sharding.api.sharding.hint.HintShardingAlgorithm;
 import org.apache.shardingsphere.sharding.api.sharding.hint.HintShardingValue;
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineShardingAlgorithm.java
index 371758e..ffeb8e3 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineShardingAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineShardingAlgorithm.java
@@ -25,6 +25,7 @@ import lombok.Setter;
 import org.apache.shardingsphere.sharding.api.sharding.standard.PreciseShardingValue;
 import org.apache.shardingsphere.sharding.api.sharding.standard.RangeShardingValue;
 import org.apache.shardingsphere.sharding.api.sharding.standard.StandardShardingAlgorithm;
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 
 import java.util.Collection;
 import java.util.Properties;
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 9c35bd5..a1ee6fc 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,7 +19,6 @@ 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.config.algorithm.ShardingSphereAlgorithmFactory;
 import org.apache.shardingsphere.infra.config.exception.ShardingSphereConfigurationException;
@@ -29,7 +28,7 @@ import org.apache.shardingsphere.infra.rule.identifier.scope.SchemaRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
 import org.apache.shardingsphere.sharding.algorithm.config.AlgorithmProvidedShardingRuleConfiguration;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
@@ -46,7 +45,6 @@ import org.apache.shardingsphere.spi.required.RequiredSPIRegistry;
 
 import javax.sql.DataSource;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
@@ -137,8 +135,8 @@ public final class ShardingRule implements FeatureRule, SchemaRule, DataNodeCont
     }
     
     private Collection<String> getDataSourceNames(final ShardingAutoTableRuleConfiguration shardingAutoTableRuleConfig) {
-        return Strings.isNullOrEmpty(shardingAutoTableRuleConfig.getActualDataSources()) 
-                ? Collections.emptyList() : Splitter.on(",").trimResults().splitToList(shardingAutoTableRuleConfig.getActualDataSources());
+        List<String> actualDataSources = new InlineExpressionParser(shardingAutoTableRuleConfig.getActualDataSources()).splitAndEvaluate();
+        return actualDataSources.stream().collect(Collectors.toSet());
     }
     
     private Collection<String> getDataSourceNames(final ShardingTableRuleConfiguration shardingTableRuleConfig) {
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 b149473..a21f298 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
@@ -25,7 +25,7 @@ import org.apache.shardingsphere.infra.config.exception.ShardingSphereConfigurat
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.datanode.DataNodeUtil;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.support.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;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineExpressionParserTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineExpressionParserTest.java
index 38210d4..82d58ee 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineExpressionParserTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineExpressionParserTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.sharding.algorithm.sharding.inline;
 
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 import org.junit.Test;
 
 import java.util.Collections;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
index d2fa21d..9af4ff8 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
@@ -330,6 +330,19 @@ public final class ShardingRuleTest {
     }
     
     @Test
+    public void assertGetDataSourceNamesWithShardingAutoTablesAndInlineExpression() {
+        ShardingRuleConfiguration shardingRuleConfig = new ShardingRuleConfiguration();
+        ShardingAutoTableRuleConfiguration autoTableRuleConfig = new ShardingAutoTableRuleConfiguration("auto_table", "resource${0..1}");
+        autoTableRuleConfig.setShardingStrategy(new StandardShardingStrategyConfiguration("order_id", "hash_mod"));
+        shardingRuleConfig.getAutoTables().add(autoTableRuleConfig);
+        Properties props = new Properties();
+        props.put("sharding-count", 4);
+        shardingRuleConfig.getShardingAlgorithms().put("hash_mod", new ShardingSphereAlgorithmConfiguration("hash_mod", props));
+        ShardingRule shardingRule = new ShardingRule(shardingRuleConfig, createDataSourceMap());
+        assertThat(shardingRule.getDataSourceNames(), is(new LinkedHashSet<>(Arrays.asList("resource0", "resource1"))));
+    }
+    
+    @Test
     public void assertGetDataSourceNamesWithoutShardingTablesAndShardingAutoTables() {
         ShardingRuleConfiguration shardingRuleConfig = new ShardingRuleConfiguration();
         ShardingRule shardingRule = new ShardingRule(shardingRuleConfig, createDataSourceMap());
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/AlterShardingTableRuleStatementUpdater.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/AlterShardingTableRuleStatementUpdater.java
index cb30dec..8a5dedc 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/AlterShardingTableRuleStatementUpdater.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/AlterShardingTableRuleStatementUpdater.java
@@ -34,6 +34,7 @@ import org.apache.shardingsphere.sharding.distsql.parser.segment.TableRuleSegmen
 import org.apache.shardingsphere.sharding.distsql.parser.statement.AlterShardingTableRuleStatement;
 import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
 import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.spi.typed.TypedSPIRegistry;
 
@@ -87,7 +88,13 @@ public final class AlterShardingTableRuleStatementUpdater implements RuleDefinit
     
     private Collection<String> getToBeAlteredResources(final AlterShardingTableRuleStatement sqlStatement) {
         Collection<String> result = new LinkedHashSet<>();
-        sqlStatement.getRules().forEach(each -> result.addAll(each.getDataSources()));
+        sqlStatement.getRules().forEach(each -> each.getDataSources().forEach(dataSource -> {
+            if (InlineExpressionParser.isInlineExpression(dataSource)) {
+                result.addAll(new InlineExpressionParser(dataSource).splitAndEvaluate());
+            } else {
+                result.add(dataSource);
+            }
+        }));
         return result;
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingTableRuleStatementUpdater.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingTableRuleStatementUpdater.java
index f9415d1..e891af6 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingTableRuleStatementUpdater.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingTableRuleStatementUpdater.java
@@ -34,6 +34,7 @@ import org.apache.shardingsphere.sharding.distsql.parser.segment.TableRuleSegmen
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingTableRuleStatement;
 import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
 import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.spi.typed.TypedSPIRegistry;
 
@@ -98,7 +99,13 @@ public final class CreateShardingTableRuleStatementUpdater implements RuleDefini
     
     private Collection<String> getToBeCreatedResources(final CreateShardingTableRuleStatement sqlStatement) {
         Collection<String> result = new LinkedHashSet<>();
-        sqlStatement.getRules().forEach(each -> result.addAll(each.getDataSources()));
+        sqlStatement.getRules().forEach(each -> each.getDataSources().forEach(dataSource -> {
+            if (InlineExpressionParser.isInlineExpression(dataSource)) {
+                result.addAll(new InlineExpressionParser(dataSource).splitAndEvaluate());
+            } else {
+                result.add(dataSource);
+            }
+        }));
         return result;
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/AlterShardingTableRuleStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/AlterShardingTableRuleStatementUpdaterTest.java
index a5ed5f8..9fbf87f 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/AlterShardingTableRuleStatementUpdaterTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/AlterShardingTableRuleStatementUpdaterTest.java
@@ -79,11 +79,21 @@ public final class AlterShardingTableRuleStatementUpdaterTest {
         updater.checkSQLStatement(shardingSphereMetaData, createSQLStatement("t_order", "INVALID_TYPE"), createCurrentRuleConfiguration());
     }
     
+    @Test
+    public void assertExecuteWithInlineExpression() throws DistSQLException {
+        TableRuleSegment ruleSegment = new TableRuleSegment("t_order", Arrays.asList("ds_${0..1}", "ds2"), "order_id", new AlgorithmSegment("MOD_TEST", new Properties()), null, null);
+        updater.checkSQLStatement(shardingSphereMetaData, createSQLStatement(ruleSegment), createCurrentRuleConfiguration());
+    }
+    
     private AlterShardingTableRuleStatement createSQLStatement(final String tableName, final String shardingAlgorithmName) {
         TableRuleSegment ruleSegment = new TableRuleSegment(tableName, Arrays.asList("ds_0", "ds_1"), "order_id", new AlgorithmSegment(shardingAlgorithmName, new Properties()), null, null);
         return new AlterShardingTableRuleStatement(Collections.singleton(ruleSegment));
     }
     
+    private AlterShardingTableRuleStatement createSQLStatement(final TableRuleSegment... ruleSegments) {
+        return new AlterShardingTableRuleStatement(Arrays.asList(ruleSegments));
+    }
+    
     private AlterShardingTableRuleStatement createDuplicatedSQLStatement() {
         TableRuleSegment ruleSegment = new TableRuleSegment("t_order", Arrays.asList("ds_0", "ds_1"), "order_id", new AlgorithmSegment("STANDARD_TEST", new Properties()), null, null);
         return new AlterShardingTableRuleStatement(Arrays.asList(ruleSegment, ruleSegment));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java
index dc91c36..818f7aa 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java
@@ -105,6 +105,12 @@ public final class CreateShardingTableRuleStatementUpdaterTest {
         updater.checkSQLStatement(shardingSphereMetaData, createSQLStatement(ruleSegment), null);
     }
     
+    @Test
+    public void assertExecuteWithInlineExpression() throws DistSQLException {
+        TableRuleSegment ruleSegment = new TableRuleSegment("t_order", Arrays.asList("ds_${0..1}", "ds2"), "order_id", new AlgorithmSegment("MOD_TEST", new Properties()), null, null);
+        updater.checkSQLStatement(shardingSphereMetaData, createSQLStatement(ruleSegment), null);
+    }
+    
     private Map<String, Collection<String>> getDataSourceMapper() {
         Map<String, Collection<String>> dataSourceMapper = new HashMap<>(2, 1);
         dataSourceMapper.put("ds0", null);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/RDLStatement.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/RDLStatement.g4
index c91eb99..e4f6e51 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/RDLStatement.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/RDLStatement.g4
@@ -64,7 +64,11 @@ shardingTableRuleDefinition
     ;
 
 resources
-    : RESOURCES LP IDENTIFIER (COMMA IDENTIFIER)* RP
+    : RESOURCES LP resource (COMMA resource)* RP
+    ;
+
+resource
+    : IDENTIFIER | STRING
     ;
 
 shardingColumn
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java
index 75cc48d..2f197d9 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java
@@ -19,28 +19,28 @@ package org.apache.shardingsphere.sharding.distsql.parser.core;
 
 import com.google.common.base.Joiner;
 import org.antlr.v4.runtime.tree.ParseTree;
-import org.antlr.v4.runtime.tree.TerminalNode;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementBaseVisitor;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.AddShardingHintDatabaseValueContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.AddShardingHintTableValueContext;
-import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ClearShardingHintContext;
-import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.SetShardingHintDatabaseValueContext;
-import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ShowShardingAlgorithmsContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.AlgorithmDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.AlgorithmPropertyContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.AlterShardingBindingTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.AlterShardingBroadcastTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.AlterShardingTableRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.BindTableRulesDefinitionContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ClearShardingHintContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateShardingBindingTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateShardingBroadcastTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateShardingTableRuleContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.DropShardingAlgorithmContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.DropShardingBindingTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.DropShardingBroadcastTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.DropShardingTableRuleContext;
-import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.DropShardingAlgorithmContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ResourcesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.SchemaNameContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.SetShardingHintDatabaseValueContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ShardingTableRuleDefinitionContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ShowShardingAlgorithmsContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ShowShardingBindingTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ShowShardingBroadcastTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ShowShardingHintStatusContext;
@@ -185,12 +185,7 @@ public final class ShardingDistSQLStatementVisitor extends ShardingDistSQLStatem
     
     @Override
     public ASTNode visitShardingTableRuleDefinition(final ShardingTableRuleDefinitionContext ctx) {
-        Collection<String> dataSources = new LinkedList<>();
-        if (null != ctx.resources()) {
-            for (TerminalNode each : ctx.resources().IDENTIFIER()) {
-                dataSources.add(new IdentifierValue(each.getText()).getValue());
-            }
-        }
+        Collection<String> dataSources = getResources(ctx.resources());
         String tableStrategyColumn = null;
         AlgorithmSegment tableStrategy = null;
         if (null != ctx.algorithmDefinition()) {
@@ -206,6 +201,10 @@ public final class ShardingDistSQLStatementVisitor extends ShardingDistSQLStatem
         return new TableRuleSegment(ctx.tableName().getText(), dataSources, tableStrategyColumn, tableStrategy, keyGenerateStrategyColumn, keyGenerateStrategy);
     }
     
+    private Collection<String> getResources(final ResourcesContext ctx) {
+        return ctx.resource().stream().map(each -> new IdentifierValue(each.getText()).getValue()).collect(Collectors.toList());
+    }
+    
     @Override
     public ASTNode visitAlgorithmDefinition(final AlgorithmDefinitionContext ctx) {
         return new AlgorithmSegment(ctx.algorithmName().getText(), getAlgorithmProperties(ctx));
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/DataSourceMapSetter.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/DataSourceMapSetter.java
index 2d28ae6..39b242f 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/DataSourceMapSetter.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/DataSourceMapSetter.java
@@ -21,7 +21,7 @@ import com.google.common.base.Preconditions;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 import org.apache.shardingsphere.spring.boot.datasource.prop.impl.DataSourcePropertiesSetterHolder;
 import org.apache.shardingsphere.spring.boot.util.DataSourceUtil;
 import org.apache.shardingsphere.spring.boot.util.PropertyUtil;
diff --git a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/util/JobConfigurationUtil.java b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/util/JobConfigurationUtil.java
index 4467d4f..c5c9bab 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/util/JobConfigurationUtil.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/util/JobConfigurationUtil.java
@@ -41,7 +41,7 @@ import org.apache.shardingsphere.scaling.core.config.datasource.ShardingSphereJD
 import org.apache.shardingsphere.scaling.core.config.datasource.StandardJDBCDataSourceConfiguration;
 import org.apache.shardingsphere.scaling.core.config.yaml.ShardingRuleConfigurationSwapper;
 import org.apache.shardingsphere.sharding.algorithm.keygen.SnowflakeKeyGenerateAlgorithm;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ComplexShardingStrategyConfiguration;
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-fixture/src/test/java/org/apache/shardingsphere/test/integration/cases/dataset/DataSet.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-fixture/src/test/java/org/apache/shardingsphere/test/integration/cases/dataset/DataSet.java
index 462ba32..c037100 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-fixture/src/test/java/org/apache/shardingsphere/test/integration/cases/dataset/DataSet.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-fixture/src/test/java/org/apache/shardingsphere/test/integration/cases/dataset/DataSet.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.test.integration.cases.dataset;
 
 import lombok.Getter;
 import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 import org.apache.shardingsphere.test.integration.cases.dataset.metadata.DataSetMetadata;
 import org.apache.shardingsphere.test.integration.cases.dataset.row.DataSetRow;
 
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-fixture/src/test/java/org/apache/shardingsphere/test/integration/env/dataset/DataSetEnvironmentManager.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-fixture/src/test/java/org/apache/shardingsphere/test/integration/env/dataset/DataSetEnvironmentManager.java
index 5703685..2977d97 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-fixture/src/test/java/org/apache/shardingsphere/test/integration/env/dataset/DataSetEnvironmentManager.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-fixture/src/test/java/org/apache/shardingsphere/test/integration/env/dataset/DataSetEnvironmentManager.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.executor.kernel.thread.ExecutorServiceManager;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 import org.apache.shardingsphere.test.integration.cases.dataset.DataSet;
 import org.apache.shardingsphere.test.integration.cases.dataset.metadata.DataSetColumn;
 import org.apache.shardingsphere.test.integration.cases.dataset.metadata.DataSetMetadata;
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/java/org/apache/shardingsphere/test/integration/engine/it/BatchITCase.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/java/org/apache/shardingsphere/test/integration/engine/it/BatchITCase.java
index 64357dd..b971ee0 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/java/org/apache/shardingsphere/test/integration/engine/it/BatchITCase.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/java/org/apache/shardingsphere/test/integration/engine/it/BatchITCase.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.test.integration.engine.it;
 import lombok.AccessLevel;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 import org.apache.shardingsphere.test.integration.cases.assertion.IntegrationTestCaseAssertion;
 import org.apache.shardingsphere.test.integration.cases.dataset.DataSet;
 import org.apache.shardingsphere.test.integration.cases.dataset.DataSetLoader;
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/java/org/apache/shardingsphere/test/integration/engine/it/ddl/BaseDDLIT.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/java/org/apache/shardingsphere/test/integration/engine/it/ddl/BaseDDLIT.java
index 808a7bd..bf00da0 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/java/org/apache/shardingsphere/test/integration/engine/it/ddl/BaseDDLIT.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/java/org/apache/shardingsphere/test/integration/engine/it/ddl/BaseDDLIT.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.test.integration.engine.it.ddl;
 import com.google.common.base.Splitter;
 import lombok.SneakyThrows;
 import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
 import org.apache.shardingsphere.test.integration.cases.dataset.metadata.DataSetColumn;
 import org.apache.shardingsphere.test.integration.cases.dataset.metadata.DataSetIndex;
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/java/org/apache/shardingsphere/test/integration/engine/it/dml/BaseDMLIT.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/java/org/apache/shardingsphere/test/integration/engine/it/dml/BaseDMLIT.java
index fc54962..fd51561 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/java/org/apache/shardingsphere/test/integration/engine/it/dml/BaseDMLIT.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/java/org/apache/shardingsphere/test/integration/engine/it/dml/BaseDMLIT.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.test.integration.engine.it.dml;
 import lombok.SneakyThrows;
 import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.support.InlineExpressionParser;
 import org.apache.shardingsphere.test.integration.cases.dataset.metadata.DataSetColumn;
 import org.apache.shardingsphere.test.integration.cases.dataset.metadata.DataSetMetadata;
 import org.apache.shardingsphere.test.integration.cases.dataset.row.DataSetRow;
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/alter.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/alter.xml
index 41d2e87..893cd5e 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/alter.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/alter.xml
@@ -34,6 +34,22 @@
         </rule>
     </alter-sharding-table-rule>
 
+    <alter-sharding-table-rule sql-case-id="alter-sharding-table-rule-with-inline-expression">
+        <rule name="t_order" table-strategy-column="order_id" key-generate-strategy-column="another_id">
+            <data-source>ms_group_${0..1}</data-source>
+            <table-strategy algorithm-name="hash_mod">
+                <properties>
+                    <property key="sharding-count" value="4"/>
+                </properties>
+            </table-strategy>
+            <key-generate-strategy algorithm-name="snowflake">
+                <properties>
+                    <property key="worker-id" value="123"/>
+                </properties>
+            </key-generate-strategy>
+        </rule>
+    </alter-sharding-table-rule>
+
     <alter-sharding-binding-table-rules sql-case-id="alter-sharding-binding-table-rules">
         <rule table-groups="t_order,t_order_item"/>
         <rule table-groups="t_1,t_2"/>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/create.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/create.xml
index 4e4d39f..1910895 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/create.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/create.xml
@@ -79,6 +79,22 @@
             </key-generate-strategy>
         </rule>
     </create-sharding-table-rule>
+
+    <create-sharding-table-rule sql-case-id="create-sharding-table-rule-with-inline-expression">
+        <rule name="t_order" table-strategy-column="order_id" key-generate-strategy-column="another_id">
+            <data-source>ms_group_${0..1}</data-source>
+            <table-strategy algorithm-name="hash_mod">
+                <properties>
+                    <property key="sharding-count" value="4"/>
+                </properties>
+            </table-strategy>
+            <key-generate-strategy algorithm-name="snowflake">
+                <properties>
+                    <property key="worker-id" value="123"/>
+                </properties>
+            </key-generate-strategy>
+        </rule>
+    </create-sharding-table-rule>
     
     <create-sharding-binding-table-rule sql-case-id="create-sharding-binding-table-rule">
         <rule table-groups="t_order,t_order_item"/>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/alter.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/alter.xml
index c46ccf6..fa13619 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/alter.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/alter.xml
@@ -18,6 +18,7 @@
 
 <sql-cases>
     <distsql-case id="alter-sharding-table-rule" value="ALTER SHARDING TABLE RULE t_order (RESOURCES(ms_group_0,ms_group_1), SHARDING_COLUMN=order_id, TYPE(NAME=hash_mod,PROPERTIES('sharding-count'=4)),GENERATED_KEY(COLUMN=another_id,TYPE(NAME=snowflake,PROPERTIES('worker-id'=123))))" />
+    <distsql-case id="alter-sharding-table-rule-with-inline-expression" value="ALTER SHARDING TABLE RULE t_order (RESOURCES('ms_group_${0..1}'), SHARDING_COLUMN=order_id, TYPE(NAME=hash_mod,PROPERTIES('sharding-count'=4)),GENERATED_KEY(COLUMN=another_id,TYPE(NAME=snowflake,PROPERTIES('worker-id'=123))))" />
     <distsql-case id="alter-sharding-binding-table-rules" value="ALTER SHARDING BINDING TABLE RULES ((t_order,t_order_item), (t_1,t_2))" />
     <distsql-case id="alter-sharding-broadcast-table-rules" value="ALTER SHARDING BROADCAST TABLE RULES(t_1,t_2)" />
     <distsql-case id="alter-readwrite-splitting-rule" value="ALTER READWRITE_SPLITTING RULE ms_group_0 (AUTO_AWARE_RESOURCE=group_0, TYPE(NAME=random,PROPERTIES(read_weight='2:1'))), ms_group_1 (WRITE_RESOURCE=primary_ds, READ_RESOURCES(replica_ds_0,replica_ds_1),TYPE(NAME=random))" />
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/create.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/create.xml
index 03e19b2..8a06595 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/create.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/create.xml
@@ -27,6 +27,7 @@
     <distsql-case id="add-resource-url-single-with-empty-properties" value="ADD RESOURCE ds_0(URL='jdbc:mysql://127.0.0.1:3306/test0',USER=ROOT,PROPERTIES())" />
     <distsql-case id="add-resource-url-single-with-properties" value="ADD RESOURCE ds_0(URL='jdbc:mysql://127.0.0.1:3306/test0',USER=ROOT,PASSWORD=123456,PROPERTIES('maxPoolSize'=30))" />
     <distsql-case id="create-sharding-table-rule" value="CREATE SHARDING TABLE RULE t_order (RESOURCES(ms_group_0,ms_group_1), SHARDING_COLUMN=order_id,TYPE(NAME=hash_mod,PROPERTIES('sharding-count'=4)), GENERATED_KEY(COLUMN=another_id,TYPE(NAME=snowflake,PROPERTIES('worker-id'=123))))" />
+    <distsql-case id="create-sharding-table-rule-with-inline-expression" value="CREATE SHARDING TABLE RULE t_order (RESOURCES('ms_group_${0..1}'), SHARDING_COLUMN=order_id,TYPE(NAME=hash_mod,PROPERTIES('sharding-count'=4)), GENERATED_KEY(COLUMN=another_id,TYPE(NAME=snowflake,PROPERTIES('worker-id'=123))))" />
     <distsql-case id="create-sharding-binding-table-rule" value="CREATE SHARDING BINDING TABLE RULES ((t_order,t_order_item), (t_1,t_2))" />
     <distsql-case id="create-sharding-broadcast-table-rule" value="CREATE SHARDING BROADCAST TABLE RULES(t_1,t_2)" />
     <distsql-case id="create-static-readwrite-splitting-rule" value="CREATE READWRITE_SPLITTING RULE ms_group_0 (WRITE_RESOURCE=primary_ds, READ_RESOURCES(replica_ds_0,replica_ds_1), TYPE(NAME=random)))" />