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/09/16 14:42:10 UTC

[shardingsphere] branch master updated: Add convert shadow distsql and test case (#20843)

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 7152fae2d62 Add convert shadow distsql and test case (#20843)
7152fae2d62 is described below

commit 7152fae2d62f1337729df9fc2af5c06c2b89063b
Author: Xin Huang <57...@users.noreply.github.com>
AuthorDate: Fri Sep 16 22:41:58 2022 +0800

    Add convert shadow distsql and test case (#20843)
    
    * rebase conflicts fixed
    
    * rebase conflict in yaml fixed
    
    * ConvertYamlConfigurationHandler conflict fixed
    
    * QueryableRALStatementAssert.java bug fixed
    
    * indentation bug fixed
    
    * ConvertYamlConfigurationHandlerTest conflict fixed
    
    * Add convert shadow distsql and test case
    
    * Conflicts-fix:Add convert sharding-broadcast distsql and test case
    
    * bugfix: shadow properties in wrong order
    
    * bugfix: shadow type properties in wrong order
    
    * Finish: Add convert sharding-broadcast distsql and test case
    
    * update: code review
    
    * update: CI fixed and code review
---
 .../common/constant/DistSQLScriptConstants.java    |  28 +++--
 .../queryable/ConvertYamlConfigurationHandler.java | 135 +++++++++++++++++++--
 .../ConvertYamlConfigurationHandlerTest.java       |  12 ++
 .../test/resources/conf/convert/config-shadow.yaml |  89 ++++++++++++++
 .../resources/conf/convert/config-sharding.yaml    |   2 +
 .../resources/expected/convert-create-encrypt.yaml |   2 +-
 ...ate-encrypt.yaml => convert-create-shadow.yaml} |  24 ++--
 .../expected/convert-create-sharding.yaml          |   1 +
 .../expected/convert-database-discovery.yaml       |   4 +-
 9 files changed, 269 insertions(+), 28 deletions(-)

diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/common/constant/DistSQLScriptConstants.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/common/constant/DistSQLScriptConstants.java
index ac8876463e8..ef0c9d7f01d 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/common/constant/DistSQLScriptConstants.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/common/constant/DistSQLScriptConstants.java
@@ -36,6 +36,8 @@ public final class DistSQLScriptConstants {
     
     public static final String ENCRYPT_DB = "encrypt_db";
     
+    public static final String SHADOW_DB = "shadow_db";
+    
     public static final String STANDARD = "standard";
     
     public static final String COMPLEX = "complex";
@@ -125,7 +127,9 @@ public final class DistSQLScriptConstants {
     
     public static final String SHARDING_BINDING_TABLE_RULES = "CREATE SHARDING BINDING TABLE RULES %s";
     
-    public static final String BINDING = "(%s)";
+    public static final String BRACKET = "(%s)";
+    
+    public static final String SHARDING_BROADCAST_TABLE_RULES = "CREATE SHARDING BROADCAST TABLE RULES %s";
     
     public static final String CREATE_READWRITE_SPLITTING_RULE = "CREATE READWRITE_SPLITTING RULE";
     
@@ -157,10 +161,6 @@ public final class DistSQLScriptConstants {
             + System.lineSeparator()
             + ")";
     
-    public static final String DB_DISCOVERY_TYPE = "TYPE(NAME=\'%s\', PROPERTIES(%s))";
-    
-    public static final String DB_DISCOVERY_PROPERTY = "\'%s\'=\'%s\'";
-    
     public static final String CREATE_ENCRYPT = "CREATE ENCRYPT RULE";
     
     public static final String ENCRYPT = " %s ("
@@ -178,9 +178,23 @@ public final class DistSQLScriptConstants {
     
     public static final String CIPHER = "CIPHER=%s";
     
-    public static final String ENCRYPT_TYPE = "TYPE(NAME=\'%s\',PROPERTIES(%s))";
+    public static final String TYPE = "TYPE(NAME=\'%s\', PROPERTIES(%s))";
     
     public static final String ENCRYPT_TYPE_WITHOUT_PROPERTIES = "TYPE(NAME=\'%s\')";
     
-    public static final String ENCRYPT_TYPE_PROPERTIES = "\'%s\'=\'%s\'";
+    public static final String CREATE_SHADOW = "CREATE SHADOW RULE";
+    
+    public static final String SHADOW = " %s("
+            + System.lineSeparator()
+            + "SOURCE=%s,"
+            + System.lineSeparator()
+            + "SHADOW=%s,"
+            + System.lineSeparator()
+            + "%s"
+            + System.lineSeparator()
+            + ")";
+    
+    public static final String SHADOW_TABLE = "%s(%s)";
+    
+    public static final String SHADOW_TABLE_TYPE = "(TYPE(NAME=\"%s\", PROPERTIES(%s)))";
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ConvertYamlConfigurationHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ConvertYamlConfigurationHandler.java
index b9ee691cc61..7ea6b93a291 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ConvertYamlConfigurationHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ConvertYamlConfigurationHandler.java
@@ -46,6 +46,9 @@ import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.common.consta
 import org.apache.shardingsphere.readwritesplitting.yaml.config.YamlReadwriteSplittingRuleConfiguration;
 import org.apache.shardingsphere.readwritesplitting.yaml.config.rule.YamlReadwriteSplittingDataSourceRuleConfiguration;
 import org.apache.shardingsphere.readwritesplitting.yaml.config.strategy.YamlStaticReadwriteSplittingStrategyConfiguration;
+import org.apache.shardingsphere.shadow.yaml.config.YamlShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.yaml.config.datasource.YamlShadowDataSourceConfiguration;
+import org.apache.shardingsphere.shadow.yaml.config.table.YamlShadowTableConfiguration;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
@@ -63,6 +66,9 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
+import java.util.SortedMap;
+import java.util.Set;
+import java.util.TreeMap;
 
 /**
  * Convert YAML configuration handler.
@@ -109,6 +115,9 @@ public final class ConvertYamlConfigurationHandler extends QueryableRALBackendHa
             case DistSQLScriptConstants.ENCRYPT_DB:
                 addEncryptDistSQL(yamlConfig, result);
                 break;
+            case DistSQLScriptConstants.SHADOW_DB:
+                addShadowDistSQL(yamlConfig, result);
+                break;
             default:
                 break;
         }
@@ -144,6 +153,12 @@ public final class ConvertYamlConfigurationHandler extends QueryableRALBackendHa
         appendEncryptRules(yamlConfig.getRules(), result);
     }
     
+    private void addShadowDistSQL(final YamlProxyDatabaseConfiguration yamlConfig, final StringBuilder result) {
+        appendDatabase(yamlConfig.getDatabaseName(), result);
+        appendResources(yamlConfig.getDataSources(), result);
+        appendShadowRules(yamlConfig.getRules(), result);
+    }
+    
     private void appendDatabase(final String databaseName, final StringBuilder result) {
         result.append(String.format(DistSQLScriptConstants.CREATE_DATABASE, databaseName)).append(System.lineSeparator());
         result.append(String.format(DistSQLScriptConstants.USE_DATABASE, databaseName)).append(System.lineSeparator());
@@ -207,13 +222,14 @@ public final class ConvertYamlConfigurationHandler extends QueryableRALBackendHa
         if (ruleConfigs.isEmpty()) {
             return;
         }
-        for (YamlRuleConfiguration each : ruleConfigs) {
-            ShardingRuleConfiguration shardingRuleConfig = new YamlShardingRuleConfigurationSwapper().swapToObject((YamlShardingRuleConfiguration) each);
+        for (YamlRuleConfiguration ruleConfig : ruleConfigs) {
+            ShardingRuleConfiguration shardingRuleConfig = new YamlShardingRuleConfigurationSwapper().swapToObject((YamlShardingRuleConfiguration) ruleConfig);
             appendShardingAlgorithms(shardingRuleConfig, result);
             appendKeyGenerators(shardingRuleConfig, result);
             appendShardingTableRules(shardingRuleConfig, result);
             // TODO append autoTables
             appendShardingBindingTableRules(shardingRuleConfig, result);
+            appendShardingBroadcastTableRules(shardingRuleConfig, result);
         }
     }
     
@@ -326,7 +342,25 @@ public final class ConvertYamlConfigurationHandler extends QueryableRALBackendHa
         StringBuilder result = new StringBuilder();
         while (iterator.hasNext()) {
             String binding = iterator.next();
-            result.append(String.format(DistSQLScriptConstants.BINDING, binding));
+            result.append(String.format(DistSQLScriptConstants.BRACKET, binding));
+            if (iterator.hasNext()) {
+                result.append(DistSQLScriptConstants.COMMA);
+            }
+        }
+        result.append(DistSQLScriptConstants.SEMI).append(System.lineSeparator());
+        return result.toString();
+    }
+    
+    private void appendShardingBroadcastTableRules(final ShardingRuleConfiguration shardingRuleConfig, final StringBuilder result) {
+        String broadcast = getBroadcast(shardingRuleConfig.getBroadcastTables().iterator());
+        result.append(String.format(DistSQLScriptConstants.SHARDING_BROADCAST_TABLE_RULES, broadcast));
+    }
+    
+    private String getBroadcast(final Iterator<String> iterator) {
+        StringBuilder result = new StringBuilder();
+        while (iterator.hasNext()) {
+            String broadcast = iterator.next();
+            result.append(String.format(DistSQLScriptConstants.BRACKET, broadcast));
             if (iterator.hasNext()) {
                 result.append(DistSQLScriptConstants.COMMA);
             }
@@ -468,7 +502,7 @@ public final class ConvertYamlConfigurationHandler extends QueryableRALBackendHa
                 getDatabaseDiscoveryProperties(entry.getValue().getProps(), properties);
                 String typeName = entry.getValue().getType();
                 String typePros = properties.toString();
-                result.append(String.format(DistSQLScriptConstants.DB_DISCOVERY_TYPE, typeName, typePros));
+                result.append(String.format(DistSQLScriptConstants.TYPE, typeName, typePros));
             }
         }
         return result.toString();
@@ -478,7 +512,7 @@ public final class ConvertYamlConfigurationHandler extends QueryableRALBackendHa
         Iterator<Entry<Object, Object>> props = heartbeatProperties.entrySet().iterator();
         while (props.hasNext()) {
             Entry<Object, Object> entry = props.next();
-            result.append(String.format(DistSQLScriptConstants.DB_DISCOVERY_PROPERTY, entry.getKey(), entry.getValue()));
+            result.append(String.format(DistSQLScriptConstants.PROPERTY, entry.getKey(), entry.getValue()));
         }
     }
     
@@ -544,7 +578,7 @@ public final class ConvertYamlConfigurationHandler extends QueryableRALBackendHa
                 String typeName = entry.getValue().getType();
                 if (!entry.getValue().getProps().isEmpty()) {
                     String properties = getColumnTypeProperties(entry.getValue().getProps());
-                    result.append(String.format(DistSQLScriptConstants.ENCRYPT_TYPE, typeName, properties));
+                    result.append(String.format(DistSQLScriptConstants.TYPE, typeName, properties));
                 } else {
                     result.append(String.format(DistSQLScriptConstants.ENCRYPT_TYPE_WITHOUT_PROPERTIES, typeName));
                 }
@@ -558,7 +592,7 @@ public final class ConvertYamlConfigurationHandler extends QueryableRALBackendHa
         Iterator<Entry<Object, Object>> props = properties.entrySet().iterator();
         while (props.hasNext()) {
             Entry<Object, Object> entry = props.next();
-            result.append(String.format(DistSQLScriptConstants.ENCRYPT_TYPE_PROPERTIES, entry.getKey(), entry.getValue()));
+            result.append(String.format(DistSQLScriptConstants.PROPERTY, entry.getKey(), entry.getValue()));
             if (props.hasNext()) {
                 result.append(DistSQLScriptConstants.COMMA);
             }
@@ -569,4 +603,91 @@ public final class ConvertYamlConfigurationHandler extends QueryableRALBackendHa
     private String getQueryWithCipher(final Boolean queryWithCipherColumn, final YamlRuleConfiguration ruleConfig) {
         return String.valueOf(null == queryWithCipherColumn ? ((YamlEncryptRuleConfiguration) ruleConfig).isQueryWithCipherColumn() : queryWithCipherColumn.toString().toLowerCase());
     }
+    
+    private void appendShadowRules(final Collection<YamlRuleConfiguration> ruleConfigs, final StringBuilder result) {
+        if (ruleConfigs.isEmpty()) {
+            return;
+        }
+        result.append(DistSQLScriptConstants.CREATE_SHADOW);
+        for (YamlRuleConfiguration ruleConfig : ruleConfigs) {
+            Iterator<Entry<String, YamlShadowDataSourceConfiguration>> shadowDataSourcesIter = ((YamlShadowRuleConfiguration) ruleConfig).getDataSources().entrySet().iterator();
+            while (shadowDataSourcesIter.hasNext()) {
+                Entry<String, YamlShadowDataSourceConfiguration> entry = shadowDataSourcesIter.next();
+                String shadowRuleName = entry.getKey();
+                String source = entry.getValue().getProductionDataSourceName();
+                String shadow = entry.getValue().getShadowDataSourceName();
+                String shadowTables = getShadowTables(entry.getKey(), ruleConfig);
+                result.append(String.format(DistSQLScriptConstants.SHADOW, shadowRuleName, source, shadow, shadowTables));
+                if (shadowDataSourcesIter.hasNext()) {
+                    result.append(DistSQLScriptConstants.COMMA);
+                }
+            }
+            result.append(DistSQLScriptConstants.SEMI).append(System.lineSeparator());
+        }
+    }
+    
+    private String getShadowTables(final String shadowName, final YamlRuleConfiguration ruleConfig) {
+        StringBuilder result = new StringBuilder();
+        Iterator<Entry<String, YamlShadowTableConfiguration>> shadowTablesIter = ((YamlShadowRuleConfiguration) ruleConfig).getTables().entrySet().iterator();
+        while (shadowTablesIter.hasNext()) {
+            Entry<String, YamlShadowTableConfiguration> entry = shadowTablesIter.next();
+            if (isBelongToShadowRule(shadowName, entry.getValue().getDataSourceNames())) {
+                String tableName = entry.getKey();
+                String tableTypes = getShadowTableTypes(entry.getValue().getShadowAlgorithmNames(), ruleConfig);
+                result.append(String.format(DistSQLScriptConstants.SHADOW_TABLE, tableName, tableTypes));
+            }
+            if (shadowTablesIter.hasNext()) {
+                result.append(DistSQLScriptConstants.COMMA).append(System.lineSeparator());
+            }
+        }
+        return result.toString();
+    }
+    
+    private Boolean isBelongToShadowRule(final String shadowName, final Collection<String> dataSourceNames) {
+        Iterator<String> dataSourceNamesIter = dataSourceNames.iterator();
+        while (dataSourceNamesIter.hasNext()) {
+            String dataSourceName = dataSourceNamesIter.next();
+            if (dataSourceName.equals(shadowName)) {
+                return true;
+            }
+        }
+        return false;
+    }
+    
+    private String getShadowTableTypes(final Collection<String> shadowAlgorithmNames, final YamlRuleConfiguration ruleConfig) {
+        StringBuilder result = new StringBuilder();
+        Iterator<String> shadowAlgorithmNamesIter = shadowAlgorithmNames.iterator();
+        while (shadowAlgorithmNamesIter.hasNext()) {
+            String shadowAlgorithmName = shadowAlgorithmNamesIter.next();
+            Iterator<Entry<String, YamlAlgorithmConfiguration>> shadowAlgorithmsIter = ((YamlShadowRuleConfiguration) ruleConfig).getShadowAlgorithms().entrySet().iterator();
+            while (shadowAlgorithmsIter.hasNext()) {
+                Entry<String, YamlAlgorithmConfiguration> entry = shadowAlgorithmsIter.next();
+                if (entry.getKey().equals(shadowAlgorithmName)) {
+                    String typeName = entry.getValue().getType();
+                    String typeProperties = getAlgorithmProperties(entry.getValue().getProps());
+                    result.append(String.format(DistSQLScriptConstants.SHADOW_TABLE_TYPE, typeName, typeProperties));
+                }
+            }
+            if (shadowAlgorithmNamesIter.hasNext()) {
+                result.append(DistSQLScriptConstants.COMMA);
+            }
+        }
+        return result.toString();
+    }
+    
+    private String getAlgorithmProperties(final Properties algorithmProperties) {
+        StringBuilder result = new StringBuilder();
+        SortedMap<String, String> sortedMap = new TreeMap(algorithmProperties);
+        Set<String> set = sortedMap.keySet();
+        Iterator<String> prosIter = set.iterator();
+        while (prosIter.hasNext()) {
+            String prosKey = prosIter.next();
+            String prosValue = algorithmProperties.getProperty(prosKey);
+            result.append(String.format(DistSQLScriptConstants.PROPERTY, prosKey, prosValue));
+            if (prosIter.hasNext()) {
+                result.append(DistSQLScriptConstants.COMMA);
+            }
+        }
+        return result.toString();
+    }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ConvertYamlConfigurationHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ConvertYamlConfigurationHandlerTest.java
index 6d6a02583e6..9c5eb125f32 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ConvertYamlConfigurationHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ConvertYamlConfigurationHandlerTest.java
@@ -56,6 +56,8 @@ public final class ConvertYamlConfigurationHandlerTest extends ProxyContextResto
     
     private final String encryptFilePath = "/conf/convert/config-encrypt.yaml";
     
+    private final String shadowFilePath = "/conf/convert/config-shadow.yaml";
+    
     private final String resourceExpectedFilePath = "/expected/convert-add-resource.yaml";
     
     private final String shardingExpectedFilePath = "/expected/convert-create-sharding.yaml";
@@ -66,6 +68,8 @@ public final class ConvertYamlConfigurationHandlerTest extends ProxyContextResto
     
     private final String encryptExpectedFilePath = "/expected/convert-create-encrypt.yaml";
     
+    private final String shadowExpectedFilePath = "/expected/convert-create-shadow.yaml";
+    
     private final String resource = "resource";
     
     private final String sharding = "sharding";
@@ -76,6 +80,8 @@ public final class ConvertYamlConfigurationHandlerTest extends ProxyContextResto
     
     private final String encrypt = "encrypt";
     
+    private final String shadow = "shadow";
+    
     private final Map<String, String> featureMap = new HashMap<>(5, 1);
     
     @Before
@@ -85,6 +91,7 @@ public final class ConvertYamlConfigurationHandlerTest extends ProxyContextResto
         featureMap.put(readWriteSplitting, readWriteSplittingFilePath);
         featureMap.put(databaseDiscovery, databaseDiscoveryFilePath);
         featureMap.put(encrypt, encryptFilePath);
+        featureMap.put(shadow, shadowFilePath);
     }
     
     @Before
@@ -118,6 +125,11 @@ public final class ConvertYamlConfigurationHandlerTest extends ProxyContextResto
         assertExecute(encrypt, encryptExpectedFilePath);
     }
     
+    @Test
+    public void assertExecuteWithCreateShadow() throws SQLException {
+        assertExecute(shadow, shadowExpectedFilePath);
+    }
+    
     public void assertExecute(final String type, final String expectedFilePath) throws SQLException {
         ConvertYamlConfigurationHandler handler = new ConvertYamlConfigurationHandler();
         handler.init(new ConvertYamlConfigurationStatement(Objects.requireNonNull(ConvertYamlConfigurationHandlerTest.class.getResource(featureMap.get(type))).getPath()),
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/conf/convert/config-shadow.yaml b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/conf/convert/config-shadow.yaml
new file mode 100644
index 00000000000..effd66b1b86
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/conf/convert/config-shadow.yaml
@@ -0,0 +1,89 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+databaseName: shadow_db
+
+dataSources:
+  ds:
+    url: jdbc:mysql://127.0.0.1:3306/demo_shadow_ds_0?serverTimezone=UTC&useSSL=false
+    username: root
+    password: 12345678
+    connectionTimeoutMilliseconds: 30000
+    idleTimeoutMilliseconds: 60000
+    maxLifetimeMilliseconds: 1800000
+    maxPoolSize: 50
+    minPoolSize: 1
+  shadow_ds:
+    url: jdbc:mysql://127.0.0.1:3306/demo_shadow_ds_1?serverTimezone=UTC&useSSL=false
+    username: root
+    password: 12345678
+    connectionTimeoutMilliseconds: 30000
+    idleTimeoutMilliseconds: 60000
+    maxLifetimeMilliseconds: 1800000
+    maxPoolSize: 50
+    minPoolSize: 1
+
+rules:
+- !SHADOW
+  dataSources:
+    shadowDataSource:
+      productionDataSourceName: ds
+      shadowDataSourceName: shadow_ds
+  tables:
+    t_order:
+      dataSourceNames:
+        - shadowDataSource
+      shadowAlgorithmNames:
+        - user-id-insert-match-algorithm
+        - user-id-select-match-algorithm
+    t_order_item:
+      dataSourceNames:
+        - shadowDataSource
+      shadowAlgorithmNames:
+        - user-id-insert-match-algorithm
+        - user-id-update-match-algorithm
+        - user-id-select-match-algorithm
+    t_address:
+      dataSourceNames:
+        - shadowDataSource
+      shadowAlgorithmNames:
+        - user-id-insert-match-algorithm
+        - user-id-select-match-algorithm
+        - simple-hint-algorithm
+  shadowAlgorithms:
+    user-id-insert-match-algorithm:
+      type: REGEX_MATCH
+      props:
+        operation: insert
+        column: user_id
+        regex: "[1]"
+    user-id-update-match-algorithm:
+      type: REGEX_MATCH
+      props:
+        operation: update
+        column: user_id
+        regex: "[1]"
+    user-id-select-match-algorithm:
+      type: REGEX_MATCH
+      props:
+        operation: select
+        column: user_id
+        regex: "[1]"
+    simple-hint-algorithm:
+      type: SIMPLE_HINT
+      props:
+        foo: bar
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/conf/convert/config-sharding.yaml b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/conf/convert/config-sharding.yaml
index 53e780392c8..3e0f8624f78 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/conf/convert/config-sharding.yaml
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/conf/convert/config-sharding.yaml
@@ -64,6 +64,8 @@ rules:
        keyGeneratorName: snowflake
  bindingTables:
    - t_order,t_order_item
+ broadcastTables:
+   - t_order,t_order_item
  defaultDatabaseStrategy:
    standard:
      shardingColumn: user_id
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-encrypt.yaml b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-encrypt.yaml
index e94215291ad..65029b2ca8c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-encrypt.yaml
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-encrypt.yaml
@@ -30,6 +30,6 @@ PROPERTIES("minPoolSize"="1","connectionTimeoutMilliseconds"="30000","maxLifetim
 );
 CREATE ENCRYPT RULE t_encrypt (
 COLUMNS(
-(NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,TYPE(NAME='AES',PROPERTIES('aes-key-value'='123456abc'))),
+(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);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-encrypt.yaml b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-shadow.yaml
similarity index 52%
copy from shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-encrypt.yaml
copy to shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-shadow.yaml
index e94215291ad..bcf9a7d1009 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-encrypt.yaml
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-shadow.yaml
@@ -15,21 +15,23 @@
 # limitations under the License.
 #
 
-CREATE DATABASE encrypt_db;
-USE encrypt_db;
-ADD RESOURCE ds_0 (
-URL="jdbc:mysql://127.0.0.1:3306/demo_encrypt_ds_0?serverTimezone=UTC&useSSL=false",
+CREATE DATABASE shadow_db;
+USE shadow_db;
+ADD RESOURCE ds (
+URL="jdbc:mysql://127.0.0.1:3306/demo_shadow_ds_0?serverTimezone=UTC&useSSL=false",
 USER="root",
 PASSWORD="12345678",
 PROPERTIES("minPoolSize"="1","connectionTimeoutMilliseconds"="30000","maxLifetimeMilliseconds"="1800000","idleTimeoutMilliseconds"="60000","maxPoolSize"="50")
-), ds_1 (
-URL="jdbc:mysql://127.0.0.1:3306/demo_encrypt_ds_1?serverTimezone=UTC&useSSL=false",
+), shadow_ds (
+URL="jdbc:mysql://127.0.0.1:3306/demo_shadow_ds_1?serverTimezone=UTC&useSSL=false",
 USER="root",
 PASSWORD="12345678",
 PROPERTIES("minPoolSize"="1","connectionTimeoutMilliseconds"="30000","maxLifetimeMilliseconds"="1800000","idleTimeoutMilliseconds"="60000","maxPoolSize"="50")
 );
-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);
+CREATE SHADOW RULE shadowDataSource(
+SOURCE=ds,
+SHADOW=shadow_ds,
+t_order((TYPE(NAME="REGEX_MATCH", PROPERTIES("column"="user_id","operation"="insert","regex"="[1]"))),(TYPE(NAME="REGEX_MATCH", PROPERTIES("column"="user_id","operation"="select","regex"="[1]")))),
+t_order_item((TYPE(NAME="REGEX_MATCH", PROPERTIES("column"="user_id","operation"="insert","regex"="[1]"))),(TYPE(NAME="REGEX_MATCH", PROPERTIES("column"="user_id","operation"="update","regex"="[1]"))),(TYPE(NAME="REGEX_MATCH", PROPERTIES("column"="user_id","operation"="select","regex"="[1]")))),
+t_address((TYPE(NAME="REGEX_MATCH", PROPERTIES("column"="user_id","operation"="insert","regex"="[1]"))),(TYPE(NAME="REGEX_MATCH", PROPERTIES("column"="user_id","operation"="select","regex"="[1]"))),(TYPE(NAME="SIMPLE_HINT", PROPERTIES("foo"="bar"))))
+);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-sharding.yaml b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-sharding.yaml
index 232a6fcfdd9..2695b54e76a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-sharding.yaml
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-create-sharding.yaml
@@ -48,3 +48,4 @@ TABLE_STRATEGY(TYPE="complex", SHARDING_COLUMNS=order_id, user_id, SHARDING_ALGO
 KEY_GENERATE_STRATEGY(COLUMN=order_item_id, KEY_GENERATOR=snowflake)
 );
 CREATE SHARDING BINDING TABLE RULES (t_order,t_order_item);
+CREATE SHARDING BROADCAST TABLE RULES (t_order,t_order_item);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-database-discovery.yaml b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-database-discovery.yaml
index b53a1bfb8de..05c79442bf6 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-database-discovery.yaml
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/resources/expected/convert-database-discovery.yaml
@@ -35,6 +35,6 @@ PROPERTIES("minPoolSize"="1","connectionTimeoutMilliseconds"="3000","maxLifetime
 );
 CREATE DB_DISCOVERY RULE readwrite_ds (
 RESOURCES(ds_0,ds_1,ds_2),
-TYPE(NAME='MySQL.MGR', PROPERTIES('group-name'='92504d5b-6dec-11e8-91ea-246e9612aaf1')),
-HEARTBEAT(PROPERTIES('keep-alive-cron'='0/5 * * * * ?'))
+TYPE(NAME='MySQL.MGR', PROPERTIES("group-name"="92504d5b-6dec-11e8-91ea-246e9612aaf1")),
+HEARTBEAT(PROPERTIES("keep-alive-cron"="0/5 * * * * ?"))
 );