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/04/16 10:46:41 UTC

[shardingsphere] branch master updated: Add create sharding table rule for rdl (#10110)

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 b5c179a  Add create sharding table rule for rdl (#10110)
b5c179a is described below

commit b5c179a2630a43605340c30bb24bd639cdb795e8
Author: Haoran Meng <me...@gmail.com>
AuthorDate: Fri Apr 16 18:45:59 2021 +0800

    Add create sharding table rule for rdl (#10110)
    
    * Add create sharding table rule
    
    * Add create sharding table rule
    
    * Add create sharding table rule
---
 .../src/main/antlr4/imports/Keyword.g4             | 24 ++++++++++
 .../src/main/antlr4/imports/RDLStatement.g4        | 20 ++++++--
 .../distsql/parser/autogen/DistSQLStatement.g4     |  2 +-
 .../distsql/parser/core/DistSQLVisitor.java        | 19 ++++++--
 .../api/DistSQLStatementParserEngineTest.java      | 24 ++++++++++
 .../impl/CreateShardingTableRuleStatement.java}    | 39 +++++++--------
 .../converter/ShardingRuleStatementConverter.java  | 24 ++++++++--
 .../text/distsql/rdl/RDLBackendHandlerFactory.java |  5 ++
 .../CreateShardingTableRuleBackendHandler.java     | 56 ++++++++++++++++++++++
 .../CreateShardingTableRuleBackendHandlerTest.java | 49 +++++++++++++++++++
 10 files changed, 227 insertions(+), 35 deletions(-)

diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/Keyword.g4 b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/Keyword.g4
index a14bc88..8dcd512 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/Keyword.g4
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/Keyword.g4
@@ -154,3 +154,27 @@ USER
 PASSWORD
     : P A S S W O R D
     ;
+
+TABLE
+    : T A B L E
+    ;
+
+SHARDING_COLUMN
+    : S H A R D I N G UL_ C O L U M N
+    ;
+    
+TYPE
+    : T Y P E
+    ;
+    
+NAME
+    : N A M E
+    ;
+
+PROPERTIES
+    : P R O P E R T I E S
+    ;
+    
+COLUMN
+    : C O L U M N
+    ;
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/RDLStatement.g4 b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/RDLStatement.g4
index 8838acf..5804e8b 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/RDLStatement.g4
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/RDLStatement.g4
@@ -59,6 +59,10 @@ password
     : IDENTIFIER | INT | STRING
     ;
 
+createShardingTableRule
+    : CREATE SHARDING TABLE RULE shardingTableRuleDefinition (COMMA shardingTableRuleDefinition)*
+    ;
+
 createShardingRule
     : CREATE SHARDING RULE LP shardingTableRuleDefinition (COMMA shardingTableRuleDefinition)* bindingTables? defaultTableStrategy? broadcastTables? RP
     ;
@@ -88,11 +92,15 @@ alterShardingTableRuleDefinition
     ;
 
 shardingTableRuleDefinition
-    : tableName resources? (columnName functionDefinition)? keyGenerateStrategy?
+    : tableName LP resources (COMMA shardingColumn)? (COMMA functionDefinition)? (COMMA keyGenerateStrategy)? RP
     ;
 
 resources
-    : RESOURCE LP IDENTIFIER (COMMA IDENTIFIER)* RP
+    : RESOURCES LP IDENTIFIER (COMMA IDENTIFIER)* RP
+    ;
+
+shardingColumn
+    : SHARDING_COLUMN EQ columnName
     ;
 
 alterBindingTables
@@ -120,7 +128,7 @@ broadcastTables
     ;
 
 keyGenerateStrategy
-    : GENERATED_KEY columnName functionDefinition
+    : GENERATED_KEY LP COLUMN EQ columnName COMMA functionDefinition RP
     ;
 
 actualDataNodes
@@ -160,7 +168,11 @@ schemaName
     ;
 
 functionDefinition
-    : functionName=IDENTIFIER LP algorithmProperties? RP
+    : TYPE LP NAME EQ functionName COMMA PROPERTIES LP algorithmProperties? RP RP
+    ;
+
+functionName
+    : IDENTIFIER
     ;
 
 algorithmProperties
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/DistSQLStatement.g4 b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/DistSQLStatement.g4
index 9c92aa7..9d1791f 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/DistSQLStatement.g4
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/DistSQLStatement.g4
@@ -22,7 +22,7 @@ import Symbol, RDLStatement, RQLStatement, RALStatement;
 execute
     : (addResource
     | dropResource
-    | createShardingRule
+    | createShardingTableRule
     | alterShardingRule
     | dropShardingRule
     | createReplicaQueryRule
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/core/DistSQLVisitor.java b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/core/DistSQLVisitor.java
index 089c390..6adb042 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/core/DistSQLVisitor.java
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/core/DistSQLVisitor.java
@@ -30,6 +30,7 @@ import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.B
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.CheckScalingJobContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.CreateReplicaQueryRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.CreateShardingRuleContext;
+import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.CreateShardingTableRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.DataSourceContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.DropReplicaQueryRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.DropResourceContext;
@@ -64,6 +65,7 @@ import org.apache.shardingsphere.distsql.parser.statement.rdl.AlterShardingRuleS
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.AddResourceStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateReadWriteSplittingRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingRuleStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingTableRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropReplicaQueryRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropResourceStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingRuleStatement;
@@ -106,6 +108,15 @@ public final class DistSQLVisitor extends DistSQLStatementBaseVisitor<ASTNode> {
     }
     
     @Override
+    public ASTNode visitCreateShardingTableRule(final CreateShardingTableRuleContext ctx) {
+        CreateShardingTableRuleStatement result = new CreateShardingTableRuleStatement();
+        for (ShardingTableRuleDefinitionContext each : ctx.shardingTableRuleDefinition()) {
+            result.getTables().add((TableRuleSegment) visit(each));
+        }
+        return result;
+    }
+    
+    @Override
     public ASTNode visitDropResource(final DropResourceContext ctx) {
         DropResourceStatement result = new DropResourceStatement();
         for (TerminalNode each : ctx.IDENTIFIER()) {
@@ -200,7 +211,7 @@ public final class DistSQLVisitor extends DistSQLStatementBaseVisitor<ASTNode> {
         result.setName(ctx.ruleName.getText());
         result.setWriteDataSource(ctx.primary.getText());
         result.setReadDataSources(replicaDatasources);
-        result.setLoadBalancer(ctx.functionDefinition().functionName.getText());
+        result.setLoadBalancer(ctx.functionDefinition().functionName().getText());
         result.setProps(props);
         return result;
     }
@@ -245,7 +256,7 @@ public final class DistSQLVisitor extends DistSQLStatementBaseVisitor<ASTNode> {
                     props.setProperty(each.key.getText(), each.value.getText());
                 }
             }
-            result.setLoadBalancer(ctx.functionDefinition().functionName.getText());
+            result.setLoadBalancer(ctx.functionDefinition().functionName().getText());
             result.setProps(props);
         }
         return result;
@@ -264,7 +275,7 @@ public final class DistSQLVisitor extends DistSQLStatementBaseVisitor<ASTNode> {
         result.setDataSources(dataSources);
         if (null != ctx.functionDefinition()) {
             result.setTableStrategy((FunctionSegment) visit(ctx.functionDefinition()));
-            result.setTableStrategyColumn(ctx.columnName().getText());
+            result.setTableStrategyColumn(ctx.shardingColumn().columnName().getText());
         }
         if (null != ctx.keyGenerateStrategy()) {
             result.setKeyGenerateStrategy((FunctionSegment) visit(ctx.keyGenerateStrategy().functionDefinition()));
@@ -276,7 +287,7 @@ public final class DistSQLVisitor extends DistSQLStatementBaseVisitor<ASTNode> {
     @Override
     public ASTNode visitFunctionDefinition(final FunctionDefinitionContext ctx) {
         FunctionSegment result = new FunctionSegment();
-        result.setAlgorithmName(ctx.functionName.getText());
+        result.setAlgorithmName(ctx.functionName().getText());
         Properties algorithmProps = new Properties();
         if (null != ctx.algorithmProperties()) {
             for (AlgorithmPropertyContext each : ctx.algorithmProperties().algorithmProperty()) {
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/test/java/org/apache/shardingsphere/distsql/parser/api/DistSQLStatementParserEngineTest.java b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/test/java/org/apache/shardingsphere/distsql/parser/api/DistSQLStatementParserEngineTest.java
index 13b7e15..1c5a256 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/test/java/org/apache/shardingsphere/distsql/parser/api/DistSQLStatementParserEngineTest.java
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/test/java/org/apache/shardingsphere/distsql/parser/api/DistSQLStatementParserEngineTest.java
@@ -18,7 +18,9 @@
 package org.apache.shardingsphere.distsql.parser.api;
 
 import org.apache.shardingsphere.distsql.parser.segment.DataSourceSegment;
+import org.apache.shardingsphere.distsql.parser.segment.TableRuleSegment;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.AddResourceStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingTableRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropResourceStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.junit.Test;
@@ -42,6 +44,12 @@ public final class DistSQLStatementParserEngineTest {
     
     private static final String RDL_DROP_RESOURCE = "DROP RESOURCE ds_0,ds_1";
     
+    private static final String RDL_CREATE_SHARDING_TABLE_RULE = "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)))" + ")";
+    
     private final DistSQLStatementParserEngine engine = new DistSQLStatementParserEngine();
     
     @Test
@@ -100,4 +108,20 @@ public final class DistSQLStatementParserEngineTest {
         assertThat(((DropResourceStatement) sqlStatement).getResourceNames().size(), is(2));
         assertTrue(((DropResourceStatement) sqlStatement).getResourceNames().containsAll(Arrays.asList("ds_0", "ds_1")));
     }
+    
+    @Test
+    public void assertParseCreateShardingTableRule() {
+        SQLStatement sqlStatement = engine.parse(RDL_CREATE_SHARDING_TABLE_RULE);
+        assertTrue(sqlStatement instanceof CreateShardingTableRuleStatement);
+        assertThat(((CreateShardingTableRuleStatement) sqlStatement).getTables().size(), is(1));
+        TableRuleSegment tableRuleSegment = ((CreateShardingTableRuleStatement) sqlStatement).getTables().iterator().next();
+        assertThat(tableRuleSegment.getLogicTable(), is("t_order"));
+        assertTrue(tableRuleSegment.getDataSources().containsAll(Arrays.asList("ms_group_0", "ms_group_1")));
+        assertThat(tableRuleSegment.getTableStrategyColumn(), is("order_id"));
+        assertThat(tableRuleSegment.getKeyGenerateStrategy().getAlgorithmName(), is("snowflake"));
+        assertThat(tableRuleSegment.getKeyGenerateStrategy().getAlgorithmProps().getProperty("worker-id"), is("123"));
+        assertThat(tableRuleSegment.getKeyGenerateStrategyColumn(), is("another_id"));
+        assertThat(tableRuleSegment.getTableStrategy().getAlgorithmName(), is("hash_mod"));
+        assertThat(tableRuleSegment.getTableStrategy().getAlgorithmProps().getProperty("sharding-count"), is("4"));
+    }
 }
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/DistSQLStatement.g4 b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/create/impl/CreateShardingTableRuleStatement.java
similarity index 57%
copy from shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/DistSQLStatement.g4
copy to shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/create/impl/CreateShardingTableRuleStatement.java
index 9c92aa7..c47935d 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/DistSQLStatement.g4
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/create/impl/CreateShardingTableRuleStatement.java
@@ -15,27 +15,22 @@
  * limitations under the License.
  */
 
-grammar DistSQLStatement;
+package org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl;
 
-import Symbol, RDLStatement, RQLStatement, RALStatement;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.distsql.parser.segment.TableRuleSegment;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.create.CreateRDLStatement;
 
-execute
-    : (addResource
-    | dropResource
-    | createShardingRule
-    | alterShardingRule
-    | dropShardingRule
-    | createReplicaQueryRule
-    | alterReplicaQueryRule
-    | dropReplicaQueryRule
-    | showResources
-    | showRule
-    | showScalingJobList
-    | showScalingJobStatus
-    | startScalingJob
-    | stopScalingJob
-    | dropScalingJob
-    | resetScalingJob
-    | checkScalingJob
-    ) SEMI?
-    ;
+import java.util.Collection;
+import java.util.LinkedList;
+
+/**
+ * Create sharding table rule statement.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class CreateShardingTableRuleStatement extends CreateRDLStatement {
+    
+    private final Collection<TableRuleSegment> tables = new LinkedList<>();
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/converter/ShardingRuleStatementConverter.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/converter/ShardingRuleStatementConverter.java
index 1027bed..f09f27a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/converter/ShardingRuleStatementConverter.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/converter/ShardingRuleStatementConverter.java
@@ -23,6 +23,7 @@ import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.segment.FunctionSegment;
 import org.apache.shardingsphere.distsql.parser.segment.TableRuleSegment;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingRuleStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingTableRuleStatement;
 import org.apache.shardingsphere.infra.yaml.config.algorithm.YamlShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.sharding.yaml.config.YamlShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.yaml.config.rule.YamlShardingAutoTableRuleConfiguration;
@@ -46,16 +47,31 @@ public final class ShardingRuleStatementConverter {
      * @return YAML sharding rule configuration
      */
     public static YamlShardingRuleConfiguration convert(final CreateShardingRuleStatement sqlStatement) {
+        YamlShardingRuleConfiguration result = convertTableRuleSegments(sqlStatement.getTables());
+        for (Collection<String> each : sqlStatement.getBindingTables()) {
+            result.getBindingTables().add(Joiner.on(",").join(each));
+        }
+        return result;
+    }
+    
+    /**
+     * Convert create sharding table rule statement context to YAML sharding rule configuration.
+     * 
+     * @param sqlStatement create sharding table rule statement
+     * @return YAML sharding rule configuration
+     */
+    public static YamlShardingRuleConfiguration convert(final CreateShardingTableRuleStatement sqlStatement) {
+        return convertTableRuleSegments(sqlStatement.getTables());
+    }
+    
+    private static YamlShardingRuleConfiguration convertTableRuleSegments(final Collection<TableRuleSegment> tableRuleSegments) {
         YamlShardingRuleConfiguration result = new YamlShardingRuleConfiguration();
-        for (TableRuleSegment each : sqlStatement.getTables()) {
+        for (TableRuleSegment each : tableRuleSegments) {
             if (null != each.getTableStrategy()) {
                 result.getShardingAlgorithms().put(getAlgorithmName(each.getLogicTable(), each.getTableStrategy().getAlgorithmName()), createAlgorithmConfiguration(each.getTableStrategy()));
                 result.getAutoTables().put(each.getLogicTable(), createAutoTableRuleConfiguration(each));
             }
         }
-        for (Collection<String> each : sqlStatement.getBindingTables()) {
-            result.getBindingTables().add(Joiner.on(",").join(each));
-        }
         return result;
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/RDLBackendHandlerFactory.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/RDLBackendHandlerFactory.java
index f4dc3de..a3bc841 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/RDLBackendHandlerFactory.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/RDLBackendHandlerFactory.java
@@ -24,6 +24,7 @@ import org.apache.shardingsphere.distsql.parser.statement.rdl.AlterShardingRuleS
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.AddResourceStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateReadWriteSplittingRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingRuleStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingTableRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropReplicaQueryRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropResourceStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingRuleStatement;
@@ -38,6 +39,7 @@ import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.AlterShardi
 import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.CreateDatabaseBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.CreateReadWriteSplittingRuleBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.CreateShardingRuleBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.CreateShardingTableRuleBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.DropDatabaseBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.DropReadWriteSplittingRuleBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.DropResourceBackendHandler;
@@ -91,6 +93,9 @@ public final class RDLBackendHandlerFactory {
         if (sqlStatement instanceof CreateShardingRuleStatement) {
             return Optional.of(new CreateShardingRuleBackendHandler((CreateShardingRuleStatement) sqlStatement, backendConnection));
         }
+        if (sqlStatement instanceof CreateShardingTableRuleStatement) {
+            return Optional.of(new CreateShardingTableRuleBackendHandler((CreateShardingTableRuleStatement) sqlStatement, backendConnection));
+        }
         if (sqlStatement instanceof AlterShardingRuleStatement) {
             return Optional.of(new AlterShardingRuleBackendHandler((AlterShardingRuleStatement) sqlStatement, backendConnection));
         }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateShardingTableRuleBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateShardingTableRuleBackendHandler.java
new file mode 100644
index 0000000..cfea21f
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateShardingTableRuleBackendHandler.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl;
+
+import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingTableRuleStatement;
+import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsAlteredEvent;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
+import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
+import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
+import org.apache.shardingsphere.proxy.backend.text.SchemaRequiredBackendHandler;
+import org.apache.shardingsphere.sharding.converter.ShardingRuleStatementConverter;
+import org.apache.shardingsphere.sharding.yaml.config.YamlShardingRuleConfiguration;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Create sharding table rule backend handler.
+ */
+public final class CreateShardingTableRuleBackendHandler extends SchemaRequiredBackendHandler<CreateShardingTableRuleStatement> {
+    
+    public CreateShardingTableRuleBackendHandler(final CreateShardingTableRuleStatement sqlStatement, final BackendConnection backendConnection) {
+        super(sqlStatement, backendConnection);
+    }
+    
+    @Override
+    public ResponseHeader execute(final String schemaName, final CreateShardingTableRuleStatement sqlStatement) {
+        YamlShardingRuleConfiguration config = ShardingRuleStatementConverter.convert(sqlStatement);
+        Collection<RuleConfiguration> rules = new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(Collections.singleton(config));
+        post(schemaName, rules);
+        return new UpdateResponseHeader(sqlStatement);
+    }
+    
+    private void post(final String schemaName, final Collection<RuleConfiguration> rules) {
+        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, rules));
+        // TODO Need to get the executed feedback from registry center for returning.
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateShardingTableRuleBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateShardingTableRuleBackendHandlerTest.java
new file mode 100644
index 0000000..55f87e3
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateShardingTableRuleBackendHandlerTest.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl;
+
+import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingTableRuleStatement;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class CreateShardingTableRuleBackendHandlerTest {
+    
+    @Mock
+    private BackendConnection backendConnection;
+    
+    @Mock
+    private CreateShardingTableRuleStatement sqlStatement;
+    
+    private CreateShardingTableRuleBackendHandler handler = new CreateShardingTableRuleBackendHandler(sqlStatement, backendConnection);
+    
+    @Test
+    public void assertExecute() {
+        ResponseHeader responseHeader = handler.execute("test", sqlStatement);
+        assertNotNull(responseHeader);
+        assertTrue(responseHeader instanceof UpdateResponseHeader);
+    }
+}