You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ji...@apache.org on 2022/11/30 13:07:33 UTC

[shardingsphere] branch master updated: Optimize algorithmDefinition naming in encryptColumnDefinition (#22494)

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

jianglongtao 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 dc57a595e0d Optimize algorithmDefinition naming in encryptColumnDefinition (#22494)
dc57a595e0d is described below

commit dc57a595e0d8d14c8dd5d646a9886b4427cf4adf
Author: gxxiong <xi...@foxmail.com>
AuthorDate: Wed Nov 30 21:07:23 2022 +0800

    Optimize algorithmDefinition naming in encryptColumnDefinition (#22494)
    
    * Optimize algorithmDefinition naming in encryptColumnDefinition
    
    Signed-off-by: gxxiong <xi...@foxmail.com>
    
    * fix it-Scaling error
    
    Signed-off-by: gxxiong <xi...@foxmail.com>
    
    * fix it error
    
    Signed-off-by: gxxiong <xi...@foxmail.com>
    
    * Rename definition on Encrypt
    
    Signed-off-by: gxxiong <xi...@foxmail.com>
    
    * fix ci error
    
    Signed-off-by: gxxiong <xi...@foxmail.com>
    
    Signed-off-by: gxxiong <xi...@foxmail.com>
---
 .../src/main/antlr4/imports/encrypt/Keyword.g4     | 28 ++++++++++++++------
 .../main/antlr4/imports/encrypt/RDLStatement.g4    | 18 ++++++++++---
 .../core/EncryptDistSQLStatementVisitor.java       | 15 +++--------
 .../encrypt/distsql/parser/EncryptDistSqlTest.java | 14 +++++++---
 .../common/constant/DistSQLScriptConstants.java    | 10 ++++++++
 .../queryable/ConvertYamlConfigurationHandler.java | 30 +++++++++++++++++++---
 .../resources/conf/convert/config-encrypt.yaml     |  6 +++++
 .../test/resources/conf/convert/config-mix.yaml    |  6 +++++
 .../test/resources/expected/convert-encrypt.yaml   |  4 +--
 .../src/test/resources/expected/convert-mix.yaml   |  4 +--
 .../resources/env/common/migration-command.xml     |  2 +-
 .../cases/rdl/rdl-integration-test-cases.xml       |  8 +++---
 .../src/main/resources/sql/supported/rdl/alter.xml |  8 +++---
 .../main/resources/sql/supported/rdl/create.xml    | 15 ++++-------
 14 files changed, 115 insertions(+), 53 deletions(-)

diff --git a/features/encrypt/distsql/parser/src/main/antlr4/imports/encrypt/Keyword.g4 b/features/encrypt/distsql/parser/src/main/antlr4/imports/encrypt/Keyword.g4
index ab24811f6ce..aba744a3034 100644
--- a/features/encrypt/distsql/parser/src/main/antlr4/imports/encrypt/Keyword.g4
+++ b/features/encrypt/distsql/parser/src/main/antlr4/imports/encrypt/Keyword.g4
@@ -58,7 +58,19 @@ ENCRYPT
 TYPE
     : T Y P E
     ;
-    
+
+ENCRYPT_ALGORITHM
+    : E N C R Y P T UL_ A L G O R I T H M
+    ;
+
+ASSISTED_QUERY_ALGORITHM
+    : A S S I S T E D UL_ Q U E R Y UL_ A L G O R I T H M
+    ;
+
+LIKE_QUERY_ALGORITHM
+    : L I K E UL_ Q U E R Y UL_ A L G O R I T H M
+    ;
+
 NAME
     : N A M E
     ;
@@ -90,7 +102,7 @@ CIPHER
 PLAIN
     : P L A I N
     ;
-    
+
 ASSISTED_QUERY_COLUMN
     : A S S I S T E D UL_ Q U E R Y UL_ C O L U M N
     ;
@@ -102,19 +114,19 @@ LIKE_QUERY_COLUMN
 QUERY_WITH_CIPHER_COLUMN
     : Q U E R Y UL_ W I T H UL_ C I P H E R UL_ C O L U M N
     ;
-    
+
 TRUE
     : T R U E
     ;
-    
+
 FALSE
     : F A L S E
-    ;    
+    ;
 
 DATA_TYPE
     : D A T A UL_ T Y P E
     ;
-    
+
 PLAIN_DATA_TYPE
     : P L A I N UL_ D A T A UL_ T Y P E
     ;
@@ -131,10 +143,10 @@ LIKE_QUERY_DATA_TYPE
     : L I K E UL_ Q U E R Y UL_ D A T A UL_ T Y P E
     ;
 
-IF  
+IF
     : I F
     ;
-    
+
 EXISTS
     : E X I S T S
     ;
diff --git a/features/encrypt/distsql/parser/src/main/antlr4/imports/encrypt/RDLStatement.g4 b/features/encrypt/distsql/parser/src/main/antlr4/imports/encrypt/RDLStatement.g4
index 459fe8fdf91..c40f0e970c9 100644
--- a/features/encrypt/distsql/parser/src/main/antlr4/imports/encrypt/RDLStatement.g4
+++ b/features/encrypt/distsql/parser/src/main/antlr4/imports/encrypt/RDLStatement.g4
@@ -44,7 +44,7 @@ resourceName
     ;
 
 encryptColumnDefinition
-    : LP columnDefinition (COMMA plainColumnDefinition)? COMMA cipherColumnDefinition (COMMA assistedQueryColumnDefinition)? (COMMA likeQueryColumnDefinition)? COMMA algorithmDefinition (COMMA algorithmDefinition)? (COMMA algorithmDefinition)? RP
+    : LP columnDefinition (COMMA plainColumnDefinition)? COMMA cipherColumnDefinition (COMMA assistedQueryColumnDefinition)? (COMMA likeQueryColumnDefinition)? COMMA encryptAlgorithm (COMMA assistedQueryAlgorithm)? (COMMA likeQueryAlgorithm)? RP
     ;
 
 columnDefinition
@@ -56,7 +56,7 @@ columnName
     ;
 
 dataType
-    : STRING 
+    : STRING
     ;
 
 plainColumnDefinition
@@ -90,7 +90,19 @@ likeQueryColumnDefinition
 likeQueryColumnName
     : IDENTIFIER
     ;
-    
+
+encryptAlgorithm
+    : ENCRYPT_ALGORITHM LP algorithmDefinition RP
+    ;
+
+assistedQueryAlgorithm
+    : ASSISTED_QUERY_ALGORITHM LP algorithmDefinition RP
+    ;
+
+likeQueryAlgorithm
+    : LIKE_QUERY_ALGORITHM LP algorithmDefinition RP
+    ;
+
 queryWithCipherColumn
     : TRUE | FALSE
     ;
diff --git a/features/encrypt/distsql/parser/src/main/java/org/apache/shardingsphere/encrypt/distsql/parser/core/EncryptDistSQLStatementVisitor.java b/features/encrypt/distsql/parser/src/main/java/org/apache/shardingsphere/encrypt/distsql/parser/core/EncryptDistSQLStatementVisitor.java
index 0d301a8d081..ef610eabb58 100644
--- a/features/encrypt/distsql/parser/src/main/java/org/apache/shardingsphere/encrypt/distsql/parser/core/EncryptDistSQLStatementVisitor.java
+++ b/features/encrypt/distsql/parser/src/main/java/org/apache/shardingsphere/encrypt/distsql/parser/core/EncryptDistSQLStatementVisitor.java
@@ -45,10 +45,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.DatabaseS
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
 
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Objects;
-import java.util.Optional;
 import java.util.Properties;
 import java.util.stream.Collectors;
 
@@ -88,10 +85,6 @@ public final class EncryptDistSQLStatementVisitor extends EncryptDistSQLStatemen
     
     @Override
     public ASTNode visitEncryptColumnDefinition(final EncryptColumnDefinitionContext ctx) {
-        List<AlgorithmSegment> algorithmSegments = new ArrayList<>();
-        for (AlgorithmDefinitionContext each : ctx.algorithmDefinition()) {
-            algorithmSegments.add((AlgorithmSegment) visit(each));
-        }
         return new EncryptColumnSegment(getIdentifierValue(ctx.columnDefinition().columnName()),
                 getIdentifierValue(ctx.cipherColumnDefinition().cipherColumnName()),
                 null == ctx.plainColumnDefinition() ? null : getIdentifierValue(ctx.plainColumnDefinition().plainColumnName()),
@@ -102,11 +95,9 @@ public final class EncryptDistSQLStatementVisitor extends EncryptDistSQLStatemen
                 null == ctx.plainColumnDefinition() ? null : getIdentifierValue(ctx.plainColumnDefinition().dataType()),
                 null == ctx.assistedQueryColumnDefinition() ? null : getIdentifierValue(ctx.assistedQueryColumnDefinition().dataType()),
                 null == ctx.likeQueryColumnDefinition() ? null : getIdentifierValue(ctx.likeQueryColumnDefinition().dataType()),
-                algorithmSegments.get(0),
-                null == ctx.assistedQueryColumnDefinition() || 1 == algorithmSegments.size() ? null : algorithmSegments.get(1),
-                null == ctx.likeQueryColumnDefinition() ? null
-                        : Optional.ofNullable(algorithmSegments).filter(algorithm -> algorithm.size() > 2).map(algorithm -> algorithm.get(2)).orElse(null));
-        
+                null == ctx.encryptAlgorithm() ? null : (AlgorithmSegment) visit(ctx.encryptAlgorithm().algorithmDefinition()),
+                null == ctx.assistedQueryAlgorithm() ? null : (AlgorithmSegment) visit(ctx.assistedQueryAlgorithm().algorithmDefinition()),
+                null == ctx.likeQueryAlgorithm() ? null : (AlgorithmSegment) visit(ctx.likeQueryAlgorithm().algorithmDefinition()));
     }
     
     @Override
diff --git a/features/encrypt/distsql/parser/src/test/java/org/apache/shardingsphere/encrypt/distsql/parser/EncryptDistSqlTest.java b/features/encrypt/distsql/parser/src/test/java/org/apache/shardingsphere/encrypt/distsql/parser/EncryptDistSqlTest.java
index 58af8e253ae..121a1434c0b 100644
--- a/features/encrypt/distsql/parser/src/test/java/org/apache/shardingsphere/encrypt/distsql/parser/EncryptDistSqlTest.java
+++ b/features/encrypt/distsql/parser/src/test/java/org/apache/shardingsphere/encrypt/distsql/parser/EncryptDistSqlTest.java
@@ -43,8 +43,9 @@ public final class EncryptDistSqlTest {
     @Test
     public void assertCreateEncryptRule() {
         String sql = "CREATE ENCRYPT RULE t_encrypt (COLUMNS("
-                + " (NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')))"
-                + ",(NAME=order_id, CIPHER =order_cipher,TYPE(NAME='MD5')))"
+                + " (NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,ASSISTED_QUERY_COLUMN=user_assisted,LIKE_QUERY_COLUMN=user_like"
+                + ",ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))),ASSISTED_QUERY_ALGORITHM(TYPE(NAME='MD5')),LIKE_QUERY_ALGORITHM(TYPE(NAME='CHAR_DIGEST_LIKE')))"
+                + ",(NAME=order_id, CIPHER =order_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='MD5'))))"
                 + ",QUERY_WITH_CIPHER_COLUMN=true)";
         CreateEncryptRuleStatement createEncryptRuleStatement = (CreateEncryptRuleStatement) getEncryptDistSQLStatement(sql);
         assertThat(createEncryptRuleStatement.getRules().size(), is(1));
@@ -54,8 +55,10 @@ public final class EncryptDistSqlTest {
     @Test
     public void assertAlterEncryptRule() {
         String sql = "ALTER ENCRYPT RULE t_encrypt (COLUMNS("
-                + " (NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))),"
-                + " (NAME=order_id,CIPHER=order_cipher,TYPE(NAME='MD5'))), QUERY_WITH_CIPHER_COLUMN=TRUE)";
+                + " (NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,ASSISTED_QUERY_COLUMN=user_assisted,LIKE_QUERY_COLUMN=user_like"
+                + ",ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))),ASSISTED_QUERY_ALGORITHM(TYPE(NAME='MD5')),LIKE_QUERY_ALGORITHM(TYPE(NAME='CHAR_DIGEST_LIKE')))"
+                + ",(NAME=order_id, CIPHER =order_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='MD5'))))"
+                + ",QUERY_WITH_CIPHER_COLUMN=true)";
         AlterEncryptRuleStatement alterEncryptRule = (AlterEncryptRuleStatement) getEncryptDistSQLStatement(sql);
         assertThat(alterEncryptRule.getRules().size(), is(1));
         assertEncryptRule(alterEncryptRule.getRules().iterator().next());
@@ -70,7 +73,10 @@ public final class EncryptDistSqlTest {
         assertThat(userEncryptColumn.getName(), is("user_id"));
         assertThat(userEncryptColumn.getPlainColumn(), is("user_plain"));
         assertThat(userEncryptColumn.getCipherColumn(), is("user_cipher"));
+        assertThat(userEncryptColumn.getAssistedQueryColumn(), is("user_assisted"));
         assertThat(userEncryptColumn.getEncryptor().getName(), is("AES"));
+        assertThat(userEncryptColumn.getAssistedQueryEncryptor().getName(), is("MD5"));
+        assertThat(userEncryptColumn.getLikeQueryEncryptor().getName(), is("CHAR_DIGEST_LIKE"));
         Properties props = new Properties();
         props.setProperty("aes-key-value", "123456abc");
         assertThat(userEncryptColumn.getEncryptor().getProps(), is(props));
diff --git a/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/common/constant/DistSQLScriptConstants.java b/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/common/constant/DistSQLScriptConstants.java
index 0e9a51796df..4bf3fe2a717 100644
--- a/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/common/constant/DistSQLScriptConstants.java
+++ b/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/common/constant/DistSQLScriptConstants.java
@@ -164,6 +164,16 @@ public final class DistSQLScriptConstants {
     
     public static final String CIPHER = "CIPHER=%s";
     
+    public static final String ASSISTED_QUERY_COLUMN = "ASSISTED_QUERY_COLUMN=%s";
+    
+    public static final String LIKE_QUERY_COLUMN = "LIKE_QUERY_COLUMN=%s";
+    
+    public static final String ENCRYPT_ALGORITHM = "ENCRYPT_ALGORITHM(%s)";
+    
+    public static final String ASSISTED_QUERY_ALGORITHM = "ASSISTED_QUERY_ALGORITHM(%s)";
+    
+    public static final String LIKE_QUERY_ALGORITHM = "LIKE_QUERY_ALGORITHM(%s)";
+    
     public static final String ALGORITHM_TYPE = "TYPE(NAME='%s', PROPERTIES(%s))";
     
     public static final String ALGORITHM_TYPE_WITHOUT_PROPS = "TYPE(NAME='%s')";
diff --git a/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ConvertYamlConfigurationHandler.java b/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ConvertYamlConfigurationHandler.java
index cafb59507f6..e2d7a4078cb 100644
--- a/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ConvertYamlConfigurationHandler.java
+++ b/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ConvertYamlConfigurationHandler.java
@@ -453,8 +453,7 @@ public final class ConvertYamlConfigurationHandler extends QueryableRALBackendHa
         Iterator<EncryptColumnRuleConfiguration> iterator = ruleConfigs.iterator();
         while (iterator.hasNext()) {
             EncryptColumnRuleConfiguration columnRuleConfig = iterator.next();
-            String columnType = getAlgorithmType(encryptors.get(columnRuleConfig.getEncryptorName()));
-            result.append(String.format(DistSQLScriptConstants.ENCRYPT_COLUMN, columnRuleConfig.getLogicColumn(), getColumnPlainAndCipher(columnRuleConfig), columnType));
+            result.append(String.format(DistSQLScriptConstants.ENCRYPT_COLUMN, columnRuleConfig.getLogicColumn(), getColumns(columnRuleConfig), getEncryptAlgorithms(columnRuleConfig, encryptors)));
             if (iterator.hasNext()) {
                 result.append(DistSQLScriptConstants.COMMA).append(System.lineSeparator());
             }
@@ -462,7 +461,7 @@ public final class ConvertYamlConfigurationHandler extends QueryableRALBackendHa
         return result.toString();
     }
     
-    private String getColumnPlainAndCipher(final EncryptColumnRuleConfiguration ruleConfig) {
+    private String getColumns(final EncryptColumnRuleConfiguration ruleConfig) {
         StringBuilder result = new StringBuilder();
         String plainColumnName = ruleConfig.getPlainColumn();
         String cipherColumnName = ruleConfig.getCipherColumn();
@@ -475,6 +474,31 @@ public final class ConvertYamlConfigurationHandler extends QueryableRALBackendHa
             }
             result.append(String.format(DistSQLScriptConstants.CIPHER, cipherColumnName));
         }
+        if (null != ruleConfig.getAssistedQueryColumn()) {
+            result.append(DistSQLScriptConstants.COMMA).append(" ").append(String.format(DistSQLScriptConstants.ASSISTED_QUERY_COLUMN, ruleConfig.getAssistedQueryColumn()));
+        }
+        if (null != ruleConfig.getLikeQueryColumn()) {
+            result.append(DistSQLScriptConstants.COMMA).append(" ").append(String.format(DistSQLScriptConstants.LIKE_QUERY_COLUMN, ruleConfig.getLikeQueryColumn()));
+        }
+        return result.toString();
+    }
+    
+    private String getEncryptAlgorithms(final EncryptColumnRuleConfiguration ruleConfig, final Map<String, AlgorithmConfiguration> encryptors) {
+        StringBuilder result = new StringBuilder();
+        String cipherEncryptorName = ruleConfig.getEncryptorName();
+        String assistedQueryEncryptorName = ruleConfig.getAssistedQueryEncryptorName();
+        String likeQueryEncryptorName = ruleConfig.getLikeQueryEncryptorName();
+        if (null != cipherEncryptorName) {
+            result.append(String.format(DistSQLScriptConstants.ENCRYPT_ALGORITHM, getAlgorithmType(encryptors.get(cipherEncryptorName))));
+        }
+        if (null != assistedQueryEncryptorName) {
+            result.append(DistSQLScriptConstants.COMMA).append(" ")
+                    .append(String.format(DistSQLScriptConstants.ASSISTED_QUERY_ALGORITHM, getAlgorithmType(encryptors.get(assistedQueryEncryptorName))));
+        }
+        if (null != likeQueryEncryptorName) {
+            result.append(DistSQLScriptConstants.COMMA).append(" ")
+                    .append(String.format(DistSQLScriptConstants.LIKE_QUERY_ALGORITHM, getAlgorithmType(encryptors.get(likeQueryEncryptorName))));
+        }
         return result.toString();
     }
     
diff --git a/proxy/backend/src/test/resources/conf/convert/config-encrypt.yaml b/proxy/backend/src/test/resources/conf/convert/config-encrypt.yaml
index 54b847c28ca..21cb4b1d89e 100644
--- a/proxy/backend/src/test/resources/conf/convert/config-encrypt.yaml
+++ b/proxy/backend/src/test/resources/conf/convert/config-encrypt.yaml
@@ -47,6 +47,8 @@ rules:
         aes-key-value: 123456abc
     md5_encryptor:
       type: MD5
+    like_encryptor:
+      type: CHAR_DIGEST_LIKE
   tables:
     t_encrypt:
       columns:
@@ -54,6 +56,10 @@ rules:
           plainColumn: user_plain
           cipherColumn: user_cipher
           encryptorName: aes_encryptor
+          assistedQueryColumn: user_assisted
+          assistedQueryEncryptorName: md5_encryptor
+          likeQueryColumn: user_like
+          likeQueryEncryptorName: like_encryptor
         order_id:
           cipherColumn: order_cipher
           encryptorName: md5_encryptor
diff --git a/proxy/backend/src/test/resources/conf/convert/config-mix.yaml b/proxy/backend/src/test/resources/conf/convert/config-mix.yaml
index a3de8567b40..a7c5ef6daae 100644
--- a/proxy/backend/src/test/resources/conf/convert/config-mix.yaml
+++ b/proxy/backend/src/test/resources/conf/convert/config-mix.yaml
@@ -146,6 +146,8 @@ rules:
           aes-key-value: 123456abc
       md5_encryptor:
         type: MD5
+      like_encryptor:
+        type: CHAR_DIGEST_LIKE
     tables:
       t_encrypt:
         columns:
@@ -153,6 +155,10 @@ rules:
             plainColumn: user_plain
             cipherColumn: user_cipher
             encryptorName: aes_encryptor
+            assistedQueryColumn: user_assisted
+            assistedQueryEncryptorName: md5_encryptor
+            likeQueryColumn: user_like
+            likeQueryEncryptorName: like_encryptor
           order_id:
             cipherColumn: order_cipher
             encryptorName: md5_encryptor
diff --git a/proxy/backend/src/test/resources/expected/convert-encrypt.yaml b/proxy/backend/src/test/resources/expected/convert-encrypt.yaml
index f6c2241dbfb..697eae28b06 100644
--- a/proxy/backend/src/test/resources/expected/convert-encrypt.yaml
+++ b/proxy/backend/src/test/resources/expected/convert-encrypt.yaml
@@ -31,6 +31,6 @@ PROPERTIES('minPoolSize'='1', 'connectionTimeoutMilliseconds'='30000', 'maxLifet
 
 CREATE ENCRYPT RULE t_encrypt (
 COLUMNS(
-(NAME=user_id, PLAIN=user_plain, CIPHER=user_cipher, TYPE(NAME='aes', PROPERTIES('aes-key-value'='123456abc'))),
-(NAME=order_id, CIPHER=order_cipher, TYPE(NAME='md5'))
+(NAME=user_id, PLAIN=user_plain, CIPHER=user_cipher, ASSISTED_QUERY_COLUMN=user_assisted, LIKE_QUERY_COLUMN=user_like, ENCRYPT_ALGORITHM(TYPE(NAME='aes', PROPERTIES('aes-key-value'='123456abc'))), ASSISTED_QUERY_ALGORITHM(TYPE(NAME='md5')), LIKE_QUERY_ALGORITHM(TYPE(NAME='char_digest_like'))),
+(NAME=order_id, CIPHER=order_cipher, ENCRYPT_ALGORITHM(TYPE(NAME='md5')))
 ),QUERY_WITH_CIPHER_COLUMN=true);
diff --git a/proxy/backend/src/test/resources/expected/convert-mix.yaml b/proxy/backend/src/test/resources/expected/convert-mix.yaml
index 157726b107e..66df564c746 100644
--- a/proxy/backend/src/test/resources/expected/convert-mix.yaml
+++ b/proxy/backend/src/test/resources/expected/convert-mix.yaml
@@ -69,8 +69,8 @@ WRITE_DATA_SOURCE_QUERY_ENABLED=true
 
 CREATE ENCRYPT RULE t_encrypt (
 COLUMNS(
-(NAME=user_id, PLAIN=user_plain, CIPHER=user_cipher, TYPE(NAME='aes', PROPERTIES('aes-key-value'='123456abc'))),
-(NAME=order_id, CIPHER=order_cipher, TYPE(NAME='md5'))
+(NAME=user_id, PLAIN=user_plain, CIPHER=user_cipher, ASSISTED_QUERY_COLUMN=user_assisted, LIKE_QUERY_COLUMN=user_like, ENCRYPT_ALGORITHM(TYPE(NAME='aes', PROPERTIES('aes-key-value'='123456abc'))), ASSISTED_QUERY_ALGORITHM(TYPE(NAME='md5')), LIKE_QUERY_ALGORITHM(TYPE(NAME='char_digest_like'))),
+(NAME=order_id, CIPHER=order_cipher, ENCRYPT_ALGORITHM(TYPE(NAME='md5')))
 ),QUERY_WITH_CIPHER_COLUMN=true);
 
 CREATE SHARDING TABLE RULE t_order (
diff --git a/test/integration-test/scaling/src/test/resources/env/common/migration-command.xml b/test/integration-test/scaling/src/test/resources/env/common/migration-command.xml
index d717426847d..cc0a0de9a50 100644
--- a/test/integration-test/scaling/src/test/resources/env/common/migration-command.xml
+++ b/test/integration-test/scaling/src/test/resources/env/common/migration-command.xml
@@ -59,7 +59,7 @@
     
     <create-target-order-table-encrypt-rule>
         CREATE ENCRYPT RULE t_order (
-        COLUMNS((NAME=status, CIPHER=status,TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')))),
+        COLUMNS((NAME=status, CIPHER=status,ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))))),
         QUERY_WITH_CIPHER_COLUMN=true)
     </create-target-order-table-encrypt-rule>
     
diff --git a/test/integration-test/test-suite/src/test/resources/cases/rdl/rdl-integration-test-cases.xml b/test/integration-test/test-suite/src/test/resources/cases/rdl/rdl-integration-test-cases.xml
index a42a08f5aaa..a99016a0e02 100644
--- a/test/integration-test/test-suite/src/test/resources/cases/rdl/rdl-integration-test-cases.xml
+++ b/test/integration-test/test-suite/src/test/resources/cases/rdl/rdl-integration-test-cases.xml
@@ -69,22 +69,22 @@
             <assertion-sql sql="SHOW BROADCAST TABLE RULES;" />
         </assertion>
     </test-case>
-    <test-case sql="CREATE ENCRYPT RULE t_user (COLUMNS((NAME=pwd,PLAIN=pwd_plain,CIPHER=pwd_cipher,TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')))),QUERY_WITH_CIPHER_COLUMN=true);" db-types="MySQL, PostgreSQL">
+    <test-case sql="CREATE ENCRYPT RULE t_user (COLUMNS((NAME=pwd,PLAIN=pwd_plain,CIPHER=pwd_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))))),QUERY_WITH_CIPHER_COLUMN=true);" db-types="MySQL, PostgreSQL">
         <assertion expected-data-file="create_encrypt_rule.xml">
             <assertion-sql sql="SHOW ENCRYPT TABLE RULE t_user;" />
             <destroy-sql sql="DROP ENCRYPT RULE t_user " />
         </assertion>
     </test-case>
-    <test-case sql="ALTER ENCRYPT RULE t_user (COLUMNS((NAME=pwd,PLAIN=pwd_plain,CIPHER=pwd_cipher,TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abcd')))),QUERY_WITH_CIPHER_COLUMN=true);" db-types="MySQL, PostgreSQL">
+    <test-case sql="ALTER ENCRYPT RULE t_user (COLUMNS((NAME=pwd,PLAIN=pwd_plain,CIPHER=pwd_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abcd'))))),QUERY_WITH_CIPHER_COLUMN=true);" db-types="MySQL, PostgreSQL">
         <assertion expected-data-file="alter_encrypt_rule.xml">
-            <initial-sql sql="CREATE ENCRYPT RULE t_user (COLUMNS((NAME=pwd,PLAIN=pwd_plain,CIPHER=pwd_cipher,TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')))),QUERY_WITH_CIPHER_COLUMN=true);" />
+            <initial-sql sql="CREATE ENCRYPT RULE t_user (COLUMNS((NAME=pwd,PLAIN=pwd_plain,CIPHER=pwd_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))))),QUERY_WITH_CIPHER_COLUMN=true);" />
             <assertion-sql sql="SHOW ENCRYPT TABLE RULE t_user;" />
             <destroy-sql sql="DROP ENCRYPT RULE t_user " />
         </assertion>
     </test-case>
     <test-case sql="DROP ENCRYPT RULE t_user;" db-types="MySQL, PostgreSQL">
         <assertion expected-data-file="drop_encrypt_rule.xml">
-            <initial-sql sql="CREATE ENCRYPT RULE t_user (COLUMNS((NAME=pwd,PLAIN=pwd_plain,CIPHER=pwd_cipher,TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')))),QUERY_WITH_CIPHER_COLUMN=true);" />
+            <initial-sql sql="CREATE ENCRYPT RULE t_user (COLUMNS((NAME=pwd,PLAIN=pwd_plain,CIPHER=pwd_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))))),QUERY_WITH_CIPHER_COLUMN=true);" />
             <assertion-sql sql="SHOW ENCRYPT TABLE RULE t_user;" />
         </assertion>
     </test-case>
diff --git a/test/parser/src/main/resources/sql/supported/rdl/alter.xml b/test/parser/src/main/resources/sql/supported/rdl/alter.xml
index 058b4ef35e2..5c702eeece1 100644
--- a/test/parser/src/main/resources/sql/supported/rdl/alter.xml
+++ b/test/parser/src/main/resources/sql/supported/rdl/alter.xml
@@ -28,10 +28,10 @@
     <sql-case id="alter-storage-unit-url-single-with-properties" value="ALTER STORAGE UNIT ds_0(URL='jdbc:mysql://127.0.0.1:3306/test0',USER='ROOT',PASSWORD='123456',PROPERTIES('maxPoolSize'='30'))" db-types="ShardingSphere" />
     <sql-case id="alter-readwrite-splitting-rule" value="ALTER READWRITE_SPLITTING RULE ms_group_0 (AUTO_AWARE_RESOURCE=group_0, TYPE(NAME='random',PROPERTIES('read_weight'='2:1'))), ms_group_1 (WRITE_STORAGE_UNIT=primary_ds, READ_STORAGE_UNITS(replica_ds_0,replica_ds_1),TYPE(NAME='random'))" db-types="ShardingSphere" />
     <sql-case id="alter-database-discovery-definition-rule" value="ALTER DB_DISCOVERY RULE ha_group_0 (STORAGE_UNITS(ds_0,ds_1), TYPE(NAME='mgr',PROPERTIES('groupName'='92504d5b-6dec')),HEARTBEAT(PROPERTIES('keepAliveCron'='0/5 * * * * ?'))), ha_group_1 (STORAGE_UNITS(ds_2,ds_3), TYPE(NAME='mgr2',PROPERTIES('groupName'='92504d5b-6dec-2')),HEARTBEAT(PROPERTIES('keepAliveCron'='0/6 * * * * ?')))" db-types="ShardingSphere" />
-    <sql-case id="alter-encrypt-rule" value="ALTER ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))), (NAME=order_id, CIPHER =order_cipher,TYPE(NAME='MD5'))))" db-types="ShardingSphere" />
-    <sql-case id="alter-encrypt-rule-with-assisted-query-column" value="ALTER ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,ASSISTED_QUERY_COLUMN=assisted_column, TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')), TYPE(NAME='MD5')), (NAME=order_id, CIPHER =order_cipher,TYPE(NAME='MD5'))))" db-types="ShardingSphere" />
-    <sql-case id="alter-encrypt-rule-with-like-query-column" value="ALTER ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,LIKE_QUERY_COLUMN=like_column, TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')), TYPE(NAME='CHAR_DIGEST_LIKE')), (NAME=order_id, CIPHER =order_cipher,TYPE(NAME='MD5'))))" db-types="ShardingSphere" />
-    <sql-case id="alter-encrypt-rule-with-query-with-cipher-column" value="ALTER ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher, TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))), (NAME=order_id, CIPHER =order_cipher,TYPE(NAME='MD5'))), QUERY_WITH_CIPHER_COLUMN=false)" db-types="ShardingSphere" />
+    <sql-case id="alter-encrypt-rule" value="ALTER ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')))), (NAME=order_id, CIPHER =order_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='MD5')))))" db-types="ShardingSphere" />
+    <sql-case id="alter-encrypt-rule-with-assisted-query-column" value="ALTER ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,ASSISTED_QUERY_COLUMN=assisted_column, ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))), ASSISTED_QUERY_ALGORITHM(TYPE(NAME='MD5'))), (NAME=order_id, CIPHER =order_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='MD5')))))" db-types="ShardingSphere" />
+    <sql-case id="alter-encrypt-rule-with-like-query-column" value="ALTER ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,LIKE_QUERY_COLUMN=like_column, ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))), LIKE_QUERY_ALGORITHM(TYPE(NAME='CHAR_DIGEST_LIKE'))), (NAME=order_id, CIPHER =order_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='MD5')))))" db-types="ShardingSphere" />
+    <sql-case id="alter-encrypt-rule-with-query-with-cipher-column" value="ALTER ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher, ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')))), (NAME=order_id, CIPHER =order_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='MD5')))), QUERY_WITH_CIPHER_COLUMN=false)" db-types="ShardingSphere" />
     <sql-case id="alter-default-shadow-algorithm" value="ALTER DEFAULT SHADOW ALGORITHM TYPE(NAME='SIMPLE_HINT', PROPERTIES('shadow'='true', 'foo'='bar'))" db-types="ShardingSphere" />
     <sql-case id="alter-shadow-rule" value="ALTER SHADOW RULE shadow_rule(SOURCE=demo_ds,SHADOW=demo_ds_shadow,t_order(TYPE(NAME='REGEX_MATCH',PROPERTIES('operation'='insert','column'='user_id','regex'='[1]')),TYPE(NAME='SIMPLE_HINT',PROPERTIES('shadow'='true','foo'='bar'))))" db-types="ShardingSphere" />
     <sql-case id="alter-sharding-auto-table-rule" value="ALTER SHARDING TABLE RULE t_order (STORAGE_UNITS(ms_group_0,ms_group_1), SHARDING_COLUMN=order_id, TYPE(NAME='hash_mod',PROPERTIES('sharding-count'='4')),KEY_GENERATE_STRATEGY(COLUMN=another_id,TYPE(NAME='snowflake')))" db-types="ShardingSphere" />
diff --git a/test/parser/src/main/resources/sql/supported/rdl/create.xml b/test/parser/src/main/resources/sql/supported/rdl/create.xml
index ff1a44fa530..d607575579b 100644
--- a/test/parser/src/main/resources/sql/supported/rdl/create.xml
+++ b/test/parser/src/main/resources/sql/supported/rdl/create.xml
@@ -35,9 +35,9 @@
     <sql-case id="create-static-readwrite-splitting-rule" value="CREATE READWRITE_SPLITTING RULE ms_group_0 (WRITE_STORAGE_UNIT=primary_ds, READ_STORAGE_UNITS(replica_ds_0,replica_ds_1), TYPE(NAME='random')))" db-types="ShardingSphere" />
     <sql-case id="create-dynamic-readwrite-splitting-rule" value="CREATE READWRITE_SPLITTING RULE ms_group_1(AUTO_AWARE_RESOURCE=group_0, TYPE(NAME='random',PROPERTIES('read_weight'='2:1')))" db-types="ShardingSphere" />
     <sql-case id="create-database-discovery-definition-rule" value="CREATE DB_DISCOVERY RULE ha_group_0 (STORAGE_UNITS(ds_0,ds_1), TYPE(NAME='mgr',PROPERTIES('groupName'='92504d5b-6dec')),HEARTBEAT(PROPERTIES('keepAliveCron'='0/5 * * * * ?'))), ha_group_1 (STORAGE_UNITS(ds_2,ds_3), TYPE(NAME='mgr2',PROPERTIES('groupName'='92504d5b-6dec-2')),HEARTBEAT(PROPERTIES('keepAliveCron'='0/6 * * * * ?')))" db-types="ShardingSphere" />
-    <sql-case id="create-encrypt-rule" value="CREATE ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))), (NAME=order_id, CIPHER =order_cipher,TYPE(NAME='MD5'))))" db-types="ShardingSphere" />
-    <sql-case id="create-encrypt-rule-with-assisted-query-column" value="CREATE ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,ASSISTED_QUERY_COLUMN=assisted_column, TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')), TYPE(NAME='MD5')), (NAME=order_id, CIPHER =order_cipher,TYPE(NAME='MD5'))))" db-types="ShardingSphere" />
-    <sql-case id="create-encrypt-rule-with-like-query-column" value="CREATE ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,LIKE_QUERY_COLUMN=like_column, TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')), TYPE(NAME='CHAR_DIGEST_LIKE')), (NAME=order_id, CIPHER =order_cipher,TYPE(NAME='MD5'))))" db-types="ShardingSphere" />
+    <sql-case id="create-encrypt-rule" value="CREATE ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')))), (NAME=order_id, CIPHER =order_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='MD5')))))" db-types="ShardingSphere" />
+    <sql-case id="create-encrypt-rule-with-assisted-query-column" value="CREATE ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,ASSISTED_QUERY_COLUMN=assisted_column, ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))), ASSISTED_QUERY_ALGORITHM(TYPE(NAME='MD5'))), (NAME=order_id, CIPHER =order_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='MD5')))))" db-types="ShardingSphere" />
+    <sql-case id="create-encrypt-rule-with-like-query-column" value="CREATE ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,LIKE_QUERY_COLUMN=like_column, ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))), LIKE_QUERY_ALGORITHM(TYPE(NAME='CHAR_DIGEST_LIKE'))), (NAME=order_id, CIPHER =order_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='MD5')))))" db-types="ShardingSphere" />
     <sql-case id="create-shadow-rule" value="CREATE SHADOW RULE shadow_rule(SOURCE=demo_ds,SHADOW=demo_ds_shadow,t_order(TYPE(NAME='REGEX_MATCH',PROPERTIES('operation'='insert','column'='user_id','regex'='[1]')),TYPE(NAME='SIMPLE_HINT',PROPERTIES('shadow'='true','foo'='bar'))))" db-types="ShardingSphere" />
     <sql-case id="create-default-sharding-table-strategy-with-lower-case" value="create default sharding table strategy(type='standard', sharding_column=order_id, SHARDING_ALGORITHM(TYPE(NAME='INLINE',PROPERTIES('algorithm-expression'='t_order_${order_id % 2}'))))" db-types="ShardingSphere" />
     <sql-case id="create-default-sharding-database-strategy" value="CREATE DEFAULT SHARDING DATABASE STRATEGY(TYPE='STANDARD', SHARDING_COLUMN=ORDER_ID, SHARDING_ALGORITHM(TYPE(NAME='INLINE',PROPERTIES('algorithm-expression'='ms_group_${order_id % 2}'))))" db-types="ShardingSphere" />
@@ -55,14 +55,9 @@
     <sql-case id="create-broadcast-table-rule-with-quota" value="CREATE BROADCAST TABLE RULE `t_1`,`t_2` " db-types="ShardingSphere" />
     <sql-case id="create-readwrite-splitting-rule-with-quota" value="CREATE READWRITE_SPLITTING RULE `ms_group_0` (WRITE_STORAGE_UNIT=primary_ds, READ_STORAGE_UNITS(replica_ds_0,replica_ds_1), TYPE(NAME='random')))" db-types="ShardingSphere" />
     <sql-case id="create-database-discovery-rule-with-quota" value="CREATE DB_DISCOVERY RULE ha_group_0 (STORAGE_UNITS(ds_0,ds_1), TYPE(NAME='mgr',PROPERTIES('groupName'='92504d5b-6dec')),HEARTBEAT(PROPERTIES('keepAliveCron'='0/5 * * * * ?'))), ha_group_1 (STORAGE_UNITS(ds_2,ds_3), TYPE(NAME='mgr2',PROPERTIES('groupName'='92504d5b-6dec-2')),HEARTBEAT(PROPERTIES('keepAliveCron'='0/6 * * * * ?')))" db-types="ShardingSphere" />
-    <sql-case id="create-encrypt-rule-with-quota" value="CREATE ENCRYPT RULE `encrypt` (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))), (NAME=order_id, CIPHER =order_cipher,TYPE(NAME='MD5'))))" db-types="ShardingSphere" />
-    <sql-case id="create-encrypt-rule-with-query-with-cipher-column" value="CREATE ENCRYPT RULE `encrypt` (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')))),QUERY_WITH_CIPHER_COLUMN=false)" db-types="ShardingSphere" />
+    <sql-case id="create-encrypt-rule-with-quota" value="CREATE ENCRYPT RULE `encrypt` (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc')))), (NAME=order_id, CIPHER =order_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='MD5')))))" db-types="ShardingSphere" />
+    <sql-case id="create-encrypt-rule-with-query-with-cipher-column" value="CREATE ENCRYPT RULE `encrypt` (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,ENCRYPT_ALGORITHM(TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))))),QUERY_WITH_CIPHER_COLUMN=false)" db-types="ShardingSphere" />
     <sql-case id="create-shadow-rule-with-quota" value="CREATE SHADOW RULE `shadow_rule`(SOURCE=demo_ds,SHADOW=demo_ds_shadow,t_order(TYPE(NAME='REGEX_MATCH',PROPERTIES('operation'='insert','column'='user_id','regex'='[1]')),TYPE(NAME='SIMPLE_HINT',PROPERTIES('shadow'='true','foo'='bar'))))" db-types="ShardingSphere" />
     <sql-case id="create-sharding-auditor" value="CREATE SHARDING AUDITOR sharding_key_required_auditor(TYPE(NAME='DML_SHARDING_CONDITIONS'))" db-types="ShardingSphere" />
     <sql-case id="single-register-migration-source-storage-unit" value="REGISTER MIGRATION SOURCE STORAGE UNIT ds_0 (URL='jdbc:mysql://127.0.0.1:3306/test0',USER='ROOT',PASSWORD='123456');" db-types="ShardingSphere" />
-    <sql-case id="create-sharding-table-rule-without-quote-on-keyword" value="CREATE SHARDING TABLE RULE t_order_item (DATANODES('ds_${0..1}.t_order_item_${0..1}'), DATABASE_STRATEGY(TYPE=standard,SHARDING_COLUMN=user_id,SHARDING_ALGORITHM(TYPE(NAME=inline,PROPERTIES('algorithm-expression'='ds_${user_id % 2}')))), TABLE_STRATEGY(TYPE=standard,SHARDING_COLUMN=order_id,SHARDING_ALGORITHM(TYPE(NAME=inline,PROPERTIES('algorithm-expression'='t_order_item_${order_id % 2}')))), KEY_GENERATE_STR [...]
-    <sql-case id="create-encrypt-rule-without-quote-on-build-in-algorithm-type-name" value="CREATE ENCRYPT RULE t_encrypt (COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,TYPE(NAME=AES,PROPERTIES('aes-key-value'='123456abc'))), (NAME=order_id, CIPHER =order_cipher,TYPE(NAME=MD5))),QUERY_WITH_CIPHER_COLUMN=true);" db-types="ShardingSphere" />
-    <sql-case id="create-readwrite-splitting-rule-without-quote-on-build-in-algorithm-type-name" value="CREATE READWRITE_SPLITTING RULE ms_group_0 (WRITE_STORAGE_UNIT=write_ds, READ_STORAGE_UNITS(read_ds_0,read_ds_1), TYPE(NAME=RANDOM));" db-types="ShardingSphere" />
-    <sql-case id="create-db-discovery-rule-without-quote-on-build-in-db-discovery-type-name" value="CREATE DB_DISCOVERY RULE db_discovery_group_0 (STORAGE_UNITS(ds_0, ds_1), TYPE(NAME=MYSQL.MGR,PROPERTIES('group-name'='92504d5b-6dec')),HEARTBEAT(PROPERTIES('keep-alive-cron'='0/5 * * * * ?')));" db-types="ShardingSphere" />
-    <sql-case id="create-shadow-rule-without-quote-on-build-in-shadow-algorithm-name" value="CREATE SHADOW RULE shadow_rule(SOURCE=demo_ds, SHADOW=demo_ds_shadow, t_order(TYPE(NAME=SIMPLE_HINT, PROPERTIES('shadow'='true', 'foo'='bar'))));" db-types="ShardingSphere" />
 </sql-cases>