You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2021/08/30 09:15:43 UTC

[shardingsphere] branch master updated: Refactor shadow rule and modify shadow value generic type. (#12099)

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

panjuan 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 5371242  Refactor shadow rule and modify shadow value generic type. (#12099)
5371242 is described below

commit 5371242e661435bf57c2b975c0932e15a81a5aba
Author: gin <ja...@163.com>
AuthorDate: Mon Aug 30 17:15:12 2021 +0800

    Refactor shadow rule and modify shadow value generic type. (#12099)
    
    * Refactor the shadow rule add rule checker.
    
    * Add shadow rule test modify shadow value generic type.
---
 .../column/ColumnRegexMatchShadowAlgorithm.java    |   6 +-
 .../shardingsphere/shadow/rule/ShadowRule.java     |  91 +++++++++--
 .../shadow/rule/ShadowTableRule.java               |  35 +++++
 ...ithmProvidedShadowRuleConfigurationChecker.java |   2 +-
 .../rule/checker/ShadowTableRuleChecker.java       | 121 ++++++++++++++
 .../ColumnRegexMatchShadowAlgorithmTest.java       |   4 +-
 .../note/SimpleSQLNoteShadowAlgorithmTest.java     |   1 +
 .../shardingsphere/shadow/rule/ShadowRuleTest.java | 175 +++++++++++++++++++--
 .../rule/checker/ShadowTableRuleCheckerTest.java   | 106 +++++++++++++
 9 files changed, 507 insertions(+), 34 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithm.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithm.java
index e8efcfa..129d5de 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithm.java
@@ -34,7 +34,7 @@ import java.util.Properties;
  */
 @Getter
 @Setter
-public final class ColumnRegexMatchShadowAlgorithm implements ColumnShadowAlgorithm<String> {
+public final class ColumnRegexMatchShadowAlgorithm implements ColumnShadowAlgorithm<Comparable<?>> {
     
     private static final String COLUMN = "column";
     
@@ -81,11 +81,11 @@ public final class ColumnRegexMatchShadowAlgorithm implements ColumnShadowAlgori
     }
     
     @Override
-    public boolean isShadow(final Collection<String> shadowTableNames, final PreciseColumnShadowValue<String> shadowValue) {
+    public boolean isShadow(final Collection<String> shadowTableNames, final PreciseColumnShadowValue<Comparable<?>> shadowValue) {
         boolean containTable = shadowTableNames.contains(shadowValue.getLogicTableName());
         boolean isSameOperation = shadowOperationType == shadowValue.getShadowOperationType();
         boolean isSameColumnName = Objects.equals(props.get(COLUMN), shadowValue.getColumnName());
-        boolean isRegexMatch = shadowValue.getValue().matches(props.get(REGEX).toString());
+        boolean isRegexMatch = String.valueOf(shadowValue.getValue()).matches(props.get(REGEX).toString());
         return containTable && isSameOperation && isSameColumnName && isRegexMatch;
     }
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java
index 9c86b05..59f9183 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java
@@ -25,6 +25,7 @@ import org.apache.shardingsphere.shadow.algorithm.config.AlgorithmProvidedShadow
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
 import org.apache.shardingsphere.shadow.api.config.datasource.ShadowDataSourceConfiguration;
 import org.apache.shardingsphere.shadow.api.config.table.ShadowTableConfiguration;
+import org.apache.shardingsphere.shadow.rule.checker.ShadowTableRuleChecker;
 import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
 import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
 
@@ -32,8 +33,12 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
 
 /**
  * Databases shadow rule.
@@ -51,43 +56,95 @@ public final class ShadowRule implements FeatureRule, SchemaRule, DataSourceCont
     
     private final boolean enable;
     
-    private final Map<String, ShadowDataSourceConfiguration> dataSources = new LinkedHashMap<>();
-    
-    private final Map<String, ShadowTableConfiguration> tables = new LinkedHashMap<>();
+    private final Map<String, String> shadowDataSourceMappings = new LinkedHashMap<>();
     
     private final Map<String, ShadowAlgorithm> shadowAlgorithms = new LinkedHashMap<>();
     
+    private final Map<String, ShadowTableRule> shadowTableRules = new LinkedHashMap<>();
+    
     public ShadowRule(final ShadowRuleConfiguration shadowRuleConfig) {
+        enable = shadowRuleConfig.isEnable();
         column = shadowRuleConfig.getColumn();
         shadowMappings = new HashMap<>(shadowRuleConfig.getShadowDataSourceNames().size());
-        enable = shadowRuleConfig.isEnable();
         for (int i = 0; i < shadowRuleConfig.getSourceDataSourceNames().size(); i++) {
             shadowMappings.put(shadowRuleConfig.getSourceDataSourceNames().get(i), shadowRuleConfig.getShadowDataSourceNames().get(i));
         }
-        if (!shadowRuleConfig.getDataSources().isEmpty()) {
-            dataSources.putAll(shadowRuleConfig.getDataSources());
-        }
-        if (!shadowRuleConfig.getTables().isEmpty()) {
-            tables.putAll(shadowRuleConfig.getTables());
-        }
     }
     
     public ShadowRule(final AlgorithmProvidedShadowRuleConfiguration shadowRuleConfig) {
+        enable = shadowRuleConfig.isEnable();
         column = shadowRuleConfig.getColumn();
         shadowMappings = new HashMap<>(shadowRuleConfig.getShadowDataSourceNames().size());
-        enable = shadowRuleConfig.isEnable();
         for (int i = 0; i < shadowRuleConfig.getSourceDataSourceNames().size(); i++) {
             shadowMappings.put(shadowRuleConfig.getSourceDataSourceNames().get(i), shadowRuleConfig.getShadowDataSourceNames().get(i));
         }
-        if (!shadowRuleConfig.getDataSources().isEmpty()) {
-            dataSources.putAll(shadowRuleConfig.getDataSources());
+        if (enable) {
+            initShadowDataSourceMappings(shadowRuleConfig.getDataSources());
+            initShadowAlgorithms(shadowRuleConfig.getShadowAlgorithms());
+            initShadowTableRules(shadowRuleConfig.getTables(), shadowRuleConfig.getShadowAlgorithms());
         }
-        if (!shadowRuleConfig.getTables().isEmpty()) {
-            tables.putAll(shadowRuleConfig.getTables());
+    }
+    
+    private void initShadowTableRules(final Map<String, ShadowTableConfiguration> tables, final Map<String, ShadowAlgorithm> shadowAlgorithms) {
+        ShadowTableRuleChecker.checkShadowTables(tables);
+        tables.forEach((key, value) -> {
+            Collection<String> tableShadowAlgorithmNames = value.getShadowAlgorithmNames();
+            uselessShadowAlgorithmFilter(tableShadowAlgorithmNames, shadowAlgorithms);
+            ShadowTableRuleChecker.checkTableShadowAlgorithms(key, tableShadowAlgorithmNames, shadowAlgorithms);
+            shadowTableRules.put(key, new ShadowTableRule(key, tableShadowAlgorithmNames));
+        });
+    }
+    
+    private void uselessShadowAlgorithmFilter(final Collection<String> tableShadowAlgorithmNames, final Map<String, ShadowAlgorithm> shadowAlgorithms) {
+        tableShadowAlgorithmNames.removeIf(each -> Objects.isNull(shadowAlgorithms.get(each)));
+    }
+    
+    private void initShadowAlgorithms(final Map<String, ShadowAlgorithm> shadowAlgorithms) {
+        ShadowTableRuleChecker.checkShadowAlgorithms(shadowAlgorithms);
+        this.shadowAlgorithms.putAll(shadowAlgorithms);
+    }
+    
+    private void initShadowDataSourceMappings(final Map<String, ShadowDataSourceConfiguration> dataSources) {
+        ShadowTableRuleChecker.checkDataSources(dataSources);
+        dataSources.forEach((key, value) -> shadowDataSourceMappings.put(value.getSourceDataSourceName(), value.getShadowDataSourceName()));
+    }
+    
+    /**
+     * Get related shadow tables.
+     *
+     * @param tableNames table names
+     * @return related shadow tables
+     */
+    public Collection<String> getRelatedShadowTables(final Collection<String> tableNames) {
+        return tableNames.stream().filter(shadowTableRules.keySet()::contains).collect(Collectors.toCollection(LinkedList::new));
+    }
+    
+    /**
+     * Get all shadow table names.
+     *
+     * @return shadow table names
+     */
+    public Collection<String> getAllShadowTableNames() {
+        return shadowTableRules.keySet();
+    }
+    
+    /**
+     * Get related shadow algorithms by table name.
+     *
+     * @param tableName table name
+     * @return shadow algorithms
+     */
+    public Optional<Collection<ShadowAlgorithm>> getRelatedShadowAlgorithms(final String tableName) {
+        ShadowTableRule shadowTableRule = shadowTableRules.get(tableName);
+        if (Objects.isNull(shadowTableRule)) {
+            return Optional.empty();
         }
-        if (!shadowRuleConfig.getShadowAlgorithms().isEmpty()) {
-            shadowAlgorithms.putAll(shadowRuleConfig.getShadowAlgorithms());
+        Collection<String> shadowAlgorithmNames = shadowTableRule.getShadowAlgorithmNames();
+        Collection<ShadowAlgorithm> result = new LinkedList<>();
+        for (String each : shadowAlgorithmNames) {
+            result.add(shadowAlgorithms.get(each));
         }
+        return shadowAlgorithmNames.isEmpty() ? Optional.of(result) : Optional.of(result);
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowTableRule.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowTableRule.java
new file mode 100644
index 0000000..86bd16e
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowTableRule.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.shadow.rule;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+
+import java.util.Collection;
+
+/**
+ * Shadow table rule.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class ShadowTableRule {
+    
+    private final String logicTableName;
+    
+    private final Collection<String> shadowAlgorithmNames;
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/AlgorithmProvidedShadowRuleConfigurationChecker.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/AlgorithmProvidedShadowRuleConfigurationChecker.java
index 7c5e582..6ea3405 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/AlgorithmProvidedShadowRuleConfigurationChecker.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/AlgorithmProvidedShadowRuleConfigurationChecker.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.shadow.constant.ShadowOrder;
 /**
  * Algorithm provided shadow rule configuration checker.
  */
-public class AlgorithmProvidedShadowRuleConfigurationChecker extends AbstractShadowRuleConfigurationChecker<AlgorithmProvidedShadowRuleConfiguration> {
+public final class AlgorithmProvidedShadowRuleConfigurationChecker extends AbstractShadowRuleConfigurationChecker<AlgorithmProvidedShadowRuleConfiguration> {
     
     @Override
     public void check(final String schemaName, final AlgorithmProvidedShadowRuleConfiguration config) {
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowTableRuleChecker.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowTableRuleChecker.java
new file mode 100644
index 0000000..173b154
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowTableRuleChecker.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.shadow.rule.checker;
+
+import com.google.common.base.Preconditions;
+import org.apache.shardingsphere.shadow.api.config.datasource.ShadowDataSourceConfiguration;
+import org.apache.shardingsphere.shadow.api.config.table.ShadowTableConfiguration;
+import org.apache.shardingsphere.shadow.api.shadow.column.ColumnShadowAlgorithm;
+import org.apache.shardingsphere.shadow.api.shadow.column.ShadowOperationType;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Shadow table rule checker.
+ */
+public final class ShadowTableRuleChecker {
+    
+    /**
+     * Check data sources mappings size.
+     *
+     * @param dataSources data sources mappings
+     */
+    public static void checkDataSources(final Map<String, ShadowDataSourceConfiguration> dataSources) {
+        Preconditions.checkState(!dataSources.isEmpty(), "No available shadow data sources mappings in shadow configuration.");
+    }
+    
+    /**
+     * Check shadow tables size.
+     *
+     * @param shadowTables shadow tables
+     */
+    public static void checkShadowTables(final Map<String, ShadowTableConfiguration> shadowTables) {
+        Preconditions.checkState(!shadowTables.isEmpty(), "No available shadow tables in shadow configuration.");
+    }
+    
+    /**
+     * Check shadow algorithms size.
+     *
+     * @param shadowAlgorithms shadow algorithms
+     */
+    public static void checkShadowAlgorithms(final Map<String, ShadowAlgorithm> shadowAlgorithms) {
+        Preconditions.checkState(!shadowAlgorithms.isEmpty(), "No available shadow data algorithms in shadow configuration.");
+    }
+    
+    /**
+     * Check table shadow algorithms.
+     *
+     * @param tableName table name
+     * @param tableShadowAlgorithmNames table shadow algorithm names
+     * @param shadowAlgorithms shadow algorithms
+     */
+    public static void checkTableShadowAlgorithms(final String tableName, final Collection<String> tableShadowAlgorithmNames, final Map<String, ShadowAlgorithm> shadowAlgorithms) {
+        Preconditions.checkState(!tableShadowAlgorithmNames.isEmpty(), "No available shadow Algorithm configuration in shadow table `%s`.", tableName);
+        checkTableColumnShadowAlgorithms(tableName, createTableShadowAlgorithms(tableShadowAlgorithmNames, shadowAlgorithms));
+    }
+    
+    private static void checkTableColumnShadowAlgorithms(final String tableName, final Collection<ShadowAlgorithm> tableShadowAlgorithms) {
+        int insertOperationCount = 0;
+        int updateOperationCount = 0;
+        int deleteOperationCount = 0;
+        int selectOperationCount = 0;
+        for (ShadowAlgorithm each : tableShadowAlgorithms) {
+            if (each instanceof ColumnShadowAlgorithm) {
+                Optional<ShadowOperationType> shadowOperationTypeOptional = ShadowOperationType.contains(each.getProps().get("operation").toString());
+                if (shadowOperationTypeOptional.isPresent()) {
+                    ShadowOperationType shadowOperationType = shadowOperationTypeOptional.get();
+                    switch (shadowOperationType) {
+                        case INSERT:
+                            insertOperationCount++;
+                            checkOperationCount(tableName, shadowOperationType, insertOperationCount);
+                            break;
+                        case UPDATE:
+                            updateOperationCount++;
+                            checkOperationCount(tableName, shadowOperationType, updateOperationCount);
+                            break;
+                        case DELETE:
+                            deleteOperationCount++;
+                            checkOperationCount(tableName, shadowOperationType, deleteOperationCount);
+                            break;
+                        case SELECT:
+                            selectOperationCount++;
+                            checkOperationCount(tableName, shadowOperationType, selectOperationCount);
+                            break;
+                        default:
+                            break;
+                    }
+                }
+            }
+        }
+    }
+    
+    private static void checkOperationCount(final String tableName, final ShadowOperationType shadowOperationType, final int operationCount) {
+        Preconditions.checkState(operationCount <= 1, "Column shadow algorithm `%s` operation only supports one column mapping in shadow table `%s`.",
+                shadowOperationType.name(), tableName);
+    }
+    
+    private static Collection<ShadowAlgorithm> createTableShadowAlgorithms(final Collection<String> tableShadowAlgorithmNames, final Map<String, ShadowAlgorithm> shadowAlgorithms) {
+        return tableShadowAlgorithmNames.stream().map(shadowAlgorithms::get).filter(shadowAlgorithm -> !Objects.isNull(shadowAlgorithm)).collect(Collectors.toCollection(LinkedList::new));
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithmTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithmTest.java
index 38bba92..bf40598 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithmTest.java
@@ -66,8 +66,8 @@ public final class ColumnRegexMatchShadowAlgorithmTest {
         assertThat(shadowAlgorithm.isShadow(createTableNames(), createPreciseColumnShadowValue("t_order", ShadowOperationType.INSERT, "shadow", "1")), is(true));
     }
     
-    private PreciseColumnShadowValue<String> createPreciseColumnShadowValue(final String logicTableName, final ShadowOperationType shadowOperationType, final String columnName, final String value) {
-        return new PreciseColumnShadowValue<>(logicTableName, shadowOperationType, columnName, value);
+    private PreciseColumnShadowValue<Comparable<?>> createPreciseColumnShadowValue(final String tableName, final ShadowOperationType operationType, final String columnName, final String value) {
+        return new PreciseColumnShadowValue<>(tableName, operationType, columnName, value);
     }
     
     private Collection<String> createTableNames() {
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/note/SimpleSQLNoteShadowAlgorithmTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/note/SimpleSQLNoteShadowAlgorithmTest.java
index 4391a1f..4e5c911 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/note/SimpleSQLNoteShadowAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/note/SimpleSQLNoteShadowAlgorithmTest.java
@@ -36,6 +36,7 @@ public final class SimpleSQLNoteShadowAlgorithmTest {
     public void init() {
         shadowAlgorithm = new SimpleSQLNoteShadowAlgorithm();
         shadowAlgorithm.setProps(createProperties());
+        shadowAlgorithm.init();
     }
     
     private Properties createProperties() {
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/rule/ShadowRuleTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/rule/ShadowRuleTest.java
index b3431df..0bb3610 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/rule/ShadowRuleTest.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/rule/ShadowRuleTest.java
@@ -17,35 +17,188 @@
 
 package org.apache.shardingsphere.shadow.rule;
 
+import com.google.common.collect.Lists;
+import org.apache.shardingsphere.shadow.algorithm.config.AlgorithmProvidedShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.algorithm.shadow.column.ColumnRegexMatchShadowAlgorithm;
+import org.apache.shardingsphere.shadow.algorithm.shadow.note.SimpleSQLNoteShadowAlgorithm;
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.api.config.datasource.ShadowDataSourceConfiguration;
+import org.apache.shardingsphere.shadow.api.config.table.ShadowTableConfiguration;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 
 public final class ShadowRuleTest {
     
+    private ShadowRule shadowRuleWithAlgorithm;
+    
+    @Before
+    public void init() {
+        shadowRuleWithAlgorithm = new ShadowRule(createAlgorithmProvidedShadowRuleConfiguration());
+    }
+    
+    private AlgorithmProvidedShadowRuleConfiguration createAlgorithmProvidedShadowRuleConfiguration() {
+        AlgorithmProvidedShadowRuleConfiguration result = new AlgorithmProvidedShadowRuleConfiguration("shadow", Arrays.asList("ds", "ds1"), Arrays.asList("shadow_ds", "shadow_ds1"));
+        result.setEnable(true);
+        result.setDataSources(createDataSources());
+        result.setTables(createTables());
+        result.setShadowAlgorithms(createShadowAlgorithms());
+        return result;
+    }
+    
+    private Map<String, ShadowAlgorithm> createShadowAlgorithms() {
+        Map<String, ShadowAlgorithm> result = new LinkedHashMap<>();
+        result.put("simple-note-algorithm", createNoteShadowAlgorithm());
+        result.put("user-id-insert-regex-algorithm", createColumnShadowAlgorithm("user_id", "insert"));
+        result.put("user-id-update-regex-algorithm", createColumnShadowAlgorithm("user_id", "update"));
+        result.put("order-id-insert-regex-algorithm", createColumnShadowAlgorithm("order_id", "insert"));
+        return result;
+    }
+    
+    private ShadowAlgorithm createNoteShadowAlgorithm() {
+        SimpleSQLNoteShadowAlgorithm simpleSQLNoteShadowAlgorithm = new SimpleSQLNoteShadowAlgorithm();
+        simpleSQLNoteShadowAlgorithm.setProps(createNoteProperties());
+        simpleSQLNoteShadowAlgorithm.init();
+        return simpleSQLNoteShadowAlgorithm;
+    }
+    
+    private Properties createNoteProperties() {
+        Properties properties = new Properties();
+        properties.setProperty("shadow", "true");
+        return properties;
+    }
+    
+    private ShadowAlgorithm createColumnShadowAlgorithm(final String column, final String operation) {
+        ColumnRegexMatchShadowAlgorithm columnRegexMatchShadowAlgorithm = new ColumnRegexMatchShadowAlgorithm();
+        columnRegexMatchShadowAlgorithm.setProps(createColumnProperties(column, operation));
+        columnRegexMatchShadowAlgorithm.init();
+        return columnRegexMatchShadowAlgorithm;
+    }
+    
+    private Properties createColumnProperties(final String column, final String operation) {
+        Properties properties = new Properties();
+        properties.setProperty("column", column);
+        properties.setProperty("operation", operation);
+        properties.setProperty("regex", "[1]");
+        return properties;
+    }
+    
+    private Map<String, ShadowTableConfiguration> createTables() {
+        Map<String, ShadowTableConfiguration> result = new LinkedHashMap<>();
+        result.put("t_user", new ShadowTableConfiguration(createShadowAlgorithmNames("t_user")));
+        result.put("t_order", new ShadowTableConfiguration(createShadowAlgorithmNames("t_order")));
+        return result;
+    }
+    
+    private Collection<String> createShadowAlgorithmNames(final String tableName) {
+        Collection<String> result = new LinkedList<>();
+        result.add("simple-note-algorithm");
+        if ("t_user".equals(tableName)) {
+            result.add("user-id-insert-regex-algorithm");
+            result.add("user-id-update-regex-algorithm");
+        } else {
+            result.add("order-id-insert-regex-algorithm");
+        }
+        return result;
+    }
+    
+    private Map<String, ShadowDataSourceConfiguration> createDataSources() {
+        Map<String, ShadowDataSourceConfiguration> result = new LinkedHashMap<>();
+        result.put("ds-data-source", new ShadowDataSourceConfiguration("ds", "ds_shadow"));
+        result.put("ds1-data-source", new ShadowDataSourceConfiguration("ds1", "ds1_shadow"));
+        return result;
+    }
+    
+    @Test
+    public void assertNewShadowRulSuccessByShadowRuleConfiguration() {
+        ShadowRule shadowRule = new ShadowRule(new ShadowRuleConfiguration("shadow", Arrays.asList("ds", "ds1"), Arrays.asList("shadow_ds", "shadow_ds1")));
+        assertThat(shadowRule.isEnable(), is(false));
+        assertBasicShadowRule(shadowRule);
+    }
+    
+    private void assertBasicShadowRule(final ShadowRule shadowRule) {
+        assertThat(shadowRule.getColumn(), is("shadow"));
+        Map<String, String> shadowMappings = shadowRule.getShadowMappings();
+        assertThat(shadowMappings.get("ds"), is("shadow_ds"));
+        assertThat(shadowMappings.get("ds1"), is("shadow_ds1"));
+    }
+    
     @Test(expected = IllegalArgumentException.class)
-    public void assertNewWithEmptyDataSourceRule() {
-        new ShadowRule(new ShadowRuleConfiguration("", Collections.emptyList(), Collections.emptyList()));
+    public void assertNewShadowRuleByShadowRuleConfiguration() {
+        new ShadowRule(new ShadowRuleConfiguration("shadow", Collections.emptyList(), Collections.emptyList()));
+    }
+    
+    @Test
+    public void assertNewShadowRulSuccessByAlgorithmProvidedShadowRuleConfiguration() {
+        assertThat(shadowRuleWithAlgorithm.isEnable(), is(true));
+        assertBasicShadowRule(shadowRuleWithAlgorithm);
+        assertShadowDataSourceMappings(shadowRuleWithAlgorithm.getShadowDataSourceMappings());
+        assertShadowTableRules(shadowRuleWithAlgorithm.getShadowTableRules());
+    }
+    
+    private void assertShadowTableRules(final Map<String, ShadowTableRule> shadowTableRules) {
+        assertThat(shadowTableRules.size(), is(2));
+        shadowTableRules.forEach((key, value) -> assertShadowTableRule(key, value));
+    }
+    
+    private void assertShadowTableRule(final String tableName, final ShadowTableRule shadowTableRule) {
+        if ("t_user".equals(tableName)) {
+            assertThat(shadowTableRule.getShadowAlgorithmNames().size(), is(3));
+        } else {
+            assertThat(shadowTableRule.getShadowAlgorithmNames().size(), is(2));
+        }
+    }
+    
+    private void assertShadowDataSourceMappings(final Map<String, String> shadowDataSourceMappings) {
+        assertThat(shadowDataSourceMappings.get("ds"), is("ds_shadow"));
+        assertThat(shadowDataSourceMappings.get("ds1"), is("ds1_shadow"));
     }
     
     @Test
-    public void assertFindDataSourceRule() {
-        assertShadowRule(createShadowRule());
+    public void assertGetRelatedShadowTables() {
+        Collection<String> relatedShadowTables = shadowRuleWithAlgorithm.getRelatedShadowTables(Lists.newArrayList("t_user", "t_auto"));
+        assertThat(relatedShadowTables.size(), is(1));
+        assertThat(relatedShadowTables.iterator().next(), is("t_user"));
     }
     
-    private ShadowRule createShadowRule() {
-        return new ShadowRule(new ShadowRuleConfiguration("shadow", Arrays.asList("ds", "ds1"), Arrays.asList("shadow_ds", "shadow_ds1")));
+    @Test
+    public void assertGetAllShadowTableNames() {
+        Collection<String> allShadowTableNames = shadowRuleWithAlgorithm.getAllShadowTableNames();
+        assertThat(allShadowTableNames.size(), is(2));
+        Iterator<String> iterator = allShadowTableNames.iterator();
+        assertThat(iterator.next(), is("t_user"));
+        assertThat(iterator.next(), is("t_order"));
     }
     
-    private void assertShadowRule(final ShadowRule rule) {
-        assertThat(rule.getColumn(), is("shadow"));
-        assertThat(rule.getShadowMappings().size(), is(2));
-        assertThat(rule.getShadowMappings().get("ds"), is("shadow_ds"));
-        assertThat(rule.getShadowMappings().get("ds1"), is("shadow_ds1"));
+    @Test
+    public void assertGetRelatedShadowAlgorithms() {
+        Optional<Collection<ShadowAlgorithm>> shadowAlgorithmsOptional = shadowRuleWithAlgorithm.getRelatedShadowAlgorithms("t_user");
+        assertThat(shadowAlgorithmsOptional.isPresent(), is(true));
+        Collection<ShadowAlgorithm> shadowAlgorithms = shadowAlgorithmsOptional.get();
+        Iterator<ShadowAlgorithm> iterator = shadowAlgorithms.iterator();
+        ShadowAlgorithm shadowAlgorithm0 = iterator.next();
+        assertThat(shadowAlgorithm0.getType(), is("SIMPLE_NOTE"));
+        assertThat(shadowAlgorithm0.getProps().get("shadow"), is("true"));
+        ShadowAlgorithm shadowAlgorithm1 = iterator.next();
+        assertThat(shadowAlgorithm1.getType(), is("COLUMN_REGEX_MATCH"));
+        assertThat(shadowAlgorithm1.getProps().get("operation"), is("insert"));
+        assertThat(shadowAlgorithm1.getProps().get("column"), is("user_id"));
+        ShadowAlgorithm shadowAlgorithm2 = iterator.next();
+        assertThat(shadowAlgorithm2.getType(), is("COLUMN_REGEX_MATCH"));
+        assertThat(shadowAlgorithm2.getProps().get("operation"), is("update"));
+        assertThat(shadowAlgorithm2.getProps().get("column"), is("user_id"));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/rule/checker/ShadowTableRuleCheckerTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/rule/checker/ShadowTableRuleCheckerTest.java
new file mode 100644
index 0000000..d9242a0
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/rule/checker/ShadowTableRuleCheckerTest.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.shadow.rule.checker;
+
+import org.apache.shardingsphere.shadow.algorithm.shadow.column.ColumnRegexMatchShadowAlgorithm;
+import org.apache.shardingsphere.shadow.algorithm.shadow.note.SimpleSQLNoteShadowAlgorithm;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Properties;
+
+public final class ShadowTableRuleCheckerTest {
+    
+    @Test
+    public void assertCheckTableShadowAlgorithmsPass() {
+        Collection<String> tableShadowAlgorithmNames = createTableShadowAlgorithmNames();
+        Map<String, ShadowAlgorithm> shadowAlgorithms = createShadowAlgorithms(tableShadowAlgorithmNames);
+        ShadowTableRuleChecker.checkTableShadowAlgorithms("t_user", tableShadowAlgorithmNames, shadowAlgorithms);
+    }
+    
+    @Test(expected = IllegalStateException.class)
+    public void assertCheckTableShadowAlgorithmsFail() {
+        Collection<String> tableShadowAlgorithmNames = createTableShadowAlgorithmNames();
+        tableShadowAlgorithmNames.add("order-id-insert-regex-algorithm");
+        Map<String, ShadowAlgorithm> shadowAlgorithms = createShadowAlgorithms(tableShadowAlgorithmNames);
+        ShadowTableRuleChecker.checkTableShadowAlgorithms("t_user", tableShadowAlgorithmNames, shadowAlgorithms);
+    }
+    
+    private Map<String, ShadowAlgorithm> createShadowAlgorithms(final Collection<String> tableShadowAlgorithmNames) {
+        Map<String, ShadowAlgorithm> result = new LinkedHashMap<>();
+        for (String each : tableShadowAlgorithmNames) {
+            switch (each) {
+                case "user-id-insert-regex-algorithm":
+                    result.put(each, createColumnShadowAlgorithm("user_id", "insert"));
+                    break;
+                case "user-id-update-regex-algorithm":
+                    result.put(each, createColumnShadowAlgorithm("user_id", "update"));
+                    break;
+                case "order-id-insert-regex-algorithm":
+                    result.put(each, createColumnShadowAlgorithm("order_id", "insert"));
+                    break;
+                case "simple_note-algorithm":
+                    result.put(each, createNoteShadowAlgorithm());
+                    break;
+                default:
+                    break;
+            }
+        }
+        return result;
+    }
+    
+    private ShadowAlgorithm createNoteShadowAlgorithm() {
+        SimpleSQLNoteShadowAlgorithm simpleSQLNoteShadowAlgorithm = new SimpleSQLNoteShadowAlgorithm();
+        simpleSQLNoteShadowAlgorithm.setProps(createNoteProperties());
+        simpleSQLNoteShadowAlgorithm.init();
+        return simpleSQLNoteShadowAlgorithm;
+    }
+    
+    private Properties createNoteProperties() {
+        Properties properties = new Properties();
+        properties.setProperty("shadow", "true");
+        return properties;
+    }
+    
+    private ShadowAlgorithm createColumnShadowAlgorithm(final String column, final String operation) {
+        ColumnRegexMatchShadowAlgorithm columnRegexMatchShadowAlgorithm = new ColumnRegexMatchShadowAlgorithm();
+        columnRegexMatchShadowAlgorithm.setProps(createColumnProperties(column, operation));
+        columnRegexMatchShadowAlgorithm.init();
+        return columnRegexMatchShadowAlgorithm;
+    }
+    
+    private Properties createColumnProperties(final String column, final String operation) {
+        Properties properties = new Properties();
+        properties.setProperty("column", column);
+        properties.setProperty("operation", operation);
+        properties.setProperty("regex", "[1]");
+        return properties;
+    }
+    
+    private Collection<String> createTableShadowAlgorithmNames() {
+        Collection<String> result = new LinkedList<>();
+        result.add("user-id-insert-regex-algorithm");
+        result.add("user-id-update-regex-algorithm");
+        result.add("simple_note-algorithm");
+        return result;
+    }
+}