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/11 08:06:41 UTC

[shardingsphere] branch master updated: Add alter sharding binding table rules (#10304)

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 810f46b  Add alter sharding binding table rules (#10304)
810f46b is described below

commit 810f46b464d302ddb87f6c98cadd45eb242dc7df
Author: Haoran Meng <me...@gmail.com>
AuthorDate: Tue May 11 16:06:07 2021 +0800

    Add alter sharding binding table rules (#10304)
    
    Co-authored-by: menghaoran <me...@apache.org>
---
 .../src/main/antlr4/imports/RDLStatement.g4        |   4 +
 .../distsql/parser/autogen/DistSQLStatement.g4     |   1 +
 .../distsql/parser/core/DistSQLVisitor.java        |  13 +++
 .../api/DistSQLStatementParserEngineTest.java      |  16 +++
 .../AlterShardingBindingTableRulesStatement.java}  |  40 +++-----
 .../converter/ShardingRuleStatementConverter.java  |  15 +++
 .../text/distsql/rdl/RDLBackendHandlerFactory.java |   9 +-
 ...terShardingBindingTableRulesBackendHandler.java |  89 ++++++++++++++++
 ...hardingBindingTableRulesBackendHandlerTest.java | 112 +++++++++++++++++++++
 9 files changed, 273 insertions(+), 26 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 156e521..e73353c 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
@@ -79,6 +79,10 @@ alterShardingTableRule
     : ALTER SHARDING TABLE RULE shardingTableRuleDefinition (COMMA shardingTableRuleDefinition)*
     ;
 
+alterShardingBindingTableRules
+    : ALTER SHARDING BINDING TABLE RULES LP bindTableRulesDefinition (COMMA bindTableRulesDefinition)* RP
+    ;
+
 createReplicaQueryRule
     : CREATE REPLICA_QUERY RULE LP replicaQueryRuleDefinition (COMMA replicaQueryRuleDefinition)* RP
     ;
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 fc0382d..07a4d09 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
@@ -26,6 +26,7 @@ execute
     | createShardingBindingTableRules
     | createShardingBroadcastTableRules
     | alterShardingTableRule
+    | alterShardingBindingTableRules
     | 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 34f600b..b4acaf8 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
@@ -63,6 +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.alter.AlterShardingBindingTableRulesStatement;
 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;
@@ -166,6 +167,18 @@ public final class DistSQLVisitor extends DistSQLStatementBaseVisitor<ASTNode> {
     }
 
     @Override
+    public ASTNode visitAlterShardingBindingTableRules(final DistSQLStatementParser.AlterShardingBindingTableRulesContext ctx) {
+        AlterShardingBindingTableRulesStatement result = new AlterShardingBindingTableRulesStatement();
+        for (BindTableRulesDefinitionContext each : ctx.bindTableRulesDefinition()) {
+            ShardingBindingTableRuleSegment segment = new ShardingBindingTableRuleSegment();
+            segment.setTables(Joiner.on(",")
+                    .join(each.tableName().stream().map(t -> new IdentifierValue(t.getText()).getValue()).collect(Collectors.toList())));
+            result.getRules().add(segment);
+        }
+        return result;
+    }
+
+    @Override
     public ASTNode visitReplicaQueryRuleDefinition(final ReplicaQueryRuleDefinitionContext ctx) {
         ReadwriteSplittingRuleSegment result = new ReadwriteSplittingRuleSegment();
         Collection<String> replicaDatasources = new LinkedList<>();
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 9cfe724..d27f991 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.AlterShardingBindingTableRulesStatement;
 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;
@@ -65,6 +66,9 @@ public final class DistSQLStatementParserEngineTest {
             + "TYPE(NAME=hash_mod,PROPERTIES('sharding-count'=4)),"
             + "GENERATED_KEY(COLUMN=another_id,TYPE(NAME=snowflake,PROPERTIES(\"worker-id\"=123))))";
 
+    private static final String RDL_ALTER_SHARDING_BINDING_TABLE_RULES = "ALTER SHARDING BINDING TABLE RULES ("
+            + "(t_order,t_order_item), (t_1,t_2))";
+
     private final DistSQLStatementParserEngine engine = new DistSQLStatementParserEngine();
     
     @Test
@@ -174,4 +178,16 @@ public final class DistSQLStatementParserEngineTest {
         assertThat(tableRuleSegment.getTableStrategy().getAlgorithmName(), is("hash_mod"));
         assertThat(tableRuleSegment.getTableStrategy().getAlgorithmProps().getProperty("sharding-count"), is("4"));
     }
+
+    @Test
+    public void assertParseAlterShardingBindingTableRules() {
+        SQLStatement sqlStatement = engine.parse(RDL_ALTER_SHARDING_BINDING_TABLE_RULES);
+        assertTrue(sqlStatement instanceof AlterShardingBindingTableRulesStatement);
+        List<ShardingBindingTableRuleSegment> shardingBindingTableRuleSegments = new ArrayList<>(((AlterShardingBindingTableRulesStatement) sqlStatement).getRules());
+        assertThat(shardingBindingTableRuleSegments.size(), is(2));
+        ShardingBindingTableRuleSegment segment = shardingBindingTableRuleSegments.get(0);
+        assertThat(segment.getTables(), is("t_order,t_order_item"));
+        segment = shardingBindingTableRuleSegments.get(1);
+        assertThat(segment.getTables(), is("t_1,t_2"));
+    }
 }
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/AlterShardingBindingTableRulesStatement.java
similarity index 56%
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/AlterShardingBindingTableRulesStatement.java
index fc0382d..5a1ace3 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/AlterShardingBindingTableRulesStatement.java
@@ -15,29 +15,21 @@
  * 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.rdl.ShardingBindingTableRuleSegment;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.RDLStatement;
+import java.util.Collection;
+import java.util.LinkedList;
 
-execute
-    : (addResource
-    | dropResource
-    | createShardingTableRule
-    | createShardingBindingTableRules
-    | createShardingBroadcastTableRules
-    | alterShardingTableRule
-    | dropShardingRule
-    | createReplicaQueryRule
-    | alterReplicaQueryRule
-    | dropReplicaQueryRule
-    | showResources
-    | showRule
-    | showScalingJobList
-    | showScalingJobStatus
-    | startScalingJob
-    | stopScalingJob
-    | dropScalingJob
-    | resetScalingJob
-    | checkScalingJob
-    ) SEMI?
-    ;
+/**
+ * Alter sharding binding table rules statement.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterShardingBindingTableRulesStatement extends RDLStatement {
+    
+    private final Collection<ShardingBindingTableRuleSegment> rules = 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 8b55715..91a741f 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.AlterShardingBindingTableRulesStatement;
 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;
@@ -75,6 +76,20 @@ public final class ShardingRuleStatementConverter {
         }
         return result;
     }
+
+    /**
+     * Convert alter sharding binding table rule statement context to YAML sharding rule configuration.
+     *
+     * @param sqlStatement alter sharding binding table rule statement
+     * @return YAML sharding rule configuration
+     */
+    public static YamlShardingRuleConfiguration convert(final AlterShardingBindingTableRulesStatement sqlStatement) {
+        YamlShardingRuleConfiguration result = new YamlShardingRuleConfiguration();
+        for (ShardingBindingTableRuleSegment each : sqlStatement.getRules()) {
+            result.getBindingTables().add(each.getTables());
+        }
+        return result;
+    }
     
     private static YamlShardingRuleConfiguration convertTableRuleSegments(final Collection<TableRuleSegment> tableRuleSegments) {
         YamlShardingRuleConfiguration result = new YamlShardingRuleConfiguration();
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 3668883..eeb6157 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,6 +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.alter.AlterShardingBindingTableRulesStatement;
 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;
@@ -34,14 +35,15 @@ 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.AlterShardingBindingTableRulesBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.AlterShardingTableRuleBackendHandler;
 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;
@@ -107,6 +109,9 @@ public final class RDLBackendHandlerFactory {
         if (sqlStatement instanceof AlterShardingTableRuleStatement) {
             return Optional.of(new AlterShardingTableRuleBackendHandler((AlterShardingTableRuleStatement) sqlStatement, backendConnection));
         }
+        if (sqlStatement instanceof AlterShardingBindingTableRulesStatement) {
+            return Optional.of(new AlterShardingBindingTableRulesBackendHandler((AlterShardingBindingTableRulesStatement) 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/AlterShardingBindingTableRulesBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterShardingBindingTableRulesBackendHandler.java
new file mode 100644
index 0000000..e7cf40c
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterShardingBindingTableRulesBackendHandler.java
@@ -0,0 +1,89 @@
+/*
+ * 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.Splitter;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterShardingBindingTableRulesStatement;
+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.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.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.Collection;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Alter sharding binding table rule backend handler.
+ */
+public final class AlterShardingBindingTableRulesBackendHandler extends SchemaRequiredBackendHandler<AlterShardingBindingTableRulesStatement> {
+
+    public AlterShardingBindingTableRulesBackendHandler(final AlterShardingBindingTableRulesStatement sqlStatement, final BackendConnection backendConnection) {
+        super(sqlStatement, backendConnection);
+    }
+    
+    @Override
+    public ResponseHeader execute(final String schemaName, final AlterShardingBindingTableRulesStatement sqlStatement) {
+        YamlShardingRuleConfiguration config = ShardingRuleStatementConverter.convert(sqlStatement);
+        ShardingRuleConfiguration shardingRuleConfiguration = check(schemaName, config);
+        shardingRuleConfiguration.getBindingTableGroups().clear();
+        shardingRuleConfiguration.getBindingTableGroups().addAll(config.getBindingTables());
+        post(schemaName, ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations());
+        return new UpdateResponseHeader(sqlStatement);
+    }
+    
+    private ShardingRuleConfiguration check(final String schemaName, final YamlShardingRuleConfiguration configuration) {
+        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();
+        }
+        Collection<String> invalidBindingTables = new HashSet<>();
+        Collection<String> existLogicTables = new HashSet<>();
+        existLogicTables.addAll(shardingRuleConfig.get().getTables().stream().map(ShardingTableRuleConfiguration::getLogicTable).collect(Collectors.toSet()));
+        existLogicTables.addAll(shardingRuleConfig.get().getAutoTables().stream().map(ShardingAutoTableRuleConfiguration::getLogicTable).collect(Collectors.toSet()));
+        for (String bindingTable : configuration.getBindingTables()) {
+            for (String logicTable : Splitter.on(",").splitToList(bindingTable)) {
+                if (!existLogicTables.contains(logicTable.trim())) {
+                    invalidBindingTables.add(logicTable);
+                }
+            }
+        }
+        if (!invalidBindingTables.isEmpty()) {
+            throw new ShardingTableRuleNotExistedException(invalidBindingTables);
+        }
+        return shardingRuleConfig.get();
+    }
+    
+    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/AlterShardingBindingTableRulesBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterShardingBindingTableRulesBackendHandlerTest.java
new file mode 100644
index 0000000..a4bf903
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterShardingBindingTableRulesBackendHandlerTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.rdl.ShardingBindingTableRuleSegment;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterShardingBindingTableRulesStatement;
+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.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.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 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 AlterShardingBindingTableRulesBackendHandlerTest {
+    
+    @Mock
+    private BackendConnection backendConnection;
+    
+    @Mock
+    private MetaDataContexts metaDataContexts;
+    
+    @Mock
+    private TransactionContexts transactionContexts;
+    
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+    
+    @Mock
+    private ShardingSphereRuleMetaData shardingSphereRuleMetaData;
+    
+    @Before
+    public void setUp() throws Exception {
+        ProxyContext.getInstance().init(metaDataContexts, transactionContexts);
+        when(metaDataContexts.getAllSchemaNames()).thenReturn(Arrays.asList("test"));
+        when(metaDataContexts.getMetaData(eq("test"))).thenReturn(shardingSphereMetaData);
+        when(shardingSphereMetaData.getRuleMetaData()).thenReturn(shardingSphereRuleMetaData);
+    }
+    
+    @Test
+    public void assertExecute() {
+        when(shardingSphereRuleMetaData.getConfigurations()).thenReturn(Arrays.asList(buildShardingRuleConfiguration()));
+        AlterShardingBindingTableRulesStatement statement = buildShardingTableRuleStatement();
+        AlterShardingBindingTableRulesBackendHandler handler = new AlterShardingBindingTableRulesBackendHandler(statement, backendConnection);
+        ResponseHeader responseHeader = handler.execute("test", statement);
+        assertNotNull(responseHeader);
+        assertTrue(responseHeader instanceof UpdateResponseHeader);
+    }
+    
+    @Test(expected = ShardingTableRuleNotExistedException.class)
+    public void assertExecuteWithNotExistTableRule() {
+        when(shardingSphereRuleMetaData.getConfigurations()).thenReturn(Arrays.asList(new ShardingRuleConfiguration()));
+        AlterShardingBindingTableRulesStatement statement = buildShardingTableRuleStatement();
+        AlterShardingBindingTableRulesBackendHandler handler = new AlterShardingBindingTableRulesBackendHandler(statement, backendConnection);
+        ResponseHeader responseHeader = handler.execute("test", statement);
+        assertNotNull(responseHeader);
+        assertTrue(responseHeader instanceof UpdateResponseHeader);
+    }
+    
+    private ShardingRuleConfiguration buildShardingRuleConfiguration() {
+        ShardingRuleConfiguration shardingRuleConfiguration = new ShardingRuleConfiguration();
+        shardingRuleConfiguration.getTables().add(new ShardingTableRuleConfiguration("t_order"));
+        shardingRuleConfiguration.getTables().add(new ShardingTableRuleConfiguration("t_order_item"));
+        shardingRuleConfiguration.getTables().add(new ShardingTableRuleConfiguration("t_1"));
+        shardingRuleConfiguration.getTables().add(new ShardingTableRuleConfiguration("t_2"));
+        shardingRuleConfiguration.getBindingTableGroups().addAll(Arrays.asList("t_order,t_order_item"));
+        return shardingRuleConfiguration;
+    }
+    
+    private AlterShardingBindingTableRulesStatement buildShardingTableRuleStatement() {
+        AlterShardingBindingTableRulesStatement result = new AlterShardingBindingTableRulesStatement();
+        ShardingBindingTableRuleSegment segment = new ShardingBindingTableRuleSegment();
+        segment.setTables("t_order,t_order_item");
+        result.getRules().add(segment);
+        ShardingBindingTableRuleSegment segmentAnother = new ShardingBindingTableRuleSegment();
+        segmentAnother.setTables("t_1,t_2");
+        result.getRules().add(segmentAnother);
+        return result;
+    }
+}