You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2020/11/12 08:39:07 UTC

[shardingsphere] branch master updated: Simplify encrypt rule (#8146)

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

zhangyonglun 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 c50e3da  Simplify encrypt rule (#8146)
c50e3da is described below

commit c50e3da55c676b11edbef07dbd2b81b442422df1
Author: Liang Zhang <te...@163.com>
AuthorDate: Thu Nov 12 16:38:45 2020 +0800

    Simplify encrypt rule (#8146)
    
    * Refactor EncryptTableMetaDataBuilder
    
    * Fix test case
    
    * Remove useless EncryptRule.getEncryptTableNames
    
    * Remove useless EncryptRule.isCipherColumn
    
    * Remove useless EncryptRule.getLogicColumnOfCipher
    
    * Remove useless EncryptRule.getAssistedQueryAndPlainColumns
    
    * For code style
---
 .../metadata/EncryptTableMetaDataBuilder.java      | 29 ++++++------
 .../shardingsphere/encrypt/rule/EncryptRule.java   | 52 +---------------------
 .../metadata/EncryptTableMetaDataBuilderTest.java  | 10 +++--
 .../encrypt/rule/EncryptRuleTest.java              | 22 +--------
 .../impl/EncryptAssignmentTokenGenerator.java      |  5 +--
 .../schema/builder/TableMetaDataBuilder.java       |  6 +--
 .../spring/boot/SpringBootStarterTest.java         |  9 ++--
 7 files changed, 35 insertions(+), 98 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptTableMetaDataBuilder.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptTableMetaDataBuilder.java
index 0fa55d3..9130bf7 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptTableMetaDataBuilder.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptTableMetaDataBuilder.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.encrypt.metadata;
 
 import org.apache.shardingsphere.encrypt.constant.EncryptOrder;
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
+import org.apache.shardingsphere.encrypt.rule.EncryptTable;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
@@ -47,28 +48,30 @@ public final class EncryptTableMetaDataBuilder implements RuleBasedTableMetaData
     
     @Override
     public TableMetaData decorate(final String tableName, final TableMetaData tableMetaData, final EncryptRule encryptRule) {
-        return new TableMetaData(getEncryptColumnMetaDataList(tableName, tableMetaData.getColumns().values(), encryptRule), tableMetaData.getIndexes().values());
+        Optional<EncryptTable> encryptTable = encryptRule.findEncryptTable(tableName);
+        return encryptTable.map(optional -> new TableMetaData(getEncryptColumnMetaDataList(optional, tableMetaData.getColumns().values()), tableMetaData.getIndexes().values())).orElse(tableMetaData);
     }
     
-    private Collection<ColumnMetaData> getEncryptColumnMetaDataList(final String tableName,
-                                                                    final Collection<ColumnMetaData> originalColumnMetaDataList, final EncryptRule encryptRule) {
+    private Collection<ColumnMetaData> getEncryptColumnMetaDataList(final EncryptTable encryptTable, final Collection<ColumnMetaData> originalColumnMetaDataList) {
         Collection<ColumnMetaData> result = new LinkedList<>();
-        Collection<String> derivedColumns = encryptRule.getAssistedQueryAndPlainColumns(tableName);
+        Collection<String> plainColumns = encryptTable.getPlainColumns();
+        Collection<String> assistedQueryColumns = encryptTable.getAssistedQueryColumns();
         for (ColumnMetaData each : originalColumnMetaDataList) {
-            if (!derivedColumns.contains(each.getName())) {
-                result.add(getEncryptColumnMetaData(tableName, each, encryptRule));
+            String columnName = each.getName();
+            if (encryptTable.isCipherColumn(columnName)) {
+                result.add(createColumnMetaData(encryptTable.getLogicColumn(columnName), each));
+                continue;
+            }
+            if (!plainColumns.contains(columnName) && !assistedQueryColumns.contains(columnName)) {
+                result.add(each);
             }
         }
         return result;
     }
     
-    private ColumnMetaData getEncryptColumnMetaData(final String tableName, final ColumnMetaData originalColumnMetaData, final EncryptRule encryptRule) {
-        if (!encryptRule.isCipherColumn(tableName, originalColumnMetaData.getName())) {
-            return originalColumnMetaData;
-        }
-        String logicColumnName = encryptRule.getLogicColumnOfCipher(tableName, originalColumnMetaData.getName());
-        return new ColumnMetaData(logicColumnName, originalColumnMetaData.getDataType(), 
-                originalColumnMetaData.getDataTypeName(), originalColumnMetaData.isPrimaryKey(), originalColumnMetaData.isGenerated(), originalColumnMetaData.isCaseSensitive());
+    private ColumnMetaData createColumnMetaData(final String columnName, final ColumnMetaData columnMetaData) {
+        return new ColumnMetaData(columnName, 
+                columnMetaData.getDataType(), columnMetaData.getDataTypeName(), columnMetaData.isPrimaryKey(), columnMetaData.isGenerated(), columnMetaData.isCaseSensitive());
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
index 600c1fe..bab30ab 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
@@ -32,7 +32,6 @@ import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedHashMap;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -107,15 +106,6 @@ public final class EncryptRule implements TableContainedRule {
     }
     
     /**
-     * Get encrypt table names.
-     *
-     * @return encrypt table names
-     */
-    public Collection<String> getEncryptTableNames() {
-        return tables.keySet();
-    }
-    
-    /**
      * Find encrypt table.
      * 
      * @param logicTable logic table
@@ -151,17 +141,6 @@ public final class EncryptRule implements TableContainedRule {
     }
     
     /**
-     * Is cipher column or not.
-     *
-     * @param tableName table name
-     * @param columnName column name
-     * @return cipher column or not
-     */
-    public boolean isCipherColumn(final String tableName, final String columnName) {
-        return tables.containsKey(tableName) && tables.get(tableName).isCipherColumn(columnName);
-    }
-    
-    /**
      * Get cipher column.
      *
      * @param logicTable logic table name
@@ -173,17 +152,6 @@ public final class EncryptRule implements TableContainedRule {
     }
     
     /**
-     * Get logic column of cipher column.
-     *
-     * @param logicTable logic table
-     * @param cipherColumn cipher column
-     * @return logic column
-     */
-    public String getLogicColumnOfCipher(final String logicTable, final String cipherColumn) {
-        return tables.get(logicTable).getLogicColumn(cipherColumn);
-    }
-    
-    /**
      * Get logic and cipher columns.
      *
      * @param logicTable logic table 
@@ -226,25 +194,7 @@ public final class EncryptRule implements TableContainedRule {
         Optional<EncryptAlgorithm> encryptor = findEncryptor(logicTable, logicColumn);
         Preconditions.checkArgument(encryptor.isPresent() && encryptor.get() instanceof QueryAssistedEncryptAlgorithm,
                 String.format("Can not find QueryAssistedEncryptAlgorithm by %s.%s.", logicTable, logicColumn));
-        return originalValues.stream().map(input -> null == input
-                ? null : ((QueryAssistedEncryptAlgorithm) encryptor.get()).queryAssistedEncrypt(input.toString())).collect(Collectors.toList());
-    }
-    
-    /**
-     * Get assisted query and plain columns.
-     *
-     * @param logicTable logic table name
-     * @return assisted query and plain columns
-     */
-    public Collection<String> getAssistedQueryAndPlainColumns(final String logicTable) {
-        Collection<String> result = new LinkedList<>();
-        result.addAll(getAssistedQueryColumns(logicTable));
-        result.addAll(getPlainColumns(logicTable));
-        return result;
-    }
-    
-    private Collection<String> getPlainColumns(final String logicTable) {
-        return tables.containsKey(logicTable) ? tables.get(logicTable).getPlainColumns() : Collections.emptyList();
+        return originalValues.stream().map(input -> null == input ? null : ((QueryAssistedEncryptAlgorithm) encryptor.get()).queryAssistedEncrypt(input.toString())).collect(Collectors.toList());
     }
     
     /**
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptTableMetaDataBuilderTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptTableMetaDataBuilderTest.java
index 13ca393..d26e88a 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptTableMetaDataBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptTableMetaDataBuilderTest.java
@@ -126,10 +126,12 @@ public final class EncryptTableMetaDataBuilderTest {
     
     private EncryptRule createEncryptRule() {
         EncryptRule result = mock(EncryptRule.class);
-        when(result.findEncryptTable(TABLE_NAME)).thenReturn(Optional.of(mock(EncryptTable.class)));
-        when(result.getLogicColumnOfCipher("t_encrypt", "pwd_cipher")).thenReturn("pwd");
-        when(result.isCipherColumn("t_encrypt", "pwd_cipher")).thenReturn(true);
-        when(result.getAssistedQueryAndPlainColumns("t_encrypt")).thenReturn(Collections.singletonList("pwd_plain"));
+        EncryptTable encryptTable = mock(EncryptTable.class);
+        when(result.findEncryptTable(TABLE_NAME)).thenReturn(Optional.of(encryptTable));
+        when(encryptTable.getAssistedQueryColumns()).thenReturn(Collections.emptyList());
+        when(encryptTable.getPlainColumns()).thenReturn(Collections.singleton("pwd_plain"));
+        when(encryptTable.isCipherColumn("pwd_cipher")).thenReturn(true);
+        when(encryptTable.getLogicColumn("pwd_cipher")).thenReturn("pwd");
         return result;
     }
     
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
index c3ce5b8..e16d334 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
@@ -46,7 +46,7 @@ public final class EncryptRuleTest {
         AlgorithmProvidedEncryptRuleConfiguration ruleConfig = new AlgorithmProvidedEncryptRuleConfiguration(
                 Collections.singleton(tableConfig), ImmutableMap.of("test_encryptor", new TestEncryptAlgorithm()));
         EncryptRule actual = new EncryptRule(ruleConfig);
-        assertThat(actual.getEncryptTableNames(), is(Collections.singleton("t_encrypt")));
+        assertTrue(actual.findEncryptTable("t_encrypt").isPresent());
     }
     
     @Test(expected = IllegalArgumentException.class)
@@ -68,11 +68,6 @@ public final class EncryptRuleTest {
     }
     
     @Test
-    public void assertGetEncryptTableNames() {
-        assertFalse(new EncryptRule(createEncryptRuleConfiguration()).getEncryptTableNames().isEmpty());
-    }
-    
-    @Test
     public void assertFindEncryptTable() {
         assertTrue(new EncryptRule(createEncryptRuleConfiguration()).findEncryptTable("t_encrypt").isPresent());
     }
@@ -96,11 +91,6 @@ public final class EncryptRuleTest {
     }
     
     @Test
-    public void assertIsCipherColumn() {
-        assertTrue(new EncryptRule(createEncryptRuleConfiguration()).isCipherColumn("t_encrypt", "pwd_cipher"));
-    }
-    
-    @Test
     public void assertGetCipherColumnWhenEncryptColumnExist() {
         assertThat(new EncryptRule(createEncryptRuleConfiguration()).getCipherColumn("t_encrypt", "pwd"), is("pwd_cipher"));
     }
@@ -111,11 +101,6 @@ public final class EncryptRuleTest {
     }
     
     @Test
-    public void assertGetLogicColumnOfCipher() {
-        assertThat(new EncryptRule(createEncryptRuleConfiguration()).getLogicColumnOfCipher("t_encrypt", "pwd_cipher"), is("pwd"));
-    }
-    
-    @Test
     public void assertGetLogicAndCipherColumns() {
         assertFalse(new EncryptRule(createEncryptRuleConfiguration()).getLogicAndCipherColumns("t_encrypt").isEmpty());
     }
@@ -139,11 +124,6 @@ public final class EncryptRuleTest {
     }
     
     @Test
-    public void assertGetAssistedQueryAndPlainColumns() {
-        assertFalse(new EncryptRule(createEncryptRuleConfiguration()).getAssistedQueryAndPlainColumns("t_encrypt").isEmpty());
-    }
-    
-    @Test
     public void assertFindPlainColumn() {
         assertTrue(new EncryptRule(createEncryptRuleConfiguration()).findPlainColumn("t_encrypt", "pwd").isPresent());
         assertTrue(new EncryptRule(createEncryptRuleConfiguration()).findPlainColumn("t_encrypt", "credit_card".toLowerCase()).isPresent());
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptAssignmentTokenGenerator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptAssignmentTokenGenerator.java
index 3b6914a..55931b6 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptAssignmentTokenGenerator.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptAssignmentTokenGenerator.java
@@ -22,11 +22,11 @@ import org.apache.shardingsphere.encrypt.rewrite.token.generator.BaseEncryptSQLT
 import org.apache.shardingsphere.encrypt.rewrite.token.pojo.EncryptAssignmentToken;
 import org.apache.shardingsphere.encrypt.rewrite.token.pojo.EncryptLiteralAssignmentToken;
 import org.apache.shardingsphere.encrypt.rewrite.token.pojo.EncryptParameterAssignmentToken;
-import org.apache.shardingsphere.infra.rewrite.sql.token.generator.CollectionSQLTokenGenerator;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementContext;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+import org.apache.shardingsphere.infra.rewrite.sql.token.generator.CollectionSQLTokenGenerator;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.SetAssignmentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
@@ -48,8 +48,7 @@ public final class EncryptAssignmentTokenGenerator extends BaseEncryptSQLTokenGe
     
     @Override
     protected boolean isGenerateSQLTokenForEncrypt(final SQLStatementContext sqlStatementContext) {
-        return sqlStatementContext instanceof UpdateStatementContext
-                || (sqlStatementContext instanceof InsertStatementContext 
+        return sqlStatementContext instanceof UpdateStatementContext || (sqlStatementContext instanceof InsertStatementContext 
                 && InsertStatementHandler.getSetAssignmentSegment(((InsertStatementContext) sqlStatementContext).getSqlStatement()).isPresent());
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableMetaDataBuilder.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableMetaDataBuilder.java
index ee7c4ab..4720b41 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableMetaDataBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableMetaDataBuilder.java
@@ -62,9 +62,9 @@ public final class TableMetaDataBuilder {
             if (entry.getKey() instanceof TableContainedRule) {
                 TableContainedRule rule = (TableContainedRule) entry.getKey();
                 RuleBasedTableMetaDataBuilder loader = entry.getValue();
-                Optional<TableMetaData> tableMetaData = loader.load(tableName, materials.getDatabaseType(), materials.getDataSourceMap(), dataNodes, rule, materials.getProps());
-                if (tableMetaData.isPresent()) {
-                    return tableMetaData;
+                Optional<TableMetaData> result = loader.load(tableName, materials.getDatabaseType(), materials.getDataSourceMap(), dataNodes, rule, materials.getProps());
+                if (result.isPresent()) {
+                    return result;
                 }
             }
         }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/SpringBootStarterTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/SpringBootStarterTest.java
index ea67759..86b2355 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/SpringBootStarterTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/SpringBootStarterTest.java
@@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
 import org.apache.shardingsphere.driver.jdbc.core.datasource.ShardingSphereDataSource;
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
+import org.apache.shardingsphere.encrypt.rule.EncryptTable;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.datanode.DataNodeUtil;
@@ -118,13 +119,15 @@ public class SpringBootStarterTest {
     }
     
     private void assertEncryptRule(final EncryptRule rule) {
-        assertThat(rule.getEncryptTableNames(), is(Sets.newLinkedHashSet(Collections.singletonList("t_order"))));
+        assertTrue(rule.findEncryptTable("t_order").isPresent());
+        EncryptTable table = rule.findEncryptTable("t_order").get();
+        assertThat(table.getLogicColumn("pwd_cipher"), is("pwd"));
+        assertThat(table.getPlainColumns(), is(Collections.singletonList("pwd_plain")));
+        assertThat(table.getAssistedQueryColumns(), is(Collections.singletonList("pwd_assisted_query_cipher")));
         assertThat(rule.getCipherColumn("t_order", "pwd"), is("pwd_cipher"));
         assertThat(rule.getAssistedQueryColumns("t_order"), is(Collections.singletonList("pwd_assisted_query_cipher")));
         assertThat(rule.getLogicAndCipherColumns("t_order"), is(Collections.singletonMap("pwd", "pwd_cipher")));
-        assertThat(rule.getLogicColumnOfCipher("t_order", "pwd_cipher"), is("pwd"));
         assertThat(rule.getEncryptValues("t_order", "pwd", Collections.singletonList("pwd_plain")), is(Collections.singletonList("V/RkV1+dVv80Y3csT3cR4g==")));
-        assertThat(rule.getAssistedQueryAndPlainColumns("t_order"), is(Arrays.asList("pwd_assisted_query_cipher", "pwd_plain")));
     }
     
     private void assertShadowRule(final ShadowRule rule) {