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/05/10 04:29:21 UTC

[shardingsphere] branch master updated: Add alter sharding table rule (#10285)

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 1e9dbb6  Add alter sharding table rule (#10285)
1e9dbb6 is described below

commit 1e9dbb647ba93687fc2fef3a0d3eec9c9dca8c92
Author: Haoran Meng <me...@gmail.com>
AuthorDate: Mon May 10 12:28:44 2021 +0800

    Add alter sharding table rule (#10285)
    
    Co-authored-by: menghaoran <me...@apache.org>
---
 .../src/main/antlr4/imports/RDLStatement.g4        |   4 +-
 .../distsql/parser/autogen/DistSQLStatement.g4     |   2 +-
 .../distsql/parser/core/DistSQLVisitor.java        |  52 ++---
 .../api/DistSQLStatementParserEngineTest.java      |  25 ++-
 .../alter/AlterShardingTableRuleStatement.java}    |  41 ++--
 .../converter/ShardingRuleStatementConverter.java  |  11 ++
 .../text/distsql/rdl/RDLBackendHandlerFactory.java |  10 +-
 .../rdl/impl/AlterShardingRuleBackendHandler.java  | 216 ---------------------
 .../impl/AlterShardingTableRuleBackendHandler.java | 121 ++++++++++++
 .../AlterShardingTableRuleBackendHandlerTest.java  | 133 +++++++++++++
 10 files changed, 326 insertions(+), 289 deletions(-)

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 944ce49..156e521 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
@@ -75,8 +75,8 @@ createShardingBroadcastTableRules
     : CREATE SHARDING BROADCAST TABLE RULES LP IDENTIFIER (COMMA IDENTIFIER)* RP
     ;
 
-alterShardingRule
-    : ALTER SHARDING RULE LP alterShardingTableRuleDefinition (COMMA alterShardingTableRuleDefinition)* alterBindingTables? defaultTableStrategy? broadcastTables? RP
+alterShardingTableRule
+    : ALTER SHARDING TABLE RULE shardingTableRuleDefinition (COMMA shardingTableRuleDefinition)*
     ;
 
 createReplicaQueryRule
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 05d72c9..fc0382d 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
@@ -25,7 +25,7 @@ execute
     | createShardingTableRule
     | createShardingBindingTableRules
     | createShardingBroadcastTableRules
-    | alterShardingRule
+    | alterShardingTableRule
     | dropShardingRule
     | createReplicaQueryRule
     | alterReplicaQueryRule
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 af17d26..34f600b 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
@@ -21,13 +21,11 @@ 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.DistSQLStatementBaseVisitor;
+import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.AddResourceContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.AlgorithmPropertyContext;
-import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.AlterBindingTableContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.AlterReplicaQueryRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.AlterReplicaQueryRuleDefinitionContext;
-import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.AlterShardingRuleContext;
-import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.AlterShardingTableRuleDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.BindTableRulesDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.CheckScalingJobContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.CreateReplicaQueryRuleContext;
@@ -65,7 +63,7 @@ import org.apache.shardingsphere.distsql.parser.statement.ral.impl.ShowScalingJo
 import org.apache.shardingsphere.distsql.parser.statement.ral.impl.StartScalingJobStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.impl.StopScalingJobStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.AlterReadwriteSplittingRuleStatement;
-import org.apache.shardingsphere.distsql.parser.statement.rdl.AlterShardingRuleStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterShardingTableRuleStatement;
 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.CreateShardingBindingTableRulesStatement;
@@ -143,41 +141,6 @@ public final class DistSQLVisitor extends DistSQLStatementBaseVisitor<ASTNode> {
     }
     
     @Override
-    public ASTNode visitAlterShardingRule(final AlterShardingRuleContext ctx) {
-        AlterShardingRuleStatement result;
-        if (null != ctx.defaultTableStrategy()) {
-            String defaultTableStrategyColumn = null != ctx.defaultTableStrategy().columnName() ? ctx.defaultTableStrategy().columnName().getText() : null;
-            result = new AlterShardingRuleStatement(defaultTableStrategyColumn, (FunctionSegment) visit(ctx.defaultTableStrategy()));
-        } else {
-            result = new AlterShardingRuleStatement(null, null);
-        }
-        for (AlterShardingTableRuleDefinitionContext each : ctx.alterShardingTableRuleDefinition()) {
-            if (null != each.ADD()) {
-                result.getAddShardingRules().add((TableRuleSegment) visit(each.shardingTableRuleDefinition()));
-            } else if (null != each.MODIFY()) {
-                result.getModifyShardingRules().add((TableRuleSegment) visit(each.shardingTableRuleDefinition()));
-            }
-        }
-        if (null != ctx.alterBindingTables()) {
-            for (AlterBindingTableContext each : ctx.alterBindingTables().alterBindingTable()) {
-                if (null != each.ADD()) {
-                    Collection<String> tables = each.bindingTable().tableNames().IDENTIFIER().stream().map(t -> new IdentifierValue(t.getText()).getValue()).collect(Collectors.toList());
-                    result.getAddBindingTables().add(tables);
-                } else if (null != each.DROP()) {
-                    Collection<String> tables = each.bindingTable().tableNames().IDENTIFIER().stream().map(t -> new IdentifierValue(t.getText()).getValue()).collect(Collectors.toList());
-                    result.getDropBindingTables().add(tables);
-                }
-            }
-        }
-        if (null != ctx.broadcastTables()) {
-            for (TerminalNode each : ctx.broadcastTables().IDENTIFIER()) {
-                result.getBroadcastTables().add(new IdentifierValue(each.getText()).getValue());
-            }
-        }
-        return result;
-    }
-    
-    @Override
     public ASTNode visitCreateReplicaQueryRule(final CreateReplicaQueryRuleContext ctx) {
         Collection<ReadwriteSplittingRuleSegment> replicaQueryRules = new LinkedList<>();
         for (ReplicaQueryRuleDefinitionContext each : ctx.replicaQueryRuleDefinition()) {
@@ -192,7 +155,16 @@ public final class DistSQLVisitor extends DistSQLStatementBaseVisitor<ASTNode> {
         result.getTables().addAll(ctx.IDENTIFIER().stream().map(ParseTree::getText).collect(Collectors.toList()));
         return result;
     }
-    
+
+    @Override
+    public ASTNode visitAlterShardingTableRule(final DistSQLStatementParser.AlterShardingTableRuleContext ctx) {
+        AlterShardingTableRuleStatement result = new AlterShardingTableRuleStatement();
+        for (ShardingTableRuleDefinitionContext each : ctx.shardingTableRuleDefinition()) {
+            result.getTables().add((TableRuleSegment) visit(each));
+        }
+        return result;
+    }
+
     @Override
     public ASTNode visitReplicaQueryRuleDefinition(final ReplicaQueryRuleDefinitionContext ctx) {
         ReadwriteSplittingRuleSegment result = new ReadwriteSplittingRuleSegment();
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 57e3960..9cfe724 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
@@ -20,6 +20,7 @@ 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.segment.rdl.ShardingBindingTableRuleSegment;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterShardingTableRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.AddResourceStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingBindingTableRulesStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingBroadcastTableRulesStatement;
@@ -57,7 +58,13 @@ public final class DistSQLStatementParserEngineTest {
             + "(t_order,t_order_item), (t_1,t_2))";
     
     private static final String RDL_CREATE_SHARDING_BROADCAST_TABLE_RULES = "CREATE SHARDING BROADCAST TABLE RULES(t_1,t_2)";
-    
+
+    private static final String RDL_ALTER_SHARDING_TABLE_RULE = "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))))";
+
     private final DistSQLStatementParserEngine engine = new DistSQLStatementParserEngine();
     
     @Test
@@ -151,4 +158,20 @@ public final class DistSQLStatementParserEngineTest {
         assertTrue(sqlStatement instanceof CreateShardingBroadcastTableRulesStatement);
         assertThat(((CreateShardingBroadcastTableRulesStatement) sqlStatement).getTables(), is(Arrays.asList("t_1", "t_2")));
     }
+
+    @Test
+    public void assertParseAlterShardingTableRule() {
+        SQLStatement sqlStatement = engine.parse(RDL_ALTER_SHARDING_TABLE_RULE);
+        assertTrue(sqlStatement instanceof AlterShardingTableRuleStatement);
+        assertThat(((AlterShardingTableRuleStatement) sqlStatement).getTables().size(), is(1));
+        TableRuleSegment tableRuleSegment = ((AlterShardingTableRuleStatement) 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/alter/AlterShardingTableRuleStatement.java
similarity index 58%
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/alter/AlterShardingTableRuleStatement.java
index 05d72c9..d8ca2f7 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/alter/AlterShardingTableRuleStatement.java
@@ -15,29 +15,22 @@
  * limitations under the License.
  */
 
-grammar DistSQLStatement;
+package org.apache.shardingsphere.distsql.parser.statement.rdl.alter;
 
-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.RDLStatement;
 
-execute
-    : (addResource
-    | dropResource
-    | createShardingTableRule
-    | createShardingBindingTableRules
-    | createShardingBroadcastTableRules
-    | alterShardingRule
-    | dropShardingRule
-    | createReplicaQueryRule
-    | alterReplicaQueryRule
-    | dropReplicaQueryRule
-    | showResources
-    | showRule
-    | showScalingJobList
-    | showScalingJobStatus
-    | startScalingJob
-    | stopScalingJob
-    | dropScalingJob
-    | resetScalingJob
-    | checkScalingJob
-    ) SEMI?
-    ;
+import java.util.Collection;
+import java.util.LinkedList;
+
+/**
+ * Alter sharding table rule statement.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterShardingTableRuleStatement extends RDLStatement {
+    
+    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 ffbba11..8b55715 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.segment.rdl.ShardingBindingTableRuleSegment;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterShardingTableRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingBindingTableRulesStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingTableRuleStatement;
 import org.apache.shardingsphere.infra.yaml.config.algorithm.YamlShardingSphereAlgorithmConfiguration;
@@ -50,6 +51,16 @@ public final class ShardingRuleStatementConverter {
     public static YamlShardingRuleConfiguration convert(final CreateShardingTableRuleStatement sqlStatement) {
         return convertTableRuleSegments(sqlStatement.getTables());
     }
+
+    /**
+     * Convert alter sharding table rule statement context to YAML sharding rule configuration.
+     *
+     * @param sqlStatement alter sharding table rule statement
+     * @return YAML sharding rule configuration
+     */
+    public static YamlShardingRuleConfiguration convert(final AlterShardingTableRuleStatement sqlStatement) {
+        return convertTableRuleSegments(sqlStatement.getTables());
+    }
     
     /**
      * Convert create sharding binding table rule statement context to YAML sharding rule configuration.
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 a2506dc..3668883 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
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.rdl;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.AlterReadwriteSplittingRuleStatement;
-import org.apache.shardingsphere.distsql.parser.statement.rdl.AlterShardingRuleStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterShardingTableRuleStatement;
 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.CreateShardingBindingTableRulesStatement;
@@ -34,14 +34,14 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.AlterShardingTableRuleBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.CreateShardingTableRuleBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.AddResourceBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.AlterReadwriteSplittingRuleBackendHandler;
-import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.AlterShardingRuleBackendHandler;
 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.CreateShardingBindingTableRulesBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.CreateShardingBroadcastTableRulesBackendHandler;
-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;
@@ -104,8 +104,8 @@ public final class RDLBackendHandlerFactory {
         if (sqlStatement instanceof CreateShardingBroadcastTableRulesStatement) {
             return Optional.of(new CreateShardingBroadcastTableRulesBackendHandler((CreateShardingBroadcastTableRulesStatement) sqlStatement, backendConnection));
         }
-        if (sqlStatement instanceof AlterShardingRuleStatement) {
-            return Optional.of(new AlterShardingRuleBackendHandler((AlterShardingRuleStatement) sqlStatement, backendConnection));
+        if (sqlStatement instanceof AlterShardingTableRuleStatement) {
+            return Optional.of(new AlterShardingTableRuleBackendHandler((AlterShardingTableRuleStatement) sqlStatement, backendConnection));
         }
         if (sqlStatement instanceof CreateReadwriteSplittingRuleStatement) {
             return Optional.of(new CreateReadwriteSplittingRuleBackendHandler((CreateReadwriteSplittingRuleStatement) sqlStatement, backendConnection));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterShardingRuleBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterShardingRuleBackendHandler.java
deleted file mode 100644
index 9bfe994..0000000
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterShardingRuleBackendHandler.java
+++ /dev/null
@@ -1,216 +0,0 @@
-/*
- * 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 com.google.common.base.Joiner;
-import org.apache.shardingsphere.distsql.parser.segment.TableRuleSegment;
-import org.apache.shardingsphere.distsql.parser.statement.rdl.AlterShardingRuleStatement;
-import org.apache.shardingsphere.governance.core.registry.listener.event.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.context.ProxyContext;
-import org.apache.shardingsphere.proxy.backend.exception.ShardingRuleNotExistedException;
-import org.apache.shardingsphere.proxy.backend.exception.ShardingTableRuleExistedException;
-import org.apache.shardingsphere.proxy.backend.exception.ShardingTableRuleNotExistedException;
-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.api.config.ShardingRuleConfiguration;
-import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
-import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
-import org.apache.shardingsphere.sharding.converter.ShardingRuleStatementConverter;
-import org.apache.shardingsphere.sharding.yaml.config.YamlShardingRuleConfiguration;
-import org.apache.shardingsphere.sharding.yaml.config.rule.YamlShardingAutoTableRuleConfiguration;
-import org.apache.shardingsphere.sharding.yaml.config.rule.YamlTableRuleConfiguration;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.Optional;
-import java.util.stream.Collectors;
-
-/**
- * Alter sharding rule backend handler.
- */
-public final class AlterShardingRuleBackendHandler extends SchemaRequiredBackendHandler<AlterShardingRuleStatement> {
-    
-    public AlterShardingRuleBackendHandler(final AlterShardingRuleStatement sqlStatement, final BackendConnection backendConnection) {
-        super(sqlStatement, backendConnection);
-    }
-    
-    @Override
-    public ResponseHeader execute(final String schemaName, final AlterShardingRuleStatement sqlStatement) {
-        Optional<ShardingRuleConfiguration> shardingRuleConfig = ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations().stream()
-                .filter(each -> each instanceof ShardingRuleConfiguration).map(each -> (ShardingRuleConfiguration) each).findFirst();
-        if (!shardingRuleConfig.isPresent()) {
-            throw new ShardingRuleNotExistedException();
-        }
-        check(shardingRuleConfig.get(), sqlStatement);
-        Optional<YamlShardingRuleConfiguration> yamlShardingRuleConfig = new YamlRuleConfigurationSwapperEngine().swapToYamlRuleConfigurations(Collections.singleton(shardingRuleConfig.get())).stream()
-                .filter(each -> each instanceof YamlShardingRuleConfiguration).map(each -> (YamlShardingRuleConfiguration) each).findFirst();
-        if (!yamlShardingRuleConfig.isPresent()) {
-            throw new ShardingRuleNotExistedException();
-        }
-        YamlShardingRuleConfiguration rule = yamlShardingRuleConfig.get();
-        alter(rule, sqlStatement);
-        Collection<RuleConfiguration> rules = new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(Collections.singleton(rule));
-        post(schemaName, rules);
-        return new UpdateResponseHeader(sqlStatement);
-    }
-    
-    private void check(final ShardingRuleConfiguration shardingRuleConfig, final AlterShardingRuleStatement statement) {
-        checkModifyRule(shardingRuleConfig, statement);
-        checkAddRule(shardingRuleConfig, statement);
-        checkBindingTable(shardingRuleConfig, statement);
-    }
-    
-    private void checkModifyRule(final ShardingRuleConfiguration shardingRuleConfig, final AlterShardingRuleStatement statement) {
-        Collection<String> notExistRules = new LinkedList<>();
-        for (TableRuleSegment each : statement.getModifyShardingRules()) {
-            Optional<ShardingTableRuleConfiguration> existTable = shardingRuleConfig.getTables().stream().filter(t -> t.getLogicTable().equals(each.getLogicTable())).findFirst();
-            if (existTable.isPresent()) {
-                continue;
-            }
-            Optional<ShardingAutoTableRuleConfiguration> existAutoTable = shardingRuleConfig.getAutoTables().stream().filter(t -> t.getLogicTable().equals(each.getLogicTable())).findFirst();
-            if (existAutoTable.isPresent()) {
-                continue;
-            }
-            notExistRules.add(each.getLogicTable());
-        }
-        if (!notExistRules.isEmpty()) {
-            throw new ShardingTableRuleNotExistedException(notExistRules);
-        }
-    }
-    
-    private void checkAddRule(final ShardingRuleConfiguration shardingRuleConfig, final AlterShardingRuleStatement statement) {
-        Collection<String> existRules = new LinkedList<>();
-        for (TableRuleSegment each : statement.getModifyShardingRules()) {
-            Optional<ShardingTableRuleConfiguration> existTable = shardingRuleConfig.getTables().stream().filter(t -> !t.getLogicTable().equals(each.getLogicTable())).findFirst();
-            if (existTable.isPresent()) {
-                continue;
-            }
-            Optional<ShardingAutoTableRuleConfiguration> existAutoTable = shardingRuleConfig.getAutoTables().stream().filter(t -> !t.getLogicTable().equals(each.getLogicTable())).findFirst();
-            if (existAutoTable.isPresent()) {
-                continue;
-            }
-            existRules.add(each.getLogicTable());
-        }
-        if (!existRules.isEmpty()) {
-            throw new ShardingTableRuleExistedException(existRules);
-        }
-    }
-    
-    private void checkBindingTable(final ShardingRuleConfiguration shardingRuleConfig, final AlterShardingRuleStatement statement) {
-        Collection<String> validTables = new HashSet<>();
-        validTables.addAll(shardingRuleConfig.getTables().stream().map(ShardingTableRuleConfiguration::getLogicTable).collect(Collectors.toSet()));
-        validTables.addAll(shardingRuleConfig.getAutoTables().stream().map(ShardingAutoTableRuleConfiguration::getLogicTable).collect(Collectors.toSet()));
-        validTables.addAll(statement.getAddShardingRules().stream().map(TableRuleSegment::getLogicTable).collect(Collectors.toList()));
-        Collection<String> invalidTables = new LinkedList<>();
-        for (Collection<String> each : statement.getAddBindingTables()) {
-            for (String t : each) {
-                if (!validTables.contains(t)) {
-                    invalidTables.add(t);
-                }
-            }
-        }
-        for (Collection<String> each : statement.getDropBindingTables()) {
-            for (String t : each) {
-                if (!validTables.contains(t)) {
-                    invalidTables.add(t);
-                }
-            }
-        }
-        if (!invalidTables.isEmpty()) {
-            throw new ShardingTableRuleNotExistedException(invalidTables);
-        }
-    }
-    
-    private void alter(final YamlShardingRuleConfiguration yamlShardingRuleConfig, final AlterShardingRuleStatement statement) {
-        modifyTableRule(yamlShardingRuleConfig, statement);
-        addTableRule(yamlShardingRuleConfig, statement);
-        Collection<String> existBindingTables = yamlShardingRuleConfig.getBindingTables();
-        for (Collection<String> each : statement.getAddBindingTables()) {
-            String addBindingTable = Joiner.on(",").join(each);
-            if (!existBindingTables.contains(addBindingTable)) {
-                existBindingTables.add(addBindingTable);
-            }
-        }
-        for (Collection<String> each : statement.getDropBindingTables()) {
-            String dropBindingTable = Joiner.on(",").join(each);
-            existBindingTables.remove(dropBindingTable);
-        }
-        if (!statement.getBroadcastTables().isEmpty()) {
-            yamlShardingRuleConfig.setBroadcastTables(statement.getBroadcastTables());
-        }
-        if (null != statement.getDefaultTableStrategy()) {
-            yamlShardingRuleConfig.setDefaultTableStrategy(ShardingRuleStatementConverter.createDefaultTableStrategyConfiguration(statement.getDefaultTableStrategyColumn(),
-                    statement.getDefaultTableStrategy()));
-        }
-    }
-    
-    private void modifyTableRule(final YamlShardingRuleConfiguration yamlShardingRuleConfig, final AlterShardingRuleStatement statement) {
-        for (TableRuleSegment each : statement.getModifyShardingRules()) {
-            YamlTableRuleConfiguration existTable = yamlShardingRuleConfig.getTables().get(each.getLogicTable());
-            if (null != existTable) {
-                yamlShardingRuleConfig.getTables().remove(each.getLogicTable());
-            }
-            YamlShardingAutoTableRuleConfiguration autoTable = new YamlShardingAutoTableRuleConfiguration();
-            if (null != each.getDataSources() && !each.getDataSources().isEmpty()) {
-                autoTable.setActualDataSources(Joiner.on(",").join(each.getDataSources()));
-            }
-            if (null != each.getTableStrategy()) {
-                autoTable.setShardingStrategy(ShardingRuleStatementConverter.createTableStrategyConfiguration(each));
-                yamlShardingRuleConfig.getShardingAlgorithms().put(ShardingRuleStatementConverter.getAlgorithmName(each.getLogicTable(), each.getTableStrategy().getAlgorithmName()),
-                        ShardingRuleStatementConverter.createAlgorithmConfiguration(each.getTableStrategy()));
-            }
-            if (null != each.getKeyGenerateStrategy()) {
-                autoTable.setKeyGenerateStrategy(ShardingRuleStatementConverter.createKeyGenerateStrategyConfiguration(each));
-                yamlShardingRuleConfig.getShardingAlgorithms().put(ShardingRuleStatementConverter.getAlgorithmName(each.getLogicTable(), each.getKeyGenerateStrategy().getAlgorithmName()),
-                        ShardingRuleStatementConverter.createAlgorithmConfiguration(each.getKeyGenerateStrategy()));
-            }
-            yamlShardingRuleConfig.getAutoTables().put(each.getLogicTable(), autoTable);
-        }
-    }
-    
-    private void addTableRule(final YamlShardingRuleConfiguration yamlShardingRuleConfig, final AlterShardingRuleStatement statement) {
-        for (TableRuleSegment each : statement.getAddShardingRules()) {
-            YamlShardingAutoTableRuleConfiguration table = new YamlShardingAutoTableRuleConfiguration();
-            if (null != each.getDataSources() && !each.getDataSources().isEmpty()) {
-                table.setActualDataSources(Joiner.on(",").join(each.getDataSources()));
-            }
-            if (null != each.getTableStrategy()) {
-                table.setShardingStrategy(ShardingRuleStatementConverter.createTableStrategyConfiguration(each));
-                yamlShardingRuleConfig.getShardingAlgorithms().put(ShardingRuleStatementConverter.getAlgorithmName(each.getLogicTable(), each.getTableStrategy().getAlgorithmName()),
-                        ShardingRuleStatementConverter.createAlgorithmConfiguration(each.getTableStrategy()));
-            }
-            if (null != each.getKeyGenerateStrategy()) {
-                table.setKeyGenerateStrategy(ShardingRuleStatementConverter.createKeyGenerateStrategyConfiguration(each));
-                yamlShardingRuleConfig.getShardingAlgorithms().put(ShardingRuleStatementConverter.getAlgorithmName(each.getLogicTable(), each.getKeyGenerateStrategy().getAlgorithmName()),
-                        ShardingRuleStatementConverter.createAlgorithmConfiguration(each.getKeyGenerateStrategy()));
-            }
-            yamlShardingRuleConfig.getAutoTables().put(each.getLogicTable(), table);
-        }
-    }
-    
-    private void post(final String schemaName, final Collection<RuleConfiguration> rules) {
-        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, rules));
-    }
-}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterShardingTableRuleBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterShardingTableRuleBackendHandler.java
new file mode 100644
index 0000000..fa3517f
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterShardingTableRuleBackendHandler.java
@@ -0,0 +1,121 @@
+/*
+ * 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.segment.TableRuleSegment;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterShardingTableRuleStatement;
+import org.apache.shardingsphere.governance.core.registry.listener.event.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.context.ProxyContext;
+import org.apache.shardingsphere.proxy.backend.exception.DuplicateTablesException;
+import org.apache.shardingsphere.proxy.backend.exception.ShardingRuleNotExistedException;
+import org.apache.shardingsphere.proxy.backend.exception.ShardingTableRuleNotExistedException;
+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.api.config.ShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
+import org.apache.shardingsphere.sharding.converter.ShardingRuleStatementConverter;
+import org.apache.shardingsphere.sharding.yaml.config.YamlShardingRuleConfiguration;
+
+import java.util.Set;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Alter sharding table rule backend handler.
+ */
+public final class AlterShardingTableRuleBackendHandler extends SchemaRequiredBackendHandler<AlterShardingTableRuleStatement> {
+    
+    public AlterShardingTableRuleBackendHandler(final AlterShardingTableRuleStatement sqlStatement, final BackendConnection backendConnection) {
+        super(sqlStatement, backendConnection);
+    }
+    
+    @Override
+    public ResponseHeader execute(final String schemaName, final AlterShardingTableRuleStatement sqlStatement) {
+        if (!getShardingRuleConfiguration(schemaName).isPresent()) {
+            throw new ShardingRuleNotExistedException();
+        }
+        check(schemaName, sqlStatement);
+        Optional<YamlShardingRuleConfiguration> yamlShardingRuleConfig = new YamlRuleConfigurationSwapperEngine()
+                .swapToYamlRuleConfigurations(Collections.singleton(getShardingRuleConfiguration(schemaName).get())).stream()
+                .filter(each -> each instanceof YamlShardingRuleConfiguration).map(each -> (YamlShardingRuleConfiguration) each).findFirst();
+        if (!yamlShardingRuleConfig.isPresent()) {
+            throw new ShardingRuleNotExistedException();
+        }
+        YamlShardingRuleConfiguration existYamlShardingRuleConfiguration = yamlShardingRuleConfig.get();
+        YamlShardingRuleConfiguration yamlShardingRuleConfiguration = ShardingRuleStatementConverter.convert(sqlStatement);
+        existYamlShardingRuleConfiguration.getShardingAlgorithms().putAll(yamlShardingRuleConfiguration.getShardingAlgorithms());
+        yamlShardingRuleConfiguration.getTables().keySet().forEach(each -> {
+            if (existYamlShardingRuleConfiguration.getTables().keySet().contains(each)) {
+                existYamlShardingRuleConfiguration.getTables().remove(each);
+            }
+        });
+        existYamlShardingRuleConfiguration.getAutoTables().putAll(yamlShardingRuleConfiguration.getAutoTables());
+        post(schemaName, new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(Collections.singleton(existYamlShardingRuleConfiguration)));
+        return new UpdateResponseHeader(sqlStatement);
+    }
+
+    private void check(final String schemaName, final AlterShardingTableRuleStatement sqlStatement) {
+        Collection<String> logicTableNames = new ArrayList<>(sqlStatement.getTables().size());
+        Set<String> duplicateTableNames = new HashSet<>(sqlStatement.getTables().size(), 1);
+        for (TableRuleSegment tableRuleSegment : sqlStatement.getTables()) {
+            if (logicTableNames.contains(tableRuleSegment.getLogicTable())) {
+                duplicateTableNames.add(tableRuleSegment.getLogicTable());
+            }
+            logicTableNames.add(tableRuleSegment.getLogicTable());
+        }
+        if (!duplicateTableNames.isEmpty()) {
+            throw new DuplicateTablesException(duplicateTableNames);
+        }
+        Collection<String> existTables = getLogicTables(schemaName);
+        Collection<String> notExistTables = logicTableNames.stream().filter(each -> !existTables.contains(each)).collect(Collectors.toList());
+        if (!notExistTables.isEmpty()) {
+            throw new ShardingTableRuleNotExistedException(notExistTables);
+        }
+    }
+
+    private Collection<String> getLogicTables(final String schemaName) {
+        Optional<ShardingRuleConfiguration> shardingRuleConfiguration = getShardingRuleConfiguration(schemaName);
+        Collection<String> result = new LinkedList<>();
+        if (!shardingRuleConfiguration.isPresent()) {
+            return result;
+        }
+        result.addAll(shardingRuleConfiguration.get().getTables().stream().map(ShardingTableRuleConfiguration::getLogicTable).collect(Collectors.toList()));
+        result.addAll(shardingRuleConfiguration.get().getAutoTables().stream().map(ShardingAutoTableRuleConfiguration::getLogicTable).collect(Collectors.toList()));
+        return result;
+    }
+
+    private Optional<ShardingRuleConfiguration> getShardingRuleConfiguration(final String schemaName) {
+        return ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations().stream()
+                .filter(each -> each instanceof ShardingRuleConfiguration).map(each -> (ShardingRuleConfiguration) each).findFirst();
+    }
+    
+    private void post(final String schemaName, final Collection<RuleConfiguration> rules) {
+        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, rules));
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterShardingTableRuleBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterShardingTableRuleBackendHandlerTest.java
new file mode 100644
index 0000000..f3013ea
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterShardingTableRuleBackendHandlerTest.java
@@ -0,0 +1,133 @@
+/*
+ * 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.segment.FunctionSegment;
+import org.apache.shardingsphere.distsql.parser.segment.TableRuleSegment;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterShardingTableRuleStatement;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
+import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.apache.shardingsphere.proxy.backend.exception.DuplicateTablesException;
+import org.apache.shardingsphere.proxy.backend.exception.ShardingRuleNotExistedException;
+import org.apache.shardingsphere.proxy.backend.exception.ShardingTableRuleNotExistedException;
+import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
+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;
+import org.apache.shardingsphere.transaction.context.TransactionContexts;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Properties;
+import java.util.Collection;
+import java.util.ArrayList;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class AlterShardingTableRuleBackendHandlerTest {
+    
+    @Mock
+    private BackendConnection backendConnection;
+    
+    @Mock
+    private AlterShardingTableRuleStatement sqlStatement;
+    
+    @Mock
+    private MetaDataContexts metaDataContexts;
+    
+    @Mock
+    private TransactionContexts transactionContexts;
+    
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+    
+    @Mock
+    private ShardingSphereRuleMetaData ruleMetaData;
+    
+    private AlterShardingTableRuleBackendHandler handler = new AlterShardingTableRuleBackendHandler(sqlStatement, backendConnection);
+    
+    @Before
+    public void setUp() {
+        ProxyContext.getInstance().init(metaDataContexts, transactionContexts);
+        when(metaDataContexts.getAllSchemaNames()).thenReturn(Collections.singletonList("test"));
+        when(metaDataContexts.getMetaData(eq("test"))).thenReturn(shardingSphereMetaData);
+        when(shardingSphereMetaData.getRuleMetaData()).thenReturn(ruleMetaData);
+    }
+    
+    @Test(expected = ShardingRuleNotExistedException.class)
+    public void assertExecuteWithoutShardingRule() {
+        handler.execute("test", sqlStatement);
+    }
+
+    @Test
+    public void assertExecute() {
+        TableRuleSegment tableRuleSegment = new TableRuleSegment();
+        tableRuleSegment.setLogicTable("t_order");
+        FunctionSegment functionSegment = new FunctionSegment();
+        functionSegment.setAlgorithmName("hash_mod");
+        functionSegment.setAlgorithmProps(new Properties());
+        tableRuleSegment.setTableStrategy(functionSegment);
+        tableRuleSegment.setDataSources(Arrays.asList("ds_0"));
+        tableRuleSegment.setTableStrategyColumn("order_id");
+        when(ruleMetaData.getConfigurations()).thenReturn(buildShardingConfigurations());
+        when(sqlStatement.getTables()).thenReturn(Arrays.asList(tableRuleSegment));
+        handler.execute("test", sqlStatement);
+        ResponseHeader responseHeader = handler.execute("test", sqlStatement);
+        assertNotNull(responseHeader);
+        assertTrue(responseHeader instanceof UpdateResponseHeader);
+    }
+    
+    @Test(expected = DuplicateTablesException.class)
+    public void assertExecuteWithDuplicateTablesInRDL() {
+        TableRuleSegment tableRuleSegment = new TableRuleSegment();
+        tableRuleSegment.setLogicTable("t_order");
+        when(ruleMetaData.getConfigurations()).thenReturn(buildShardingConfigurations());
+        when(sqlStatement.getTables()).thenReturn(Arrays.asList(tableRuleSegment, tableRuleSegment));
+        handler.execute("test", sqlStatement);
+    }
+    
+    @Test(expected = ShardingTableRuleNotExistedException.class)
+    public void assertExecuteWithoutExistTable() {
+        TableRuleSegment tableRuleSegment = new TableRuleSegment();
+        tableRuleSegment.setLogicTable("t_order_1");
+        when(ruleMetaData.getConfigurations()).thenReturn(buildShardingConfigurations());
+        when(sqlStatement.getTables()).thenReturn(Arrays.asList(tableRuleSegment));
+        handler.execute("test", sqlStatement);
+    }
+    
+    private Collection<RuleConfiguration> buildShardingConfigurations() {
+        ShardingRuleConfiguration configuration = new ShardingRuleConfiguration();
+        configuration.getTables().add(new ShardingTableRuleConfiguration("t_order_item"));
+        configuration.getAutoTables().add(new ShardingAutoTableRuleConfiguration("t_order"));
+        return new ArrayList<>(Collections.singletonList(configuration));
+    }
+}