You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by wu...@apache.org on 2021/05/28 07:03:58 UTC

[shardingsphere] branch master updated: Refactor CreateEncryptRuleBackendHandler (#10520)

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

wuweijie 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 8c9cd45  Refactor CreateEncryptRuleBackendHandler (#10520)
8c9cd45 is described below

commit 8c9cd4540d9000e324eabd0f32b1c311d5f7abf1
Author: Haoran Meng <me...@gmail.com>
AuthorDate: Fri May 28 15:03:27 2021 +0800

    Refactor CreateEncryptRuleBackendHandler (#10520)
    
    Co-authored-by: menghaoranss <me...@apache.org>
---
 .../db/protocol/error/CommonErrorCode.java         |  2 -
 .../exception/EncryptRuleExistsException.java      | 33 ------------
 .../rdl/impl/CreateEncryptRuleBackendHandler.java  | 59 ++++++++++++++++++----
 .../impl/CreateEncryptRuleBackendHandlerTest.java  | 13 +++--
 .../frontend/mysql/err/MySQLErrPacketFactory.java  |  4 --
 5 files changed, 58 insertions(+), 53 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 b4c0487..3c200fb 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
@@ -61,8 +61,6 @@ public enum CommonErrorCode implements SQLErrorCode {
 
     DATABASE_DISCOVERY_RULES_NOT_EXIST(1120, "C1120", "Database discovery rules %s do not exist in schema %s."),
 
-    ENCRYPT_RULE_EXIST(1122, "C1122", "Encrypt rule already exists in schema %s."),
-
     INVALID_ENCRYPTORS(1123, "C1123", "Invalid encryptors %s."),
 
     ENCRYPT_RULES_NOT_EXIST(1124, "C1124", "Encrypt rules %s do not exist in schema %s."),
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/exception/EncryptRuleExistsException.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/exception/EncryptRuleExistsException.java
deleted file mode 100644
index 39ef700..0000000
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/exception/EncryptRuleExistsException.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.proxy.backend.exception;
-
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
-
-/**
- * Encrypt rule exists exception.
- */
-@RequiredArgsConstructor
-@Getter
-public final class EncryptRuleExistsException extends BackendException {
-
-    private static final long serialVersionUID = -1673568903990383982L;
-
-    private final String schemaName;
-}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateEncryptRuleBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateEncryptRuleBackendHandler.java
index ecc685c..b17ebdc 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateEncryptRuleBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateEncryptRuleBackendHandler.java
@@ -17,20 +17,21 @@
 
 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.create.impl.CreateEncryptRuleStatement;
 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.EncryptRuleStatementConverter;
 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.ShardingSphereServiceLoader;
 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.EncryptRuleExistsException;
+import org.apache.shardingsphere.proxy.backend.exception.DuplicateRuleNamesException;
 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;
@@ -39,6 +40,7 @@ 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;
 
@@ -58,17 +60,30 @@ public final class CreateEncryptRuleBackendHandler extends SchemaRequiredBackend
     @Override
     public ResponseHeader execute(final String schemaName, final CreateEncryptRuleStatement sqlStatement) {
         check(schemaName, sqlStatement);
-        YamlEncryptRuleConfiguration config = EncryptRuleStatementConverter.convert(sqlStatement.getEncryptRules());
-        Collection<RuleConfiguration> rules = new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(Collections.singleton(config));
-        post(schemaName, rules);
+        create(schemaName, sqlStatement);
+        post(schemaName);
         return new UpdateResponseHeader(sqlStatement);
     }
     
     private void check(final String schemaName, final CreateEncryptRuleStatement sqlStatement) {
-        if (ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations().stream().anyMatch(each -> each instanceof EncryptRuleConfiguration)) {
-            throw new EncryptRuleExistsException(schemaName);
-        }
+        checkDuplicateRuleNames(schemaName, sqlStatement);
+        checkEncryptors(sqlStatement);
         // TODO check resource
+    }
+
+    private void checkDuplicateRuleNames(final String schemaName, final CreateEncryptRuleStatement sqlStatement) {
+        Optional<EncryptRuleConfiguration> optional = getEncryptRuleConfiguration(schemaName);
+        if (optional.isPresent()) {
+            Collection<String> existRuleNames = getRuleNames(optional.get());
+            Collection<String> duplicateRuleNames = sqlStatement.getEncryptRules().stream()
+                    .map(EncryptRuleSegment::getTableName).filter(existRuleNames::contains).collect(Collectors.toList());
+            if (!duplicateRuleNames.isEmpty()) {
+                throw new DuplicateRuleNamesException(schemaName, duplicateRuleNames);
+            }
+        }
+    }
+
+    private void checkEncryptors(final CreateEncryptRuleStatement sqlStatement) {
         Collection<String> encryptors = new LinkedHashSet<>();
         sqlStatement.getEncryptRules().stream().forEach(each -> encryptors.addAll(each.getColumns().stream()
                 .map(column -> column.getEncryptor().getAlgorithmName()).collect(Collectors.toSet())));
@@ -79,7 +94,31 @@ public final class CreateEncryptRuleBackendHandler extends SchemaRequiredBackend
         }
     }
 
-    private void post(final String schemaName, final Collection<RuleConfiguration> rules) {
-        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, rules));
+    private void create(final String schemaName, final CreateEncryptRuleStatement sqlStatement) {
+        YamlEncryptRuleConfiguration yamlEncryptRuleConfiguration = EncryptRuleStatementConverter.convert(sqlStatement.getEncryptRules());
+        EncryptRuleConfiguration createdEncryptRuleConfiguration = new YamlRuleConfigurationSwapperEngine()
+                .swapToRuleConfigurations(Collections.singleton(yamlEncryptRuleConfiguration))
+                .stream().filter(each -> each instanceof EncryptRuleConfiguration).findAny().map(each -> (EncryptRuleConfiguration) each).get();
+        if (getEncryptRuleConfiguration(schemaName).isPresent()) {
+            EncryptRuleConfiguration existEncryptRuleConfiguration = getEncryptRuleConfiguration(schemaName).get();
+            existEncryptRuleConfiguration.getTables().addAll(createdEncryptRuleConfiguration.getTables());
+            existEncryptRuleConfiguration.getEncryptors().putAll(createdEncryptRuleConfiguration.getEncryptors());
+        } else {
+            ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations().add(createdEncryptRuleConfiguration);
+        }
+    }
+
+    private Optional<EncryptRuleConfiguration> getEncryptRuleConfiguration(final String schemaName) {
+        return ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations()
+                .stream().filter(each -> each instanceof EncryptRuleConfiguration).findAny().map(each -> (EncryptRuleConfiguration) each);
+    }
+
+    private Collection<String> getRuleNames(final EncryptRuleConfiguration encryptRuleConfiguration) {
+        return encryptRuleConfiguration.getTables().stream().map(EncryptTableRuleConfiguration::getName).collect(Collectors.toList());
+    }
+
+    private void post(final String schemaName) {
+        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/CreateEncryptRuleBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateEncryptRuleBackendHandlerTest.java
index 77e4085..3c890f4 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateEncryptRuleBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateEncryptRuleBackendHandlerTest.java
@@ -23,6 +23,7 @@ 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.create.impl.CreateEncryptRuleStatement;
 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;
@@ -30,7 +31,7 @@ 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.EncryptRuleExistsException;
+import org.apache.shardingsphere.proxy.backend.exception.DuplicateRuleNamesException;
 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;
@@ -90,9 +91,13 @@ public final class CreateEncryptRuleBackendHandlerTest {
         assertTrue(responseHeader instanceof UpdateResponseHeader);
     }
     
-    @Test(expected = EncryptRuleExistsException.class)
-    public void assertExecuteWithExistEncryptRule() {
-        when(ruleMetaData.getConfigurations()).thenReturn(Collections.singletonList(new EncryptRuleConfiguration(Collections.emptyList(), Maps.newHashMap())));
+    @Test(expected = DuplicateRuleNamesException.class)
+    public void assertExecuteWithDuplicateEncryptRule() {
+        EncryptTableRuleConfiguration encryptTableRuleConfiguration = new EncryptTableRuleConfiguration("t_encrypt", Collections.emptyList());
+        when(ruleMetaData.getConfigurations()).thenReturn(Collections.singletonList(new EncryptRuleConfiguration(Collections
+                .singleton(encryptTableRuleConfiguration), Maps.newHashMap())));
+        EncryptRuleSegment encryptRuleSegment = new EncryptRuleSegment("t_encrypt", buildColumns("MD5"));
+        when(sqlStatement.getEncryptRules()).thenReturn(Collections.singletonList(encryptRuleSegment));
         handler.execute("test", sqlStatement);
     }
 
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 c7140dd..13793b7 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
@@ -32,7 +32,6 @@ import org.apache.shardingsphere.proxy.backend.exception.DatabaseDiscoveryRulesN
 import org.apache.shardingsphere.proxy.backend.exception.DuplicateResourceException;
 import org.apache.shardingsphere.proxy.backend.exception.DuplicateRuleNamesException;
 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;
@@ -185,9 +184,6 @@ public final class MySQLErrPacketFactory {
             return new MySQLErrPacket(1, CommonErrorCode.DATABASE_DISCOVERY_RULES_NOT_EXIST, ((DatabaseDiscoveryRulesNotExistedException) cause).getRuleNames(),
                     ((DatabaseDiscoveryRulesNotExistedException) cause).getSchemaName());
         }
-        if (cause instanceof EncryptRuleExistsException) {
-            return new MySQLErrPacket(1, CommonErrorCode.ENCRYPT_RULE_EXIST, ((EncryptRuleExistsException) cause).getSchemaName());
-        }
         if (cause instanceof InvalidEncryptorsException) {
             return new MySQLErrPacket(1, CommonErrorCode.INVALID_ENCRYPTORS, ((InvalidEncryptorsException) cause).getEncryptors());
         }