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 2022/06/08 15:50:22 UTC

[shardingsphere] branch master updated: make assistEncryptorName work and effect (#18173)

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

zhangliang 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 ccd678fe988 make assistEncryptorName work and effect (#18173)
ccd678fe988 is described below

commit ccd678fe988f2844932e225de379cabc1458f39c
Author: cheese8 <yi...@163.com>
AuthorDate: Wed Jun 8 23:50:07 2022 +0800

    make assistEncryptorName work and effect (#18173)
    
    * add AssistEncryptorName
    
    * Update LocalEncryptConfiguration.java
    
    * add assistEncryptorName for EncryptColumn
    
    * make assistEncryptorName work and effect
    
    * Update RC4EncryptAlgorithm.java
    
    * align to previous
    
    * Update EncryptRuleTest.java
    
    * Update query-with-cipher.yaml
---
 ...OnDuplicateKeyUpdateValueParameterRewriter.java |  6 +++---
 .../EncryptInsertValueParameterRewriter.java       | 14 ++++++------
 .../EncryptInsertValuesTokenGenerator.java         |  9 ++++----
 .../shardingsphere/encrypt/rule/EncryptRule.java   | 25 ++++++++++++++++------
 .../shardingsphere/encrypt/rule/EncryptTable.java  | 10 +++++++++
 .../encrypt/rule/EncryptRuleTest.java              |  8 +++----
 .../converter/EncryptRuleStatementConverter.java   |  2 +-
 .../JDBCQueryAssistedEncryptAlgorithmFixture.java  |  2 +-
 .../config/config-encrypt-query-with-cipher.yaml   |  3 ++-
 .../config/config-encrypt-query-with-plain.yaml    |  3 ++-
 ...ewriteQueryAssistedEncryptAlgorithmFixture.java |  2 +-
 .../scenario/encrypt/config/query-with-cipher.yaml | 14 ++++++------
 .../scenario/encrypt/config/query-with-plain.yaml  | 21 ++++++++++++------
 .../scenario/mix/config/query-with-cipher.yaml     |  9 +++++---
 .../scenario/mix/config/query-with-plain.yaml      |  6 ++++--
 15 files changed, 85 insertions(+), 49 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertOnDuplicateKeyUpdateValueParameterRewriter.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertOnDuplicateKeyUpdateValueParameterRewriter.java
index 2fe6be47338..d84bd7089df 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertOnDuplicateKeyUpdateValueParameterRewriter.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertOnDuplicateKeyUpdateValueParameterRewriter.java
@@ -24,7 +24,6 @@ import org.apache.shardingsphere.encrypt.rewrite.aware.DatabaseNameAware;
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.encrypt.rule.aware.EncryptRuleAware;
 import org.apache.shardingsphere.encrypt.spi.EncryptAlgorithm;
-import org.apache.shardingsphere.encrypt.spi.QueryAssistedEncryptAlgorithm;
 import org.apache.shardingsphere.encrypt.spi.context.EncryptContext;
 import org.apache.shardingsphere.infra.binder.segment.insert.values.OnDuplicateUpdateContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
@@ -77,10 +76,11 @@ public final class EncryptInsertOnDuplicateKeyUpdateValueParameterRewriter imple
                 Object cipherColumnValue = encryptor.get().encrypt(plainColumnValue, encryptContext);
                 groupedParameterBuilder.getGenericParameterBuilder().addReplacedParameters(columnIndex, cipherColumnValue);
                 Collection<Object> addedParameters = new LinkedList<>();
-                if (optional instanceof QueryAssistedEncryptAlgorithm) {
+                Optional<EncryptAlgorithm> assistedQueryEncryptor = encryptRule.findAssistedQueryEncryptor(tableName, encryptLogicColumnName);
+                if (assistedQueryEncryptor.isPresent()) {
                     Optional<String> assistedColumnName = encryptRule.findAssistedQueryColumn(tableName, encryptLogicColumnName);
                     Preconditions.checkArgument(assistedColumnName.isPresent(), "Can not find assisted query Column Name");
-                    addedParameters.add(((QueryAssistedEncryptAlgorithm) optional).queryAssistedEncrypt(plainColumnValue, encryptContext));
+                    addedParameters.add(assistedQueryEncryptor.get().encrypt(plainColumnValue, encryptContext));
                 }
                 if (encryptRule.findPlainColumn(tableName, encryptLogicColumnName).isPresent()) {
                     addedParameters.add(plainColumnValue);
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertValueParameterRewriter.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertValueParameterRewriter.java
index 9266f7c0e23..8a778787f16 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertValueParameterRewriter.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertValueParameterRewriter.java
@@ -24,7 +24,6 @@ import org.apache.shardingsphere.encrypt.rewrite.aware.DatabaseNameAware;
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.encrypt.rule.aware.EncryptRuleAware;
 import org.apache.shardingsphere.encrypt.spi.EncryptAlgorithm;
-import org.apache.shardingsphere.encrypt.spi.QueryAssistedEncryptAlgorithm;
 import org.apache.shardingsphere.encrypt.spi.context.EncryptContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
@@ -69,12 +68,13 @@ public final class EncryptInsertValueParameterRewriter implements ParameterRewri
         while (descendingColumnNames.hasNext()) {
             String columnName = descendingColumnNames.next();
             EncryptContext encryptContext = EncryptContextBuilder.build(databaseName, schemaName, tableName, columnName);
-            encryptRule.findEncryptor(tableName, columnName).ifPresent(optional -> encryptInsertValues((GroupedParameterBuilder) parameterBuilder, insertStatementContext, optional, encryptContext));
+            encryptRule.findEncryptor(tableName, columnName).ifPresent(optional -> encryptInsertValues((GroupedParameterBuilder) parameterBuilder, insertStatementContext, optional,
+                    encryptRule.findAssistedQueryEncryptor(tableName, columnName), encryptContext));
         }
     }
     
     private void encryptInsertValues(final GroupedParameterBuilder parameterBuilder, final InsertStatementContext insertStatementContext,
-                                     final EncryptAlgorithm<?, ?> encryptAlgorithm, final EncryptContext encryptContext) {
+                                     final EncryptAlgorithm<?, ?> encryptAlgorithm, final Optional<EncryptAlgorithm> assistEncryptAlgorithm, final EncryptContext encryptContext) {
         int columnIndex = getColumnIndex(parameterBuilder, insertStatementContext, encryptContext.getColumnName());
         int count = 0;
         for (List<Object> each : insertStatementContext.getGroupedParameters()) {
@@ -84,7 +84,7 @@ public final class EncryptInsertValueParameterRewriter implements ParameterRewri
                 ExpressionSegment expressionSegment = insertStatementContext.getInsertValueContexts().get(count).getValueExpressions().get(columnIndex);
                 if (expressionSegment instanceof ParameterMarkerExpressionSegment) {
                     encryptInsertValue(
-                            encryptAlgorithm, parameterIndex, insertStatementContext.getInsertValueContexts().get(count).getValue(columnIndex)
+                            encryptAlgorithm, assistEncryptAlgorithm, parameterIndex, insertStatementContext.getInsertValueContexts().get(count).getValue(columnIndex)
                                     .orElseThrow(() -> new ShardingSphereException("Not support for encrypt!")),
                             standardParameterBuilder, encryptContext);
                 }
@@ -105,14 +105,14 @@ public final class EncryptInsertValueParameterRewriter implements ParameterRewri
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
-    private void encryptInsertValue(final EncryptAlgorithm encryptAlgorithm, final int parameterIndex,
+    private void encryptInsertValue(final EncryptAlgorithm encryptAlgorithm, final Optional<EncryptAlgorithm> assistEncryptor, final int parameterIndex,
                                     final Object originalValue, final StandardParameterBuilder parameterBuilder, final EncryptContext encryptContext) {
         parameterBuilder.addReplacedParameters(parameterIndex, encryptAlgorithm.encrypt(originalValue, encryptContext));
         Collection<Object> addedParameters = new LinkedList<>();
-        if (encryptAlgorithm instanceof QueryAssistedEncryptAlgorithm) {
+        if (assistEncryptor.isPresent()) {
             Optional<String> assistedColumnName = encryptRule.findAssistedQueryColumn(encryptContext.getTableName(), encryptContext.getColumnName());
             Preconditions.checkArgument(assistedColumnName.isPresent(), "Can not find assisted query Column Name");
-            addedParameters.add(((QueryAssistedEncryptAlgorithm) encryptAlgorithm).queryAssistedEncrypt(originalValue, encryptContext));
+            addedParameters.add(assistEncryptor.get().encrypt(originalValue, encryptContext));
         }
         if (encryptRule.findPlainColumn(encryptContext.getTableName(), encryptContext.getColumnName()).isPresent()) {
             addedParameters.add(originalValue);
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptInsertValuesTokenGenerator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptInsertValuesTokenGenerator.java
index d4b4102160e..24a6f37bdda 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptInsertValuesTokenGenerator.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptInsertValuesTokenGenerator.java
@@ -24,7 +24,6 @@ import org.apache.shardingsphere.encrypt.rewrite.token.pojo.EncryptInsertValuesT
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.encrypt.rule.aware.EncryptRuleAware;
 import org.apache.shardingsphere.encrypt.spi.EncryptAlgorithm;
-import org.apache.shardingsphere.encrypt.spi.QueryAssistedEncryptAlgorithm;
 import org.apache.shardingsphere.encrypt.spi.context.EncryptContext;
 import org.apache.shardingsphere.infra.binder.segment.insert.values.InsertValueContext;
 import org.apache.shardingsphere.infra.binder.segment.insert.values.expression.DerivedLiteralExpressionSegment;
@@ -138,7 +137,9 @@ public final class EncryptInsertValuesTokenGenerator implements OptionalSQLToken
                 Object originalValue = insertValueContext.getValue(columnIndex).orElseThrow(() -> new ShardingSphereException("Not support for encrypt!"));
                 EncryptContext encryptContext = EncryptContextBuilder.build(databaseName, schemaName, tableName, columnName);
                 addPlainColumn(insertValueToken, columnIndex, encryptContext, insertValueContext, originalValue);
-                addAssistedQueryColumn(insertValueToken, encryptor.get(), columnIndex, encryptContext, insertValueContext, originalValue);
+                if (encryptRule.findAssistedQueryEncryptor(tableName, columnName).isPresent()) {
+                    addAssistedQueryColumn(insertValueToken, encryptRule.findAssistedQueryEncryptor(tableName, columnName).get(), columnIndex, encryptContext, insertValueContext, originalValue);
+                }
                 setCipherColumn(insertValueToken, encryptor.get(), columnIndex, encryptContext, insertValueContext.getValueExpressions().get(columnIndex), originalValue);
             }
         }
@@ -155,11 +156,11 @@ public final class EncryptInsertValuesTokenGenerator implements OptionalSQLToken
     }
     
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private void addAssistedQueryColumn(final InsertValue insertValueToken, final EncryptAlgorithm<?, ?> encryptAlgorithm, final int columnIndex,
+    private void addAssistedQueryColumn(final InsertValue insertValueToken, final EncryptAlgorithm encryptAlgorithm, final int columnIndex,
                                         final EncryptContext encryptContext, final InsertValueContext insertValueContext, final Object originalValue) {
         if (encryptRule.findAssistedQueryColumn(encryptContext.getTableName(), encryptContext.getColumnName()).isPresent()) {
             DerivedSimpleExpressionSegment derivedExpressionSegment = isAddLiteralExpressionSegment(insertValueContext, columnIndex)
-                    ? new DerivedLiteralExpressionSegment(((QueryAssistedEncryptAlgorithm) encryptAlgorithm).queryAssistedEncrypt(originalValue, encryptContext))
+                    ? new DerivedLiteralExpressionSegment(encryptAlgorithm.encrypt(originalValue, encryptContext))
                     : new DerivedParameterMarkerExpressionSegment(getParameterIndexCount(insertValueToken));
             insertValueToken.getValues().add(columnIndex + 1, derivedExpressionSegment);
         }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
index c679561e255..c87972f35f0 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
@@ -27,7 +27,6 @@ import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfigu
 import org.apache.shardingsphere.encrypt.context.EncryptContextBuilder;
 import org.apache.shardingsphere.encrypt.factory.EncryptAlgorithmFactory;
 import org.apache.shardingsphere.encrypt.spi.EncryptAlgorithm;
-import org.apache.shardingsphere.encrypt.spi.QueryAssistedEncryptAlgorithm;
 import org.apache.shardingsphere.encrypt.spi.context.EncryptContext;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.rewrite.sql.token.generator.aware.SchemaMetaDataAware;
@@ -146,6 +145,19 @@ public final class EncryptRule implements SchemaRule, TableContainedRule {
         return tables.containsKey(lowerCaseLogicTable) ? tables.get(lowerCaseLogicTable).findEncryptorName(logicColumn).map(encryptors::get) : Optional.empty();
     }
     
+    /**
+     * Find assist encryptor.
+     *
+     * @param logicTable logic table name
+     * @param logicColumn logic column name
+     * @return encryptor
+     */
+    @SuppressWarnings("rawtypes")
+    public Optional<EncryptAlgorithm> findAssistedQueryEncryptor(final String logicTable, final String logicColumn) {
+        String lowerCaseLogicTable = logicTable.toLowerCase();
+        return tables.containsKey(lowerCaseLogicTable) ? tables.get(lowerCaseLogicTable).findAssistedQueryEncryptorName(logicColumn).map(encryptors::get) : Optional.empty();
+    }
+    
     /**
      * get encrypt values.
      *
@@ -230,18 +242,17 @@ public final class EncryptRule implements SchemaRule, TableContainedRule {
      */
     @SuppressWarnings("rawtypes")
     public List<Object> getEncryptAssistedQueryValues(final String databaseName, final String schemaName, final String logicTable, final String logicColumn, final List<Object> originalValues) {
-        Optional<EncryptAlgorithm> encryptor = findEncryptor(logicTable, logicColumn);
+        Optional<EncryptAlgorithm> encryptor = findAssistedQueryEncryptor(logicTable, logicColumn);
         EncryptContext encryptContext = EncryptContextBuilder.build(databaseName, schemaName, logicTable, logicColumn);
-        Preconditions.checkArgument(encryptor.isPresent() && encryptor.get() instanceof QueryAssistedEncryptAlgorithm,
-                "Can not find QueryAssistedEncryptAlgorithm by %s.%s.", logicTable, logicColumn);
-        return getEncryptAssistedQueryValues((QueryAssistedEncryptAlgorithm) encryptor.get(), originalValues, encryptContext);
+        Preconditions.checkArgument(encryptor.isPresent(), "Can not find assist encryptor by %s.%s.", logicTable, logicColumn);
+        return getEncryptAssistedQueryValues(encryptor.get(), originalValues, encryptContext);
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
-    private List<Object> getEncryptAssistedQueryValues(final QueryAssistedEncryptAlgorithm encryptor, final List<Object> originalValues, final EncryptContext encryptContext) {
+    private List<Object> getEncryptAssistedQueryValues(final EncryptAlgorithm encryptor, final List<Object> originalValues, final EncryptContext encryptContext) {
         List<Object> result = new LinkedList<>();
         for (Object each : originalValues) {
-            result.add(null == each ? null : encryptor.queryAssistedEncrypt(each, encryptContext));
+            result.add(null == each ? null : encryptor.encrypt(each, encryptContext));
         }
         return result;
     }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptTable.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptTable.java
index 97ec3f80f3a..07e7b352232 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptTable.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptTable.java
@@ -57,6 +57,16 @@ public final class EncryptTable {
         return columns.containsKey(logicColumn) ? Optional.of(columns.get(logicColumn).getEncryptorName()) : Optional.empty();
     }
     
+    /**
+     * Find assisted query encrypt algorithm name.
+     *
+     * @param logicColumn column name
+     * @return assist encrypt algorithm name
+     */
+    public Optional<String> findAssistedQueryEncryptorName(final String logicColumn) {
+        return columns.containsKey(logicColumn) ? Optional.ofNullable(columns.get(logicColumn).getAssistedQueryEncryptorName()) : Optional.empty();
+    }
+    
     /**
      * Get logic columns.
      *
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
index 9ff9ede8dd5..d478285ef3a 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
@@ -149,19 +149,19 @@ public final class EncryptRuleTest {
                 Collections.singleton(tableConfig), Collections.singletonMap("test_encryptor", new CoreEncryptAlgorithmFixture()), true);
         EncryptRule actual = new EncryptRule(ruleConfig);
         assertTrue(actual.isQueryWithCipherColumn("t_encrypt", "encrypt_column"));
-        
+    
         encryptColumnConfig = new EncryptColumnRuleConfiguration("encrypt_column", "encrypt_cipher", "", "", "test_encryptor", null);
         tableConfig = new EncryptTableRuleConfiguration("t_encrypt", Collections.singletonList(encryptColumnConfig), false);
         ruleConfig = new AlgorithmProvidedEncryptRuleConfiguration(Collections.singleton(tableConfig), Collections.singletonMap("test_encryptor", new CoreEncryptAlgorithmFixture()), true);
         actual = new EncryptRule(ruleConfig);
         assertFalse(actual.isQueryWithCipherColumn("t_encrypt", "encrypt_column"));
-        
+    
         encryptColumnConfig = new EncryptColumnRuleConfiguration("encrypt_column", "encrypt_cipher", "", "", "test_encryptor", true);
         tableConfig = new EncryptTableRuleConfiguration("t_encrypt", Collections.singletonList(encryptColumnConfig), false);
         ruleConfig = new AlgorithmProvidedEncryptRuleConfiguration(Collections.singleton(tableConfig), Collections.singletonMap("test_encryptor", new CoreEncryptAlgorithmFixture()), true);
         actual = new EncryptRule(ruleConfig);
         assertTrue(actual.isQueryWithCipherColumn("t_encrypt", "encrypt_column"));
-        
+    
         encryptColumnConfig = new EncryptColumnRuleConfiguration("encrypt_column", "encrypt_cipher", "", "", "test_encryptor", false);
         tableConfig = new EncryptTableRuleConfiguration("t_encrypt", Collections.singletonList(encryptColumnConfig), null);
         ruleConfig = new AlgorithmProvidedEncryptRuleConfiguration(Collections.singleton(tableConfig), Collections.singletonMap("test_encryptor", new CoreEncryptAlgorithmFixture()), true);
@@ -203,7 +203,7 @@ public final class EncryptRuleTest {
     private EncryptRuleConfiguration createEncryptRuleConfiguration() {
         ShardingSphereAlgorithmConfiguration queryAssistedEncryptConfig = new ShardingSphereAlgorithmConfiguration("CORE.QUERY_ASSISTED.FIXTURE", new Properties());
         ShardingSphereAlgorithmConfiguration metaDataAwareEncryptConfig = new ShardingSphereAlgorithmConfiguration("CORE.METADATA_AWARE.FIXTURE", new Properties());
-        EncryptColumnRuleConfiguration pwdColumnConfig = new EncryptColumnRuleConfiguration("pwd", "pwd_cipher", "", "pwd_plain", "test_encryptor", null);
+        EncryptColumnRuleConfiguration pwdColumnConfig = new EncryptColumnRuleConfiguration("pwd", "pwd_cipher", "", "pwd_plain", "test_encryptor", "test_encryptor", null);
         EncryptColumnRuleConfiguration creditCardColumnConfig = new EncryptColumnRuleConfiguration("credit_card", "credit_card_cipher", "", "credit_card_plain", "test_encryptor", null);
         EncryptColumnRuleConfiguration nameColumnConfig = new EncryptColumnRuleConfiguration("name", "name_cipher", "", "name_plain", "customized_encryptor", null);
         EncryptTableRuleConfiguration tableConfig = new EncryptTableRuleConfiguration("t_encrypt", Arrays.asList(pwdColumnConfig, creditCardColumnConfig, nameColumnConfig), null);
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/src/main/java/org/apache/shardingsphere/encrypt/distsql/handler/converter/EncryptRuleStatementConverter.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/src/main/java/org/apache/shardingsphere/encrypt/distsql/handler/converter/EncryptRuleStatementConverter.java
index 382d5c780fe..18402b74bf0 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/src/main/java/org/apache/shardingsphere/encrypt/distsql/handler/converter/EncryptRuleStatementConverter.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/src/main/java/org/apache/shardingsphere/encrypt/distsql/handler/converter/EncryptRuleStatementConverter.java
@@ -62,7 +62,7 @@ public final class EncryptRuleStatementConverter {
     // FIXME: support assistedEncryptorName on EncryptColumnSegment later
     private static EncryptColumnRuleConfiguration createEncryptColumnRuleConfiguration(final String tableName, final EncryptColumnSegment columnSegment) {
         return new EncryptColumnRuleConfiguration(columnSegment.getName(), columnSegment.getCipherColumn(), columnSegment.getAssistedQueryColumn(),
-                columnSegment.getPlainColumn(), getEncryptorName(tableName, columnSegment.getName()), null);
+                columnSegment.getPlainColumn(), getEncryptorName(tableName, columnSegment.getName()), "", null);
     }
     
     private static Map<String, ShardingSphereAlgorithmConfiguration> createEncryptorConfigurations(final EncryptRuleSegment ruleSegment) {
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/fixture/encrypt/JDBCQueryAssistedEncryptAlgorithmFixture.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/fixture/encrypt/JDBCQueryAssistedEncryptAlgorithmFixture.java
index a0621e3a4fb..3e98fac37d1 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/fixture/encrypt/JDBCQueryAssistedEncryptAlgorithmFixture.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/fixture/encrypt/JDBCQueryAssistedEncryptAlgorithmFixture.java
@@ -35,7 +35,7 @@ public final class JDBCQueryAssistedEncryptAlgorithmFixture implements QueryAssi
     
     @Override
     public String encrypt(final Object plainValue, final EncryptContext encryptContext) {
-        return "encryptValue";
+        return "assistedEncryptValue";
     }
     
     @Override
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/config-encrypt-query-with-cipher.yaml b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/config-encrypt-query-with-cipher.yaml
index 11d5aa064f6..ad3d5453c56 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/config-encrypt-query-with-cipher.yaml
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/config-encrypt-query-with-cipher.yaml
@@ -31,7 +31,8 @@ rules:
         pwd:
           cipherColumn: cipher_pwd
           assistedQueryColumn: assist_pwd
-          encryptorName: jdbc_query_assisted_encryptor_fixture
+          encryptorName: jdbc_encryptor_fixture
+          assistedQueryEncryptorName: jdbc_query_assisted_encryptor_fixture
     t_encrypt_contains_column:
       columns:
         plain_pwd:
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/config-encrypt-query-with-plain.yaml b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/config-encrypt-query-with-plain.yaml
index a607ee84932..2833d05ca09 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/config-encrypt-query-with-plain.yaml
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/config-encrypt-query-with-plain.yaml
@@ -31,7 +31,8 @@ rules:
         pwd:
           cipherColumn: cipher_pwd
           assistedQueryColumn: assist_pwd
-          encryptorName: jdbc_query_assisted_encryptor_fixture
+          encryptorName: jdbc_encryptor_fixture
+          assistedQueryEncryptorName: jdbc_query_assisted_encryptor_fixture
     t_encrypt_contains_column:
       columns:
         plain_pwd:
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/fixture/encrypt/RewriteQueryAssistedEncryptAlgorithmFixture.java b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/fixture/encrypt/RewriteQueryAssistedEncryptAlgorithmFixture.java
index 2b46e404b4e..011b5022709 100644
--- a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/fixture/encrypt/RewriteQueryAssistedEncryptAlgorithmFixture.java
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/fixture/encrypt/RewriteQueryAssistedEncryptAlgorithmFixture.java
@@ -35,7 +35,7 @@ public final class RewriteQueryAssistedEncryptAlgorithmFixture implements QueryA
     
     @Override
     public String encrypt(final Object plainValue, final EncryptContext encryptContext) {
-        return "encrypt_" + plainValue;
+        return "assisted_query_" + plainValue;
     }
     
     @Override
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/encrypt/config/query-with-cipher.yaml b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/encrypt/config/query-with-cipher.yaml
index 6d4ccb2985c..965bbf5f675 100644
--- a/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/encrypt/config/query-with-cipher.yaml
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/encrypt/config/query-with-cipher.yaml
@@ -27,12 +27,12 @@ rules:
         certificate_number:
           cipherColumn: cipher_certificate_number
           assistedQueryColumn: assisted_query_certificate_number
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
           assistedQueryEncryptorName: rewrite_assisted_query_fixture
         password:
           cipherColumn: cipher_password
           assistedQueryColumn: assisted_query_password
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
           assistedQueryEncryptorName: rewrite_assisted_query_fixture
         amount:
           cipherColumn: cipher_amount
@@ -46,19 +46,19 @@ rules:
           cipherColumn: cipher_certificate_number
           assistedQueryColumn: assisted_query_certificate_number
           plainColumn: plain_certificate_number
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
           assistedQueryEncryptorName: rewrite_assisted_query_fixture
         password:
           cipherColumn: cipher_password
           assistedQueryColumn: assisted_query_password
           plainColumn: plain_password
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
           assistedQueryEncryptorName: rewrite_assisted_query_fixture
         password_new:
           cipherColumn: password_new_cipher
           assistedQueryColumn: password_new_assisted
           plainColumn: password_new_plain
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
           assistedQueryEncryptorName: rewrite_assisted_query_fixture
         amount:
           cipherColumn: cipher_amount
@@ -70,13 +70,13 @@ rules:
           cipherColumn: cipher_certificate_number
           assistedQueryColumn: assisted_query_certificate_number
           plainColumn: plain_certificate_number
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
           assistedQueryEncryptorName: rewrite_assisted_query_fixture
         password:
           cipherColumn: cipher_password
           assistedQueryColumn: assisted_query_password
           plainColumn: plain_password
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
           assistedQueryEncryptorName: rewrite_assisted_query_fixture
         amount:
           cipherColumn: cipher_amount
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/encrypt/config/query-with-plain.yaml b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/encrypt/config/query-with-plain.yaml
index 0f42d0dc845..8275809f667 100644
--- a/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/encrypt/config/query-with-plain.yaml
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/encrypt/config/query-with-plain.yaml
@@ -27,11 +27,13 @@ rules:
         certificate_number:
           cipherColumn: cipher_certificate_number
           assistedQueryColumn: assisted_query_certificate_number
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
+          assistedQueryEncryptorName: rewrite_assisted_query_fixture
         password:
           cipherColumn: cipher_password
           assistedQueryColumn: assisted_query_password
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
+          assistedQueryEncryptorName: rewrite_assisted_query_fixture
         amount:
           cipherColumn: cipher_amount
           encryptorName: rewrite_normal_fixture
@@ -44,17 +46,20 @@ rules:
           cipherColumn: cipher_certificate_number
           assistedQueryColumn: assisted_query_certificate_number
           plainColumn: plain_certificate_number
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
+          assistedQueryEncryptorName: rewrite_assisted_query_fixture
         password:
           cipherColumn: cipher_password
           assistedQueryColumn: assisted_query_password
           plainColumn: plain_password
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
+          assistedQueryEncryptorName: rewrite_assisted_query_fixture
         password_new:
           cipherColumn: password_new_cipher
           assistedQueryColumn: password_new_assisted
           plainColumn: password_new_plain
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
+          assistedQueryEncryptorName: rewrite_assisted_query_fixture
         amount:
           cipherColumn: cipher_amount
           plainColumn: plain_amount
@@ -65,12 +70,14 @@ rules:
           cipherColumn: cipher_certificate_number
           assistedQueryColumn: assisted_query_certificate_number
           plainColumn: plain_certificate_number
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
+          assistedQueryEncryptorName: rewrite_assisted_query_fixture
         password:
           cipherColumn: cipher_password
           assistedQueryColumn: assisted_query_password
           plainColumn: plain_password
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
+          assistedQueryEncryptorName: rewrite_assisted_query_fixture
         amount:
           cipherColumn: cipher_amount
           plainColumn: plain_amount
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/mix/config/query-with-cipher.yaml b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/mix/config/query-with-cipher.yaml
index 01d23d3f29d..9492fedb24f 100644
--- a/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/mix/config/query-with-cipher.yaml
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/mix/config/query-with-cipher.yaml
@@ -74,7 +74,8 @@ rules:
         password:
           cipherColumn: cipher_password
           assistedQueryColumn: assisted_query_password
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
+          assistedQueryEncryptorName: rewrite_assisted_query_fixture
         amount:
           cipherColumn: cipher_amount
           encryptorName: rewrite_normal_fixture
@@ -84,7 +85,8 @@ rules:
           cipherColumn: cipher_password
           assistedQueryColumn: assisted_query_password
           plainColumn: plain_password
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
+          assistedQueryEncryptorName: rewrite_assisted_query_fixture
         amount:
           cipherColumn: cipher_amount
           plainColumn: plain_amount
@@ -95,7 +97,8 @@ rules:
           cipherColumn: cipher_password
           assistedQueryColumn: assisted_query_password
           plainColumn: plain_password
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
+          assistedQueryEncryptorName: rewrite_assisted_query_fixture
         amount:
           cipherColumn: cipher_amount
           plainColumn: plain_amount
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/mix/config/query-with-plain.yaml b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/mix/config/query-with-plain.yaml
index 7424ba08cf7..63c9b4b2f2b 100644
--- a/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/mix/config/query-with-plain.yaml
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/mix/config/query-with-plain.yaml
@@ -74,7 +74,8 @@ rules:
         password:
           cipherColumn: cipher_password
           assistedQueryColumn: assisted_query_password
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
+          assistedQueryEncryptorName: rewrite_assisted_query_fixture
         amount:
           cipherColumn: cipher_amount
           encryptorName: rewrite_normal_fixture
@@ -84,7 +85,8 @@ rules:
           cipherColumn: cipher_password
           assistedQueryColumn: assisted_query_password
           plainColumn: plain_password
-          encryptorName: rewrite_assisted_query_fixture
+          encryptorName: rewrite_normal_fixture
+          assistedQueryEncryptorName: rewrite_assisted_query_fixture
         amount:
           cipherColumn: cipher_amount
           plainColumn: plain_amount