You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2021/05/21 06:43:59 UTC

[shardingsphere] branch master updated: Add alter & drop encrypt rule handler (#10412)

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

duanzhengqiang 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 3286918  Add alter & drop encrypt rule handler (#10412)
3286918 is described below

commit 328691873120e9c1a639c48a88d19c7d83aa42d3
Author: Haoran Meng <me...@gmail.com>
AuthorDate: Fri May 21 14:43:27 2021 +0800

    Add alter & drop encrypt rule handler (#10412)
    
    * Add alter & drop encrypt rule handler
    
    * Add alter & drop encrypt rule handler
    
    Co-authored-by: menghaoranss <me...@apache.org>
---
 .../db/protocol/error/CommonErrorCode.java         |   2 +
 .../api/DistSQLStatementParserEngineTest.java      |   2 +-
 .../rdl/drop/impl/DropEncryptRuleStatement.java    |   2 +-
 .../exception/EncryptRulesNotExistedException.java |  13 +-
 .../text/distsql/rdl/RDLBackendHandlerFactory.java |  10 ++
 .../rdl/impl/AlterEncryptRuleBackendHandler.java   | 117 ++++++++++++++++++
 .../rdl/impl/DropEncryptRuleBackendHandler.java    |  89 ++++++++++++++
 .../impl/AlterEncryptRuleBackendHandlerTest.java   | 136 +++++++++++++++++++++
 .../impl/DropEncryptRuleBackendHandlerTest.java    | 120 ++++++++++++++++++
 .../frontend/mysql/err/MySQLErrPacketFactory.java  |   4 +
 10 files changed, 487 insertions(+), 8 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 8599d32..c43e5cd 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
@@ -75,6 +75,8 @@ public enum CommonErrorCode implements SQLErrorCode {
 
     INVALID_ENCRYPTORS(1123, "C1123", "Invalid encryptors %s."),
 
+    ENCRYPT_RULES_NOT_EXIST(1124, "C1124", "Encrypt rules %s do not exist."),
+
     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/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 5f50043..a69ea43 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
@@ -411,6 +411,6 @@ public final class DistSQLStatementParserEngineTest {
     public void assertParseDropEncryptRule() {
         SQLStatement sqlStatement = engine.parse(RDL_DROP_ENCRYPT_RULE);
         assertTrue(sqlStatement instanceof DropEncryptRuleStatement);
-        assertThat(((DropEncryptRuleStatement) sqlStatement).getTableNames(), is(Arrays.asList("t_encrypt", "t_encrypt_order")));
+        assertThat(((DropEncryptRuleStatement) sqlStatement).getTables(), is(Arrays.asList("t_encrypt", "t_encrypt_order")));
     }
 }
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropEncryptRuleStatement.java b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropEncryptRuleStatement.java
index 2cfa03c..afc1128 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropEncryptRuleStatement.java
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropEncryptRuleStatement.java
@@ -30,5 +30,5 @@ import java.util.Collection;
 @Getter
 public final class DropEncryptRuleStatement extends DropRDLStatement {
     
-    private final Collection<String> tableNames;
+    private final Collection<String> tables;
 }
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropEncryptRuleStatement.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/exception/EncryptRulesNotExistedException.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/DropEncryptRuleStatement.java
copy to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/exception/EncryptRulesNotExistedException.java
index 2cfa03c..eeeb762 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rdl/drop/impl/DropEncryptRuleStatement.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/exception/EncryptRulesNotExistedException.java
@@ -15,20 +15,21 @@
  * 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 lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.DropRDLStatement;
 
 import java.util.Collection;
 
 /**
- * Drop encrypt rule statement.
+ * Encrypt rules not existed exception.
  */
 @RequiredArgsConstructor
 @Getter
-public final class DropEncryptRuleStatement extends DropRDLStatement {
-    
-    private final Collection<String> tableNames;
+public final class EncryptRulesNotExistedException extends BackendException {
+
+    private static final long serialVersionUID = -3070306238552517428L;
+
+    private final Collection<String> tables;
 }
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 b12a41d..63248d6 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.alter.AlterDatabaseDiscoveryRuleStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterEncryptRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterReadwriteSplittingRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterShardingBindingTableRulesStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterShardingBroadcastTableRulesStatement;
@@ -32,6 +33,7 @@ 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.DropDatabaseDiscoveryRuleStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropEncryptRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropReadwriteSplittingRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropResourceStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingBindingTableRulesStatement;
@@ -44,6 +46,7 @@ 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.AddResourceBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.AlterDatabaseDiscoveryRuleBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.AlterEncryptRuleBackendHandler;
 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.AlterShardingBroadcastTableRulesBackendHandler;
@@ -57,6 +60,7 @@ import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.CreateShard
 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.DropDatabaseDiscoveryRuleBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl.DropEncryptRuleBackendHandler;
 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.DropShardingBindingTableRulesBackendHandler;
@@ -162,6 +166,12 @@ public final class RDLBackendHandlerFactory {
         if (sqlStatement instanceof CreateEncryptRuleStatement) {
             return Optional.of(new CreateEncryptRuleBackendHandler((CreateEncryptRuleStatement) sqlStatement, backendConnection));
         }
+        if (sqlStatement instanceof DropEncryptRuleStatement) {
+            return Optional.of(new DropEncryptRuleBackendHandler((DropEncryptRuleStatement) sqlStatement, backendConnection));
+        }
+        if (sqlStatement instanceof AlterEncryptRuleStatement) {
+            return Optional.of(new AlterEncryptRuleBackendHandler((AlterEncryptRuleStatement) 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/AlterEncryptRuleBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterEncryptRuleBackendHandler.java
new file mode 100644
index 0000000..1844571
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterEncryptRuleBackendHandler.java
@@ -0,0 +1,117 @@
+/*
+ * 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.EncryptRuleSegment;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterEncryptRuleStatement;
+import org.apache.shardingsphere.encrypt.api.config.EncryptRuleConfiguration;
+import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfiguration;
+import org.apache.shardingsphere.encrypt.spi.EncryptAlgorithm;
+import org.apache.shardingsphere.encrypt.yaml.config.YamlEncryptRuleConfiguration;
+import org.apache.shardingsphere.encrypt.yaml.converter.EncryptRuleConverter;
+import org.apache.shardingsphere.governance.core.registry.listener.event.rule.RuleConfigurationsAlteredEvent;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
+import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
+import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
+import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.apache.shardingsphere.proxy.backend.exception.EncryptRulesNotExistedException;
+import org.apache.shardingsphere.proxy.backend.exception.InvalidEncryptorsException;
+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 java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
+/**
+ * Alter encrypt rule backend handler.
+ */
+public final class AlterEncryptRuleBackendHandler extends SchemaRequiredBackendHandler<AlterEncryptRuleStatement> {
+
+    public AlterEncryptRuleBackendHandler(final AlterEncryptRuleStatement sqlStatement, final BackendConnection backendConnection) {
+        super(sqlStatement, backendConnection);
+    }
+    
+    @Override
+    public ResponseHeader execute(final String schemaName, final AlterEncryptRuleStatement sqlStatement) {
+        Optional<EncryptRuleConfiguration> ruleConfig = ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations().stream()
+                .filter(each -> each instanceof EncryptRuleConfiguration).map(each -> (EncryptRuleConfiguration) each).findFirst();
+        if (!ruleConfig.isPresent()) {
+            throw new EncryptRulesNotExistedException(getAlteredTables(sqlStatement));
+        }
+        check(sqlStatement, ruleConfig.get());
+        YamlEncryptRuleConfiguration alterConfig = alter(ruleConfig.get(), sqlStatement);
+        Collection<RuleConfiguration> rules = new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(Collections.singleton(alterConfig));
+        post(schemaName, rules);
+        return new UpdateResponseHeader(sqlStatement);
+    }
+
+    private Collection<String> getAlteredTables(final AlterEncryptRuleStatement sqlStatement) {
+        return sqlStatement.getEncryptRules().stream().map(EncryptRuleSegment::getTableName).collect(Collectors.toList());
+    }
+
+    private void check(final AlterEncryptRuleStatement sqlStatement, final EncryptRuleConfiguration encryptRuleConfiguration) {
+        checkAlteredTables(encryptRuleConfiguration, sqlStatement);
+        checkEncryptors(sqlStatement);
+    }
+
+    private void checkAlteredTables(final EncryptRuleConfiguration encryptRuleConfiguration, final AlterEncryptRuleStatement sqlStatement) {
+        Collection<String> existTables = getExistTables(encryptRuleConfiguration);
+        Collection<String> notExistTables = getAlteredTables(sqlStatement).stream()
+                .filter(each -> !existTables.contains(each)).collect(Collectors.toList());
+        if (!notExistTables.isEmpty()) {
+            throw new EncryptRulesNotExistedException(notExistTables);
+        }
+    }
+
+    private Collection<String> getExistTables(final EncryptRuleConfiguration encryptRuleConfiguration) {
+        return encryptRuleConfiguration.getTables().stream().map(EncryptTableRuleConfiguration::getName).collect(Collectors.toList());
+    }
+
+    private void checkEncryptors(final AlterEncryptRuleStatement sqlStatement) {
+        Collection<String> encryptors = new LinkedHashSet<>();
+        sqlStatement.getEncryptRules().stream().forEach(each -> encryptors.addAll(each.getColumns().stream()
+                .map(column -> column.getEncryptor().getAlgorithmName()).collect(Collectors.toSet())));
+        Collection<String> invalidEncryptors = encryptors.stream().filter(each -> !TypedSPIRegistry.findRegisteredService(EncryptAlgorithm.class, each, new Properties()).isPresent())
+                .collect(Collectors.toList());
+        if (!invalidEncryptors.isEmpty()) {
+            throw new InvalidEncryptorsException(invalidEncryptors);
+        }
+    }
+    
+    private YamlEncryptRuleConfiguration alter(final EncryptRuleConfiguration encryptRuleConfiguration, final AlterEncryptRuleStatement sqlStatement) {
+        YamlEncryptRuleConfiguration alterYamlEncryptRuleConfiguration
+                = EncryptRuleConverter.convert(sqlStatement.getEncryptRules());
+        YamlEncryptRuleConfiguration result = new YamlRuleConfigurationSwapperEngine()
+                .swapToYamlRuleConfigurations(Collections.singletonList(encryptRuleConfiguration)).stream()
+                .map(each -> (YamlEncryptRuleConfiguration) each).findFirst().get();
+        result.getTables().putAll(alterYamlEncryptRuleConfiguration.getTables());
+        result.getEncryptors().putAll(alterYamlEncryptRuleConfiguration.getEncryptors());
+        return result;
+    }
+
+    private void post(final String schemaName, final Collection<RuleConfiguration> rules) {
+        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, rules));
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropEncryptRuleBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropEncryptRuleBackendHandler.java
new file mode 100644
index 0000000..8eb6545
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropEncryptRuleBackendHandler.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 org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropEncryptRuleStatement;
+import org.apache.shardingsphere.encrypt.api.config.EncryptRuleConfiguration;
+import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfiguration;
+import org.apache.shardingsphere.encrypt.yaml.config.YamlEncryptRuleConfiguration;
+import org.apache.shardingsphere.governance.core.registry.listener.event.rule.RuleConfigurationsAlteredEvent;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
+import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
+import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.apache.shardingsphere.proxy.backend.exception.EncryptRulesNotExistedException;
+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 java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Drop encrypt rule backend handler.
+ */
+public final class DropEncryptRuleBackendHandler extends SchemaRequiredBackendHandler<DropEncryptRuleStatement> {
+
+    public DropEncryptRuleBackendHandler(final DropEncryptRuleStatement sqlStatement, final BackendConnection backendConnection) {
+        super(sqlStatement, backendConnection);
+    }
+    
+    @Override
+    public ResponseHeader execute(final String schemaName, final DropEncryptRuleStatement sqlStatement) {
+        Optional<EncryptRuleConfiguration> ruleConfig = ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations().stream()
+                .filter(each -> each instanceof EncryptRuleConfiguration).map(each -> (EncryptRuleConfiguration) each).findFirst();
+        if (!ruleConfig.isPresent()) {
+            throw new EncryptRulesNotExistedException(sqlStatement.getTables());
+        }
+        check(ruleConfig.get(), sqlStatement.getTables());
+        YamlEncryptRuleConfiguration yamlEncryptRuleConfiguration = new YamlRuleConfigurationSwapperEngine()
+                .swapToYamlRuleConfigurations(Collections.singletonList(ruleConfig.get())).stream()
+                .map(each -> (YamlEncryptRuleConfiguration) each).findFirst().get();
+        drop(yamlEncryptRuleConfiguration, sqlStatement.getTables());
+        post(schemaName, new YamlRuleConfigurationSwapperEngine()
+                .swapToRuleConfigurations(Collections.singletonList(yamlEncryptRuleConfiguration)));
+        return new UpdateResponseHeader(sqlStatement);
+    }
+    
+    private void check(final EncryptRuleConfiguration ruleConfig, final Collection<String> droppedTables) {
+        Collection<String> encryptTables = ruleConfig.getTables().stream().map(EncryptTableRuleConfiguration::getName).collect(Collectors.toList());
+        Collection<String> notExistedTables = droppedTables.stream().filter(each -> !encryptTables.contains(each)).collect(Collectors.toList());
+        if (!notExistedTables.isEmpty()) {
+            throw new EncryptRulesNotExistedException(notExistedTables);
+        }
+    }
+
+    private void drop(final YamlEncryptRuleConfiguration yamlEncryptRuleConfiguration, final Collection<String> droppedTables) {
+        for (String each : droppedTables) {
+            dropEncryptors(each, yamlEncryptRuleConfiguration);
+            yamlEncryptRuleConfiguration.getTables().remove(each);
+        }
+    }
+
+    private void dropEncryptors(final String droppedTable, final YamlEncryptRuleConfiguration yamlEncryptRuleConfiguration) {
+        yamlEncryptRuleConfiguration.getTables().get(droppedTable).getColumns()
+                .values().forEach(value -> yamlEncryptRuleConfiguration.getEncryptors().remove(value.getEncryptorName()));
+    }
+    
+    private void post(final String schemaName, final Collection<RuleConfiguration> rules) {
+        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, rules));
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterEncryptRuleBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterEncryptRuleBackendHandlerTest.java
new file mode 100644
index 0000000..9d11e31
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/AlterEncryptRuleBackendHandlerTest.java
@@ -0,0 +1,136 @@
+/*
+ * 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.collect.Maps;
+import org.apache.shardingsphere.distsql.parser.segment.FunctionSegment;
+import org.apache.shardingsphere.distsql.parser.segment.rdl.EncryptColumnSegment;
+import org.apache.shardingsphere.distsql.parser.segment.rdl.EncryptRuleSegment;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterEncryptRuleStatement;
+import org.apache.shardingsphere.encrypt.api.config.EncryptRuleConfiguration;
+import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfiguration;
+import org.apache.shardingsphere.encrypt.spi.EncryptAlgorithm;
+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.spi.ShardingSphereServiceLoader;
+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.EncryptRulesNotExistedException;
+import org.apache.shardingsphere.proxy.backend.exception.InvalidEncryptorsException;
+import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
+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.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 AlterEncryptRuleBackendHandlerTest {
+    
+    @Mock
+    private BackendConnection backendConnection;
+    
+    @Mock
+    private AlterEncryptRuleStatement sqlStatement;
+    
+    @Mock
+    private MetaDataContexts metaDataContexts;
+    
+    @Mock
+    private TransactionContexts transactionContexts;
+    
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+    
+    @Mock
+    private ShardingSphereRuleMetaData ruleMetaData;
+
+    @Mock
+    private EncryptTableRuleConfiguration encryptTableRuleConfiguration;
+    
+    private AlterEncryptRuleBackendHandler handler = new AlterEncryptRuleBackendHandler(sqlStatement, backendConnection);
+    
+    @Before
+    public void setUp() {
+        ShardingSphereServiceLoader.register(EncryptAlgorithm.class);
+        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
+    public void assertExecute() {
+        EncryptRuleSegment encryptRuleSegment = new EncryptRuleSegment("t_encrypt", buildColumns("MD5"));
+        when(sqlStatement.getEncryptRules()).thenReturn(Collections.singletonList(encryptRuleSegment));
+        when(ruleMetaData.getConfigurations()).thenReturn(Collections
+                .singletonList(new EncryptRuleConfiguration(Collections
+                        .singleton(encryptTableRuleConfiguration), Maps.newHashMap())));
+        when(encryptTableRuleConfiguration.getName()).thenReturn("t_encrypt");
+        ResponseHeader responseHeader = handler.execute("test", sqlStatement);
+        assertNotNull(responseHeader);
+        assertTrue(responseHeader instanceof UpdateResponseHeader);
+    }
+    
+    @Test(expected = EncryptRulesNotExistedException.class)
+    public void assertExecuteWithNotExistEncryptRule() {
+        when(ruleMetaData.getConfigurations()).thenReturn(Collections.emptyList());
+        handler.execute("test", sqlStatement);
+    }
+
+    @Test(expected = EncryptRulesNotExistedException.class)
+    public void assertExecuteWithNoAlteredEncryptRules() {
+        EncryptRuleSegment encryptRuleSegment = new EncryptRuleSegment("t_encrypt", buildColumns("MD5"));
+        when(sqlStatement.getEncryptRules()).thenReturn(Collections.singletonList(encryptRuleSegment));
+        when(ruleMetaData.getConfigurations()).thenReturn(Collections.singletonList(new EncryptRuleConfiguration(Collections.emptyList(), Maps.newHashMap())));
+        handler.execute("test", sqlStatement);
+    }
+
+    @Test(expected = InvalidEncryptorsException.class)
+    public void assertExecuteWithInvalidEncryptors() {
+        EncryptRuleSegment encryptRuleSegment = new EncryptRuleSegment("t_encrypt", buildColumns("notExistEncryptor"));
+        when(sqlStatement.getEncryptRules()).thenReturn(Collections.singletonList(encryptRuleSegment));
+        when(ruleMetaData.getConfigurations()).thenReturn(Collections
+                .singletonList(new EncryptRuleConfiguration(Collections
+                        .singleton(encryptTableRuleConfiguration), Maps.newHashMap())));
+        when(encryptTableRuleConfiguration.getName()).thenReturn("t_encrypt");
+        handler.execute("test", sqlStatement);
+    }
+
+    private Collection<EncryptColumnSegment> buildColumns(final String encryptorName) {
+        EncryptColumnSegment encryptColumnSegment = new EncryptColumnSegment();
+        encryptColumnSegment.setName("user_id");
+        encryptColumnSegment.setPlainColumn("user_plain");
+        encryptColumnSegment.setCipherColumn("user_cipher");
+        FunctionSegment functionSegment = new FunctionSegment();
+        functionSegment.setAlgorithmName(encryptorName);
+        encryptColumnSegment.setEncryptor(functionSegment);
+        return Collections.singleton(encryptColumnSegment);
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropEncryptRuleBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropEncryptRuleBackendHandlerTest.java
new file mode 100644
index 0000000..ef8d061
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropEncryptRuleBackendHandlerTest.java
@@ -0,0 +1,120 @@
+/*
+ * 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.collect.Maps;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropEncryptRuleStatement;
+import org.apache.shardingsphere.encrypt.api.config.EncryptRuleConfiguration;
+import org.apache.shardingsphere.encrypt.api.config.rule.EncryptColumnRuleConfiguration;
+import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfiguration;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
+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.EncryptRulesNotExistedException;
+import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
+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.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+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 DropEncryptRuleBackendHandlerTest {
+    
+    @Mock
+    private BackendConnection backendConnection;
+    
+    @Mock
+    private DropEncryptRuleStatement sqlStatement;
+    
+    @Mock
+    private MetaDataContexts metaDataContexts;
+    
+    @Mock
+    private TransactionContexts transactionContexts;
+    
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+    
+    @Mock
+    private ShardingSphereRuleMetaData ruleMetaData;
+
+    @Mock
+    private EncryptTableRuleConfiguration encryptTableRuleConfiguration;
+
+    @Mock
+    private ShardingSphereAlgorithmConfiguration shardingSphereAlgorithmConfiguration;
+    
+    private DropEncryptRuleBackendHandler handler = new DropEncryptRuleBackendHandler(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
+    public void assertExecute() {
+        when(sqlStatement.getTables()).thenReturn(Collections.singletonList("t_encrypt"));
+        Map<String, ShardingSphereAlgorithmConfiguration> encryptors = new HashMap<>(1, 1);
+        encryptors.put("t_encrypt_user_id_MD5", shardingSphereAlgorithmConfiguration);
+        when(ruleMetaData.getConfigurations()).thenReturn(Collections
+                .singletonList(new EncryptRuleConfiguration(Collections
+                        .singleton(encryptTableRuleConfiguration), encryptors)));
+        when(encryptTableRuleConfiguration.getName()).thenReturn("t_encrypt");
+        when(encryptTableRuleConfiguration.getColumns()).thenReturn(buildColumnRuleConfigurations());
+        ResponseHeader responseHeader = handler.execute("test", sqlStatement);
+        assertNotNull(responseHeader);
+        assertTrue(responseHeader instanceof UpdateResponseHeader);
+    }
+    
+    @Test(expected = EncryptRulesNotExistedException.class)
+    public void assertExecuteWithNotExistEncryptRule() {
+        when(ruleMetaData.getConfigurations()).thenReturn(Collections.emptyList());
+        handler.execute("test", sqlStatement);
+    }
+
+    @Test(expected = EncryptRulesNotExistedException.class)
+    public void assertExecuteWithNoDroppedEncryptRule() {
+        when(sqlStatement.getTables()).thenReturn(Collections.singletonList("t_encrypt"));
+        when(ruleMetaData.getConfigurations()).thenReturn(Collections.singletonList(new EncryptRuleConfiguration(Collections.emptyList(), Maps.newHashMap())));
+        handler.execute("test", sqlStatement);
+    }
+
+    private Collection<EncryptColumnRuleConfiguration> buildColumnRuleConfigurations() {
+        return Collections.singleton(new EncryptColumnRuleConfiguration("user_id", "user_cipher",
+                "", "user_plain", "t_encrypt_user_id_MD5"));
+    }
+}
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 46eccd8..35ebfbf 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
@@ -33,6 +33,7 @@ import org.apache.shardingsphere.proxy.backend.exception.DatabaseDiscoveryRuleNo
 import org.apache.shardingsphere.proxy.backend.exception.DuplicateResourceException;
 import org.apache.shardingsphere.proxy.backend.exception.DuplicateTablesException;
 import org.apache.shardingsphere.proxy.backend.exception.EncryptRuleExistsException;
+import org.apache.shardingsphere.proxy.backend.exception.EncryptRulesNotExistedException;
 import org.apache.shardingsphere.proxy.backend.exception.InvalidDatabaseDiscoveryTypesException;
 import org.apache.shardingsphere.proxy.backend.exception.InvalidEncryptorsException;
 import org.apache.shardingsphere.proxy.backend.exception.InvalidLoadBalancersException;
@@ -206,6 +207,9 @@ public final class MySQLErrPacketFactory {
         if (cause instanceof InvalidEncryptorsException) {
             return new MySQLErrPacket(1, CommonErrorCode.INVALID_ENCRYPTORS, ((InvalidEncryptorsException) cause).getEncryptors());
         }
+        if (cause instanceof EncryptRulesNotExistedException) {
+            return new MySQLErrPacket(1, CommonErrorCode.ENCRYPT_RULES_NOT_EXIST, ((EncryptRulesNotExistedException) cause).getTables());
+        }
         return new MySQLErrPacket(1, CommonErrorCode.UNKNOWN_EXCEPTION, cause.getMessage());
     }
 }