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/12 08:13:54 UTC

[shardingsphere] branch master updated: Drop sharding table rule & binding table rules & broadcast table rules (#10312)

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 ae18a1c  Drop sharding table rule & binding table rules & broadcast table rules (#10312)
ae18a1c is described below

commit ae18a1cfceb7881cfa5b3b136f0ed5615b6eca50
Author: Haoran Meng <me...@gmail.com>
AuthorDate: Wed May 12 16:13:09 2021 +0800

    Drop sharding table rule & binding table rules & broadcast table rules (#10312)
    
    Co-authored-by: menghaoranss <me...@apache.org>
---
 .../db/protocol/error/CommonErrorCode.java         |   6 +-
 .../src/main/antlr4/imports/RDLStatement.g4        |  12 +-
 .../distsql/parser/autogen/DistSQLStatement.g4     |   4 +-
 .../distsql/parser/core/DistSQLVisitor.java        |  35 +++--
 .../api/DistSQLStatementParserEngineTest.java      |  30 +++++
 ...=> DropShardingBindingTableRulesStatement.java} |  10 +-
 ... DropShardingBroadcastTableRulesStatement.java} |  10 +-
 ...nt.java => DropShardingTableRuleStatement.java} |   4 +-
 ...hardingBindingTableRulesNotExistsException.java |  19 ++-
 ...rdingBroadcastTableRulesNotExistsException.java |  19 ++-
 .../text/distsql/rdl/RDLBackendHandlerFactory.java |  18 ++-
 ...ropShardingBindingTableRulesBackendHandler.java |  67 ++++++++++
 ...pShardingBroadcastTableRulesBackendHandler.java |  67 ++++++++++
 ...va => DropShardingTableRuleBackendHandler.java} |  72 +++++------
 ...hardingBindingTableRulesBackendHandlerTest.java | 111 ++++++++++++++++
 ...rdingBroadcastTableRulesBackendHandlerTest.java | 111 ++++++++++++++++
 .../DropShardingTableRuleBackendHandlerTest.java   | 141 +++++++++++++++++++++
 .../frontend/mysql/err/MySQLErrPacketFactory.java  |   8 ++
 18 files changed, 643 insertions(+), 101 deletions(-)

diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-core/src/main/java/org/apache/shardingsphere/db/protocol/error/CommonErrorCode.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-core/src/main/java/org/apache/shardingsphere/db/protocol/error/CommonErrorCode.java
index 7625fc7..ea48a7a 100644
--- a/shardingsphere-db-protocol/shardingsphere-db-protocol-core/src/main/java/org/apache/shardingsphere/db/protocol/error/CommonErrorCode.java
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-core/src/main/java/org/apache/shardingsphere/db/protocol/error/CommonErrorCode.java
@@ -56,7 +56,11 @@ public enum CommonErrorCode implements SQLErrorCode {
     DUPLICATE_TABLE(1113, "C1113", "Duplicate table names %s."),
     
     SHARDING_BROADCAST_EXIST(1114, "C1114", "Sharding broadcast table rules already exist in schema %s."),
-    
+
+    SHARDING_BINDING_TABLE_RULES_NOT_EXIST(1115, "C1115", "Sharding binding table rules not exist in schema %s."),
+
+    SHARDING_BROADCAST_TABLE_RULES_NOT_EXIST(1116, "C1116", "Sharding broadcast table rules not exist in schema %s."),
+
     SCALING_JOB_NOT_EXIST(1201, "C1201", "Scaling job %s does not exist."),
     
     SCALING_OPERATE_FAILED(1209, "C1209", "Scaling Operate Failed: [%s]"),
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 9236601..61d6751 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
@@ -171,8 +171,16 @@ dropReplicaQueryRule
     : DROP REPLICA_QUERY RULE LP IDENTIFIER (COMMA IDENTIFIER)* RP
     ;
 
-dropShardingRule
-    : DROP SHARDING RULE LP tableName (COMMA tableName)* RP
+dropShardingTableRule
+    : DROP SHARDING TABLE RULE tableName (COMMA tableName)*
+    ;
+
+dropShardingBindingTableRules
+    : DROP SHARDING BINDING TABLE RULES
+    ;
+
+dropShardingBroadcastTableRules
+    : DROP SHARDING BROADCAST TABLE RULES
     ;
 
 showShardingRule
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 b626eab..08835c1 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
@@ -28,7 +28,9 @@ execute
     | alterShardingTableRule
     | alterShardingBindingTableRules
     | alterShardingBroadcastTableRules
-    | dropShardingRule
+    | dropShardingTableRule
+    | dropShardingBindingTableRules
+    | dropShardingBroadcastTableRules
     | createReplicaQueryRule
     | alterReplicaQueryRule
     | dropReplicaQueryRule
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 2becd41..75c9c1a 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
@@ -36,7 +36,6 @@ import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.D
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.DropReplicaQueryRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.DropResourceContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.DropScalingJobContext;
-import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.DropShardingRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.FunctionDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ReplicaQueryRuleDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ResetScalingJobContext;
@@ -73,7 +72,9 @@ import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.Create
 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;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingBindingTableRulesStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingBroadcastTableRulesStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingTableRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowResourcesStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowRuleStatement;
 import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
@@ -229,7 +230,26 @@ public final class DistSQLVisitor extends DistSQLStatementBaseVisitor<ASTNode> {
         }
         return result;
     }
-    
+
+    @Override
+    public ASTNode visitDropShardingTableRule(final DistSQLStatementParser.DropShardingTableRuleContext ctx) {
+        DropShardingTableRuleStatement result = new DropShardingTableRuleStatement();
+        for (TableNameContext each : ctx.tableName()) {
+            result.getTableNames().add((TableNameSegment) visit(each));
+        }
+        return result;
+    }
+
+    @Override
+    public ASTNode visitDropShardingBindingTableRules(final DistSQLStatementParser.DropShardingBindingTableRulesContext ctx) {
+        return new DropShardingBindingTableRulesStatement();
+    }
+
+    @Override
+    public ASTNode visitDropShardingBroadcastTableRules(final DistSQLStatementParser.DropShardingBroadcastTableRulesContext ctx) {
+        return new DropShardingBroadcastTableRulesStatement();
+    }
+
     @Override
     public ASTNode visitAlterReplicaQueryRuleDefinition(final AlterReplicaQueryRuleDefinitionContext ctx) {
         ReadwriteSplittingRuleSegment result = new ReadwriteSplittingRuleSegment();
@@ -290,15 +310,6 @@ public final class DistSQLVisitor extends DistSQLStatementBaseVisitor<ASTNode> {
     }
     
     @Override
-    public ASTNode visitDropShardingRule(final DropShardingRuleContext ctx) {
-        DropShardingRuleStatement result = new DropShardingRuleStatement();
-        for (TableNameContext each : ctx.tableName()) {
-            result.getTableNames().add((TableNameSegment) visit(each));
-        }
-        return result;
-    }
-    
-    @Override
     public ASTNode visitTableName(final TableNameContext ctx) {
         return new TableNameSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), new IdentifierValue(ctx.getText()));
     }
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 24cf2ed..3f0180d 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
@@ -28,12 +28,16 @@ import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.Create
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingBroadcastTableRulesStatement;
 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.distsql.parser.statement.rdl.drop.impl.DropShardingBindingTableRulesStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingBroadcastTableRulesStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingTableRuleStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.stream.Collectors;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
@@ -72,6 +76,12 @@ public final class DistSQLStatementParserEngineTest {
 
     private static final String RDL_ALTER_SHARDING_BROADCAST_TABLE_RULES = "ALTER SHARDING BROADCAST TABLE RULES(t_1,t_2)";
 
+    private static final String RDL_DROP_SHARDING_TABLE_RULE = "DROP SHARDING TABLE RULE t_order,t_order_item";
+
+    private static final String RDL_DROP_SHARDING_BINDING_TABLE_RULES = "DROP SHARDING BINDING TABLE RULES";
+
+    private static final String RDL_DROP_SHARDING_BROADCAST_TABLE_RULES = "DROP SHARDING BROADCAST TABLE RULES";
+
     private final DistSQLStatementParserEngine engine = new DistSQLStatementParserEngine();
     
     @Test
@@ -200,4 +210,24 @@ public final class DistSQLStatementParserEngineTest {
         assertTrue(sqlStatement instanceof AlterShardingBroadcastTableRulesStatement);
         assertThat(((AlterShardingBroadcastTableRulesStatement) sqlStatement).getTables(), is(Arrays.asList("t_1", "t_2")));
     }
+
+    @Test
+    public void assertParseDropShardingTableRule() {
+        SQLStatement sqlStatement = engine.parse(RDL_DROP_SHARDING_TABLE_RULE);
+        assertTrue(sqlStatement instanceof DropShardingTableRuleStatement);
+        assertThat(((DropShardingTableRuleStatement) sqlStatement).getTableNames().stream().map(each -> each.getIdentifier().getValue()).collect(Collectors.toList()),
+                is(Arrays.asList("t_order", "t_order_item")));
+    }
+
+    @Test
+    public void assertParseDropShardingBindingTableRules() {
+        SQLStatement sqlStatement = engine.parse(RDL_DROP_SHARDING_BINDING_TABLE_RULES);
+        assertTrue(sqlStatement instanceof DropShardingBindingTableRulesStatement);
+    }
+
+    @Test
+    public void assertParseDropShardingBroadcastTableRules() {
+        SQLStatement sqlStatement = engine.parse(RDL_DROP_SHARDING_BROADCAST_TABLE_RULES);
+        assertTrue(sqlStatement instanceof DropShardingBroadcastTableRulesStatement);
+    }
 }
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingBindingTableRulesStatement.java
similarity index 74%
copy from shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java
copy to shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingBindingTableRulesStatement.java
index 450ba55..fbff731 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingBindingTableRulesStatement.java
@@ -19,16 +19,10 @@ package org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl;
 
 import lombok.Getter;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.DropRDLStatement;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
-
-import java.util.Collection;
-import java.util.LinkedList;
 
 /**
- * Drop sharding rule statement.
+ * Drop sharding binding table rules statement.
  */
 @Getter
-public final class DropShardingRuleStatement extends DropRDLStatement {
-    
-    private final Collection<TableNameSegment> tableNames = new LinkedList<>();
+public final class DropShardingBindingTableRulesStatement extends DropRDLStatement {
 }
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingBroadcastTableRulesStatement.java
similarity index 74%
copy from shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java
copy to shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingBroadcastTableRulesStatement.java
index 450ba55..e53ef0f 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingBroadcastTableRulesStatement.java
@@ -19,16 +19,10 @@ package org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl;
 
 import lombok.Getter;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.DropRDLStatement;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
-
-import java.util.Collection;
-import java.util.LinkedList;
 
 /**
- * Drop sharding rule statement.
+ * Drop sharding broadcast table rules statement.
  */
 @Getter
-public final class DropShardingRuleStatement extends DropRDLStatement {
-    
-    private final Collection<TableNameSegment> tableNames = new LinkedList<>();
+public final class DropShardingBroadcastTableRulesStatement extends DropRDLStatement {
 }
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingTableRuleStatement.java
similarity index 91%
copy from shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java
copy to shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingTableRuleStatement.java
index 450ba55..b41f89d 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingTableRuleStatement.java
@@ -25,10 +25,10 @@ import java.util.Collection;
 import java.util.LinkedList;
 
 /**
- * Drop sharding rule statement.
+ * Drop sharding table rule statement.
  */
 @Getter
-public final class DropShardingRuleStatement extends DropRDLStatement {
+public final class DropShardingTableRuleStatement extends DropRDLStatement {
     
     private final Collection<TableNameSegment> tableNames = new LinkedList<>();
 }
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/exception/ShardingBindingTableRulesNotExistsException.java
similarity index 62%
copy from shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java
copy to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/exception/ShardingBindingTableRulesNotExistsException.java
index 450ba55..d207734d 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/exception/ShardingBindingTableRulesNotExistsException.java
@@ -15,20 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl;
+package org.apache.shardingsphere.proxy.backend.exception;
 
 import lombok.Getter;
-import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.DropRDLStatement;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
-
-import java.util.Collection;
-import java.util.LinkedList;
+import lombok.RequiredArgsConstructor;
 
 /**
- * Drop sharding rule statement.
+ * Sharding binding table rules not exists exception.
  */
+@RequiredArgsConstructor
 @Getter
-public final class DropShardingRuleStatement extends DropRDLStatement {
-    
-    private final Collection<TableNameSegment> tableNames = new LinkedList<>();
+public final class ShardingBindingTableRulesNotExistsException extends BackendException {
+
+    private static final long serialVersionUID = -1930065143932541970L;
+
+    private final String schemaName;
 }
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/exception/ShardingBroadcastTableRulesNotExistsException.java
similarity index 62%
rename from shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/exception/ShardingBroadcastTableRulesNotExistsException.java
index 450ba55..33cafff 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropShardingRuleStatement.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/exception/ShardingBroadcastTableRulesNotExistsException.java
@@ -15,20 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl;
+package org.apache.shardingsphere.proxy.backend.exception;
 
 import lombok.Getter;
-import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.DropRDLStatement;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
-
-import java.util.Collection;
-import java.util.LinkedList;
+import lombok.RequiredArgsConstructor;
 
 /**
- * Drop sharding rule statement.
+ * Sharding broadcast table rules not exists exception.
  */
+@RequiredArgsConstructor
 @Getter
-public final class DropShardingRuleStatement extends DropRDLStatement {
-    
-    private final Collection<TableNameSegment> tableNames = new LinkedList<>();
+public final class ShardingBroadcastTableRulesNotExistsException extends BackendException {
+
+    private static final long serialVersionUID = 2816401356514609173L;
+
+    private final String schemaName;
 }
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 1f67d99..8b38138 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
@@ -30,7 +30,9 @@ import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.Create
 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;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingBindingTableRulesStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingBroadcastTableRulesStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingTableRuleStatement;
 import org.apache.shardingsphere.infra.context.metadata.impl.StandardMetaDataContexts;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
@@ -49,7 +51,9 @@ import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.CreateShard
 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;
-import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.DropShardingRuleBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.DropShardingBindingTableRulesBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.DropShardingBroadcastTableRulesBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.DropShardingTableRuleBackendHandler;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateDatabaseStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropDatabaseStatement;
@@ -129,8 +133,14 @@ public final class RDLBackendHandlerFactory {
         if (sqlStatement instanceof DropDatabaseStatement) {
             return Optional.of(new DropDatabaseBackendHandler((DropDatabaseStatement) sqlStatement));
         }
-        if (sqlStatement instanceof DropShardingRuleStatement) {
-            return Optional.of(new DropShardingRuleBackendHandler((DropShardingRuleStatement) sqlStatement, backendConnection));
+        if (sqlStatement instanceof DropShardingTableRuleStatement) {
+            return Optional.of(new DropShardingTableRuleBackendHandler((DropShardingTableRuleStatement) sqlStatement, backendConnection));
+        }
+        if (sqlStatement instanceof DropShardingBindingTableRulesStatement) {
+            return Optional.of(new DropShardingBindingTableRulesBackendHandler((DropShardingBindingTableRulesStatement) sqlStatement, backendConnection));
+        }
+        if (sqlStatement instanceof DropShardingBroadcastTableRulesStatement) {
+            return Optional.of(new DropShardingBroadcastTableRulesBackendHandler((DropShardingBroadcastTableRulesStatement) sqlStatement, backendConnection));
         }
         return Optional.empty();
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingBindingTableRulesBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingBindingTableRulesBackendHandler.java
new file mode 100644
index 0000000..c9d0704
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingBindingTableRulesBackendHandler.java
@@ -0,0 +1,67 @@
+/*
+ * 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.drop.impl.DropShardingBindingTableRulesStatement;
+import org.apache.shardingsphere.governance.core.registry.listener.event.rule.RuleConfigurationsAlteredEvent;
+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.ShardingBindingTableRulesNotExistsException;
+import org.apache.shardingsphere.proxy.backend.exception.ShardingRuleNotExistedException;
+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 java.util.Optional;
+
+/**
+ * Drop sharding binding table rules backend handler.
+ */
+public final class DropShardingBindingTableRulesBackendHandler extends SchemaRequiredBackendHandler<DropShardingBindingTableRulesStatement> {
+
+    public DropShardingBindingTableRulesBackendHandler(final DropShardingBindingTableRulesStatement sqlStatement, final BackendConnection backendConnection) {
+        super(sqlStatement, backendConnection);
+    }
+    
+    @Override
+    public ResponseHeader execute(final String schemaName, final DropShardingBindingTableRulesStatement sqlStatement) {
+        Optional<ShardingRuleConfiguration> shardingRuleConfiguration = getShardingRuleConfiguration(schemaName);
+        if (!shardingRuleConfiguration.isPresent()) {
+            throw new ShardingRuleNotExistedException();
+        }
+        if (shardingRuleConfiguration.get().getBindingTableGroups().isEmpty()) {
+            throw new ShardingBindingTableRulesNotExistsException(schemaName);
+        }
+        shardingRuleConfiguration.get().getBindingTableGroups().clear();
+        post(schemaName);
+        return new UpdateResponseHeader(sqlStatement);
+    }
+
+    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) {
+        // TODO should use RuleConfigurationsChangeEvent
+        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations()));
+        // TODO Need to get the executed feedback from registry center for returning.
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingBroadcastTableRulesBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingBroadcastTableRulesBackendHandler.java
new file mode 100644
index 0000000..69aac63
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingBroadcastTableRulesBackendHandler.java
@@ -0,0 +1,67 @@
+/*
+ * 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.drop.impl.DropShardingBroadcastTableRulesStatement;
+import org.apache.shardingsphere.governance.core.registry.listener.event.rule.RuleConfigurationsAlteredEvent;
+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.ShardingBroadcastTableRulesNotExistsException;
+import org.apache.shardingsphere.proxy.backend.exception.ShardingRuleNotExistedException;
+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 java.util.Optional;
+
+/**
+ * Drop sharding broadcast table rules backend handler.
+ */
+public final class DropShardingBroadcastTableRulesBackendHandler extends SchemaRequiredBackendHandler<DropShardingBroadcastTableRulesStatement> {
+
+    public DropShardingBroadcastTableRulesBackendHandler(final DropShardingBroadcastTableRulesStatement sqlStatement, final BackendConnection backendConnection) {
+        super(sqlStatement, backendConnection);
+    }
+    
+    @Override
+    public ResponseHeader execute(final String schemaName, final DropShardingBroadcastTableRulesStatement sqlStatement) {
+        Optional<ShardingRuleConfiguration> shardingRuleConfiguration = getShardingRuleConfiguration(schemaName);
+        if (!shardingRuleConfiguration.isPresent()) {
+            throw new ShardingRuleNotExistedException();
+        }
+        if (shardingRuleConfiguration.get().getBroadcastTables().isEmpty()) {
+            throw new ShardingBroadcastTableRulesNotExistsException(schemaName);
+        }
+        shardingRuleConfiguration.get().getBroadcastTables().clear();
+        post(schemaName);
+        return new UpdateResponseHeader(sqlStatement);
+    }
+
+    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) {
+        // TODO should use RuleConfigurationsChangeEvent
+        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations()));
+        // TODO Need to get the executed feedback from registry center for returning.
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingRuleBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingTableRuleBackendHandler.java
similarity index 53%
rename from shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingRuleBackendHandler.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingTableRuleBackendHandler.java
index 079e16e..9763d7f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingRuleBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingTableRuleBackendHandler.java
@@ -17,52 +17,50 @@
 
 package org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl;
 
-import com.google.common.base.Preconditions;
-import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingRuleStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingTableRuleStatement;
 import org.apache.shardingsphere.governance.core.registry.listener.event.rule.RuleConfigurationsAlteredEvent;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 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.exception.TablesInUsedException;
 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.rule.ShardingRule;
+import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.LinkedList;
 import java.util.Optional;
 import java.util.stream.Collectors;
 
 /**
- * Drop sharding rule backend handler.
+ * Drop sharding table rule backend handler.
  */
-public final class DropShardingRuleBackendHandler extends SchemaRequiredBackendHandler<DropShardingRuleStatement> {
+public final class DropShardingTableRuleBackendHandler extends SchemaRequiredBackendHandler<DropShardingTableRuleStatement> {
     
-    public DropShardingRuleBackendHandler(final DropShardingRuleStatement sqlStatement, final BackendConnection backendConnection) {
+    public DropShardingTableRuleBackendHandler(final DropShardingTableRuleStatement sqlStatement, final BackendConnection backendConnection) {
         super(sqlStatement, backendConnection);
     }
     
     @Override
-    public ResponseHeader execute(final String schemaName, final DropShardingRuleStatement sqlStatement) {
+    public ResponseHeader execute(final String schemaName, final DropShardingTableRuleStatement sqlStatement) {
         Collection<String> tableNames = sqlStatement.getTableNames().stream().map(each -> each.getIdentifier().getValue()).collect(Collectors.toList());
-        check(schemaName, tableNames);
-        drop(schemaName, tableNames);
+        ShardingRuleConfiguration shardingRuleConfiguration = check(schemaName, tableNames);
+        drop(shardingRuleConfiguration, tableNames);
         post(schemaName);
         return new UpdateResponseHeader(sqlStatement);
     }
     
-    private void check(final String schemaName, final Collection<String> tableNames) {
-        ShardingSphereMetaData metaData = ProxyContext.getInstance().getMetaData(schemaName);
-        Optional<ShardingRule> shardingRule = metaData.getRuleMetaData().getRules().stream().filter(each -> each instanceof ShardingRule).map(each -> (ShardingRule) each).findFirst();
-        if (!shardingRule.isPresent()) {
-            throw new ShardingTableRuleNotExistedException(tableNames);
+    private ShardingRuleConfiguration check(final String schemaName, final Collection<String> tableNames) {
+        Optional<ShardingRuleConfiguration> shardingRuleConfiguration = getShardingRuleConfiguration(schemaName);
+        if (!shardingRuleConfiguration.isPresent()) {
+            throw new ShardingRuleNotExistedException();
         }
-        Collection<String> shardingTableNames = getShardingTableNames(shardingRule.get());
+        Collection<String> shardingTableNames = getShardingTables(shardingRuleConfiguration.get());
         Collection<String> notExistedTableNames = tableNames.stream().filter(each -> !shardingTableNames.contains(each)).collect(Collectors.toList());
         if (!notExistedTableNames.isEmpty()) {
             throw new ShardingTableRuleNotExistedException(notExistedTableNames);
@@ -71,45 +69,33 @@ public final class DropShardingRuleBackendHandler extends SchemaRequiredBackendH
         if (!inUsedTableNames.isEmpty()) {
             throw new TablesInUsedException(inUsedTableNames);
         }
+        return shardingRuleConfiguration.get();
     }
-    
-    private Collection<String> getShardingTableNames(final ShardingRule shardingRule) {
-        Collection<String> result = new LinkedList<>(shardingRule.getTables());
-        result.addAll(shardingRule.getBroadcastTables());
+
+    private Collection<String> getShardingTables(final ShardingRuleConfiguration shardingRuleConfiguration) {
+        Collection<String> result = new LinkedList<>();
+        result.addAll(shardingRuleConfiguration.getTables().stream().map(ShardingTableRuleConfiguration::getLogicTable).collect(Collectors.toList()));
+        result.addAll(shardingRuleConfiguration.getAutoTables().stream().map(ShardingAutoTableRuleConfiguration::getLogicTable).collect(Collectors.toList()));
         return result;
     }
     
-    private void drop(final String schemaName, final Collection<String> tableNames) {
-        Optional<ShardingRuleConfiguration> shardingRuleConfig = ProxyContext.getInstance().getMetaData(schemaName)
-                .getRuleMetaData().getConfigurations().stream().filter(each -> each instanceof ShardingRuleConfiguration).map(each -> (ShardingRuleConfiguration) each).findFirst();
-        Preconditions.checkState(shardingRuleConfig.isPresent());
+    private void drop(final ShardingRuleConfiguration shardingRuleConfiguration, final Collection<String> tableNames) {
         // TODO add global lock
         for (String each : tableNames) {
-            dropShardingTable(each, shardingRuleConfig.get());
-            dropBroadcastTable(each, shardingRuleConfig.get());
-            dropBindingTable(each, shardingRuleConfig.get());
+            dropShardingTable(each, shardingRuleConfiguration);
         }
     }
     
     private void dropShardingTable(final String tableName, final ShardingRuleConfiguration shardingRuleConfig) {
         shardingRuleConfig.getTables().removeAll(shardingRuleConfig.getTables().stream().filter(each -> tableName.equalsIgnoreCase(each.getLogicTable())).collect(Collectors.toList()));
+        shardingRuleConfig.getAutoTables().removeAll(shardingRuleConfig.getAutoTables().stream().filter(each -> tableName.equalsIgnoreCase(each.getLogicTable())).collect(Collectors.toList()));
     }
-    
-    private void dropBroadcastTable(final String tableName, final ShardingRuleConfiguration shardingRuleConfig) {
-        shardingRuleConfig.getBroadcastTables().removeAll(shardingRuleConfig.getBroadcastTables().stream().filter(tableName::equalsIgnoreCase).collect(Collectors.toList()));
-    }
-    
-    private void dropBindingTable(final String tableName, final ShardingRuleConfiguration shardingRuleConfig) {
-        Collection<String> toBeDroppedGroups = shardingRuleConfig.getBindingTableGroups().stream().filter(each -> Arrays.asList(each.split(",")).contains(tableName)).collect(Collectors.toList());
-        shardingRuleConfig.getBindingTableGroups().removeAll(toBeDroppedGroups);
-        Collection<String> newGroups = toBeDroppedGroups.stream().map(each -> createBindingTableGroupWithoutTableName(tableName, each)).collect(Collectors.toList());
-        shardingRuleConfig.getBindingTableGroups().addAll(newGroups);
-    }
-    
-    private String createBindingTableGroupWithoutTableName(final String expectedTable, final String originalBindingTableGroup) {
-        return Arrays.stream(originalBindingTableGroup.split(",")).filter(each -> !each.trim().equalsIgnoreCase(expectedTable)).collect(Collectors.joining(","));
+
+    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) {
         // TODO should use RuleConfigurationsChangeEvent
         ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations()));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingBindingTableRulesBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingBindingTableRulesBackendHandlerTest.java
new file mode 100644
index 0000000..9f7b0d6
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingBindingTableRulesBackendHandlerTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.drop.impl.DropShardingBindingTableRulesStatement;
+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.ShardingBindingTableRulesNotExistsException;
+import org.apache.shardingsphere.proxy.backend.exception.ShardingRuleNotExistedException;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+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 DropShardingBindingTableRulesBackendHandlerTest {
+    
+    @Mock
+    private BackendConnection backendConnection;
+    
+    @Mock
+    private DropShardingBindingTableRulesStatement sqlStatement;
+    
+    @Mock
+    private MetaDataContexts metaDataContexts;
+    
+    @Mock
+    private TransactionContexts transactionContexts;
+    
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+    
+    @Mock
+    private ShardingSphereRuleMetaData ruleMetaData;
+
+    private DropShardingBindingTableRulesBackendHandler handler = new DropShardingBindingTableRulesBackendHandler(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(expected = ShardingBindingTableRulesNotExistsException.class)
+    public void assertExecuteWithNotExistBindingTableRule() {
+        when(ruleMetaData.getConfigurations()).thenReturn(Arrays.asList(new ShardingRuleConfiguration()));
+        handler.execute("test", sqlStatement);
+    }
+
+    @Test
+    public void assertExecute() {
+        when(ruleMetaData.getConfigurations()).thenReturn(buildShardingConfigurations());
+        ResponseHeader responseHeader = handler.execute("test", sqlStatement);
+        assertNotNull(responseHeader);
+        assertTrue(responseHeader instanceof UpdateResponseHeader);
+        ShardingRuleConfiguration shardingRuleConfiguration = (ShardingRuleConfiguration) ProxyContext.getInstance()
+                .getMetaData("test").getRuleMetaData().getConfigurations().iterator().next();
+        assertTrue(shardingRuleConfiguration.getBindingTableGroups().isEmpty());
+    }
+    
+    private Collection<RuleConfiguration> buildShardingConfigurations() {
+        ShardingRuleConfiguration configuration = new ShardingRuleConfiguration();
+        configuration.getTables().add(new ShardingTableRuleConfiguration("t_order_item"));
+        configuration.getAutoTables().add(new ShardingAutoTableRuleConfiguration("t_order"));
+        configuration.getBindingTableGroups().add("t_order,t_order_item");
+        return new ArrayList<>(Collections.singletonList(configuration));
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingBroadcastTableRulesBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingBroadcastTableRulesBackendHandlerTest.java
new file mode 100644
index 0000000..9cf1c60
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingBroadcastTableRulesBackendHandlerTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.drop.impl.DropShardingBroadcastTableRulesStatement;
+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.ShardingBroadcastTableRulesNotExistsException;
+import org.apache.shardingsphere.proxy.backend.exception.ShardingRuleNotExistedException;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+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 DropShardingBroadcastTableRulesBackendHandlerTest {
+    
+    @Mock
+    private BackendConnection backendConnection;
+    
+    @Mock
+    private DropShardingBroadcastTableRulesStatement sqlStatement;
+    
+    @Mock
+    private MetaDataContexts metaDataContexts;
+    
+    @Mock
+    private TransactionContexts transactionContexts;
+    
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+    
+    @Mock
+    private ShardingSphereRuleMetaData ruleMetaData;
+    
+    private DropShardingBroadcastTableRulesBackendHandler handler = new DropShardingBroadcastTableRulesBackendHandler(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(expected = ShardingBroadcastTableRulesNotExistsException.class)
+    public void assertExecuteWithNotExistBroadcastTableRule() {
+        when(ruleMetaData.getConfigurations()).thenReturn(Arrays.asList(new ShardingRuleConfiguration()));
+        handler.execute("test", sqlStatement);
+    }
+
+    @Test
+    public void assertExecute() {
+        when(ruleMetaData.getConfigurations()).thenReturn(buildShardingConfigurations());
+        ResponseHeader responseHeader = handler.execute("test", sqlStatement);
+        assertNotNull(responseHeader);
+        assertTrue(responseHeader instanceof UpdateResponseHeader);
+        ShardingRuleConfiguration shardingRuleConfiguration = (ShardingRuleConfiguration) ProxyContext.getInstance()
+                .getMetaData("test").getRuleMetaData().getConfigurations().iterator().next();
+        assertTrue(shardingRuleConfiguration.getBroadcastTables().isEmpty());
+    }
+    
+    private Collection<RuleConfiguration> buildShardingConfigurations() {
+        ShardingRuleConfiguration configuration = new ShardingRuleConfiguration();
+        configuration.getTables().add(new ShardingTableRuleConfiguration("t_order_item"));
+        configuration.getAutoTables().add(new ShardingAutoTableRuleConfiguration("t_order"));
+        configuration.getBroadcastTables().add("t_order");
+        return new ArrayList<>(Collections.singletonList(configuration));
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingTableRuleBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingTableRuleBackendHandlerTest.java
new file mode 100644
index 0000000..04216a3
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingTableRuleBackendHandlerTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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.drop.impl.DropShardingTableRuleStatement;
+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.infra.metadata.schema.ShardingSphereSchema;
+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.exception.TablesInUsedException;
+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.sql.parser.sql.common.segment.generic.table.TableNameSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.stream.Collectors;
+
+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 DropShardingTableRuleBackendHandlerTest {
+    
+    @Mock
+    private BackendConnection backendConnection;
+    
+    @Mock
+    private DropShardingTableRuleStatement sqlStatement;
+    
+    @Mock
+    private MetaDataContexts metaDataContexts;
+    
+    @Mock
+    private TransactionContexts transactionContexts;
+    
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+    
+    @Mock
+    private ShardingSphereRuleMetaData ruleMetaData;
+
+    @Mock
+    private ShardingSphereSchema shardingSphereSchema;
+    
+    private DropShardingTableRuleBackendHandler handler = new DropShardingTableRuleBackendHandler(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);
+        when(shardingSphereMetaData.getSchema()).thenReturn(shardingSphereSchema);
+    }
+    
+    @Test(expected = ShardingRuleNotExistedException.class)
+    public void assertExecuteWithoutShardingRule() {
+        handler.execute("test", sqlStatement);
+    }
+
+    @Test(expected = ShardingTableRuleNotExistedException.class)
+    public void assertExecuteWithNotExistTableRule() {
+        TableNameSegment tableRuleSegment = new TableNameSegment(0, 3, new IdentifierValue("t_order"));
+        when(ruleMetaData.getConfigurations()).thenReturn(Arrays.asList(new ShardingRuleConfiguration()));
+        when(sqlStatement.getTableNames()).thenReturn(Arrays.asList(tableRuleSegment));
+        handler.execute("test", sqlStatement);
+    }
+
+    @Test(expected = TablesInUsedException.class)
+    public void assertExecuteWithTableRuleInUsed() {
+        TableNameSegment tableRuleSegment = new TableNameSegment(0, 3, new IdentifierValue("t_order"));
+        when(ruleMetaData.getConfigurations()).thenReturn(buildShardingConfigurations());
+        when(sqlStatement.getTableNames()).thenReturn(Arrays.asList(tableRuleSegment));
+        when(shardingSphereSchema.containsTable(eq("t_order"))).thenReturn(true);
+        handler.execute("test", sqlStatement);
+    }
+
+    @Test
+    public void assertExecute() {
+        TableNameSegment tableRuleSegment = new TableNameSegment(0, 3, new IdentifierValue("t_order"));
+        when(ruleMetaData.getConfigurations()).thenReturn(buildShardingConfigurations());
+        when(sqlStatement.getTableNames()).thenReturn(Arrays.asList(tableRuleSegment));
+        ResponseHeader responseHeader = handler.execute("test", sqlStatement);
+        assertNotNull(responseHeader);
+        assertTrue(responseHeader instanceof UpdateResponseHeader);
+        ShardingRuleConfiguration shardingRuleConfiguration = (ShardingRuleConfiguration) ProxyContext.getInstance()
+                .getMetaData("test").getRuleMetaData().getConfigurations().iterator().next();
+        Collection<String> shardingTables = getShardingTables(shardingRuleConfiguration);
+        assertTrue(!shardingTables.contains("t_order"));
+    }
+    
+    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));
+    }
+
+    private Collection<String> getShardingTables(final ShardingRuleConfiguration shardingRuleConfiguration) {
+        Collection<String> result = new LinkedList<>();
+        result.addAll(shardingRuleConfiguration.getTables().stream().map(ShardingTableRuleConfiguration::getLogicTable).collect(Collectors.toList()));
+        result.addAll(shardingRuleConfiguration.getAutoTables().stream().map(ShardingAutoTableRuleConfiguration::getLogicTable).collect(Collectors.toList()));
+        return result;
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/err/MySQLErrPacketFactory.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/err/MySQLErrPacketFactory.java
index 5b435a2..169f128 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/err/MySQLErrPacketFactory.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/err/MySQLErrPacketFactory.java
@@ -37,7 +37,9 @@ import org.apache.shardingsphere.proxy.backend.exception.ReadwriteSplittingRuleN
 import org.apache.shardingsphere.proxy.backend.exception.ResourceInUsedException;
 import org.apache.shardingsphere.proxy.backend.exception.ResourceNotExistedException;
 import org.apache.shardingsphere.proxy.backend.exception.RuleNotExistsException;
+import org.apache.shardingsphere.proxy.backend.exception.ShardingBindingTableRulesNotExistsException;
 import org.apache.shardingsphere.proxy.backend.exception.ShardingBroadcastTableRulesExistsException;
+import org.apache.shardingsphere.proxy.backend.exception.ShardingBroadcastTableRulesNotExistsException;
 import org.apache.shardingsphere.proxy.backend.exception.ShardingRuleNotExistedException;
 import org.apache.shardingsphere.proxy.backend.exception.ShardingTableRuleExistedException;
 import org.apache.shardingsphere.proxy.backend.exception.ShardingTableRuleNotExistedException;
@@ -155,6 +157,12 @@ public final class MySQLErrPacketFactory {
         if (cause instanceof ShardingTableRuleExistedException) {
             return new MySQLErrPacket(1, CommonErrorCode.SHARDING_TABLE_RULE_EXIST, ((ShardingTableRuleExistedException) cause).getTableNames());
         }
+        if (cause instanceof ShardingBindingTableRulesNotExistsException) {
+            return new MySQLErrPacket(1, CommonErrorCode.SHARDING_BINDING_TABLE_RULES_NOT_EXIST, ((ShardingBindingTableRulesNotExistsException) cause).getSchemaName());
+        }
+        if (cause instanceof ShardingBroadcastTableRulesNotExistsException) {
+            return new MySQLErrPacket(1, CommonErrorCode.SHARDING_BROADCAST_TABLE_RULES_NOT_EXIST, ((ShardingBroadcastTableRulesNotExistsException) cause).getSchemaName());
+        }
         if (cause instanceof ReadwriteSplittingRuleNotExistedException) {
             return new MySQLErrPacket(1, CommonErrorCode.REPLICA_QUERY_RULE_NOT_EXIST);
         }