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 2022/02/07 11:21:53 UTC

[shardingsphere] branch master updated: Optimize shadow rule and shadow algorithm determiner (#15278)

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 3b36499  Optimize shadow rule and shadow algorithm determiner (#15278)
3b36499 is described below

commit 3b36499c3ddf43f7b2f661ccb191b7bd22123395
Author: gin <ja...@163.com>
AuthorDate: Mon Feb 7 19:20:50 2022 +0800

    Optimize shadow rule and shadow algorithm determiner (#15278)
---
 .../column/AbstractColumnMatchShadowAlgorithm.java |   7 +-
 .../ColumnShadowAlgorithmDeterminer.java           |  34 +++---
 .../HintShadowAlgorithmDeterminer.java             |  29 ++---
 .../determiner/ShadowAlgorithmDeterminer.java      |  36 -------
 .../engine/determiner/ShadowDeterminerFactory.java |  52 ---------
 .../dml/AbstractShadowDMLStatementRouteEngine.java |  67 ++++--------
 .../impl/ShadowNonDMLStatementRoutingEngine.java   |  18 +---
 .../shardingsphere/shadow/rule/ShadowRule.java     |  93 +++++++++++-----
 .../ColumnShadowAlgorithmDeterminerTest.java       |  63 +++++++++++
 .../HintShadowAlgorithmDeterminerTest.java         |  24 ++---
 .../determiner/ShadowDeterminerFactoryTest.java    |  37 -------
 .../ColumnShadowAlgorithmDeterminerTest.java       | 117 ---------------------
 12 files changed, 201 insertions(+), 376 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/AbstractColumnMatchShadowAlgorithm.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/AbstractColumnMatchShadowAlgorithm.java
index 6c12235..ab393bd 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/AbstractColumnMatchShadowAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/AbstractColumnMatchShadowAlgorithm.java
@@ -54,8 +54,11 @@ public abstract class AbstractColumnMatchShadowAlgorithm implements ColumnShadow
         String table = shadowValue.getLogicTableName();
         String column = shadowValue.getColumnName();
         Comparable<?> value = shadowValue.getValue();
-        SHADOW_VALUE_VALIDATORS.forEach(each -> each.preValidate(table, column, value));
-        return shadowOperationType == shadowValue.getShadowOperationType() && String.valueOf(props.get(COLUMN_PROPS_KEY)).equals(column) && isMatchValue(value);
+        if (shadowOperationType == shadowValue.getShadowOperationType() && String.valueOf(props.get(COLUMN_PROPS_KEY)).equals(column)) {
+            SHADOW_VALUE_VALIDATORS.forEach(each -> each.preValidate(table, column, value));
+            return isMatchValue(value);
+        }
+        return false;
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/algorithm/ColumnShadowAlgorithmDeterminer.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/ColumnShadowAlgorithmDeterminer.java
similarity index 65%
rename from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/algorithm/ColumnShadowAlgorithmDeterminer.java
rename to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/ColumnShadowAlgorithmDeterminer.java
index 083cd75..6443055 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/algorithm/ColumnShadowAlgorithmDeterminer.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/ColumnShadowAlgorithmDeterminer.java
@@ -15,16 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.shadow.route.engine.determiner.algorithm;
+package org.apache.shardingsphere.shadow.route.engine.determiner;
 
-import lombok.RequiredArgsConstructor;
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.shadow.api.shadow.ShadowOperationType;
 import org.apache.shardingsphere.shadow.api.shadow.column.ColumnShadowAlgorithm;
 import org.apache.shardingsphere.shadow.api.shadow.column.PreciseColumnShadowValue;
 import org.apache.shardingsphere.shadow.condition.ShadowColumnCondition;
 import org.apache.shardingsphere.shadow.condition.ShadowDetermineCondition;
-import org.apache.shardingsphere.shadow.route.engine.determiner.ShadowAlgorithmDeterminer;
-import org.apache.shardingsphere.shadow.rule.ShadowRule;
 
 import java.util.Collection;
 import java.util.LinkedList;
@@ -32,26 +31,29 @@ import java.util.LinkedList;
 /**
  * Column shadow algorithm determiner.
  */
-@RequiredArgsConstructor
-public final class ColumnShadowAlgorithmDeterminer implements ShadowAlgorithmDeterminer {
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class ColumnShadowAlgorithmDeterminer {
     
-    private final ColumnShadowAlgorithm<Comparable<?>> columnShadowAlgorithm;
-    
-    @Override
-    public boolean isShadow(final ShadowDetermineCondition shadowDetermineCondition, final ShadowRule shadowRule) {
-        return isShadowColumn(shadowDetermineCondition.getShadowColumnCondition(), shadowDetermineCondition.getTableName(), shadowDetermineCondition.getShadowOperationType());
-    }
-    
-    private boolean isShadowColumn(final ShadowColumnCondition shadowColumnCondition, final String tableName, final ShadowOperationType operationType) {
+    /**
+     * Is shadow in column shadow algorithm.
+     *
+     * @param shadowAlgorithm column shadow algorithm
+     * @param shadowCondition shadow determine condition
+     * @return is shadow or not
+     */
+    public static boolean isShadow(final ColumnShadowAlgorithm<Comparable<?>> shadowAlgorithm, final ShadowDetermineCondition shadowCondition) {
+        ShadowColumnCondition shadowColumnCondition = shadowCondition.getShadowColumnCondition();
+        String tableName = shadowCondition.getTableName();
+        ShadowOperationType operationType = shadowCondition.getShadowOperationType();
         for (PreciseColumnShadowValue<Comparable<?>> each : createColumnShadowValues(shadowColumnCondition.getColumn(), shadowColumnCondition.getValues(), tableName, operationType)) {
-            if (!tableName.equals(shadowColumnCondition.getOwner()) || !columnShadowAlgorithm.isShadow(each)) {
+            if (!tableName.equals(shadowColumnCondition.getOwner()) || !shadowAlgorithm.isShadow(each)) {
                 return false;
             }
         }
         return true;
     }
     
-    private Collection<PreciseColumnShadowValue<Comparable<?>>> createColumnShadowValues(final String columnName, final Collection<Comparable<?>> columnValues, final String tableName,
+    private static Collection<PreciseColumnShadowValue<Comparable<?>>> createColumnShadowValues(final String columnName, final Collection<Comparable<?>> columnValues, final String tableName,
                                                                                          final ShadowOperationType operationType) {
         Collection<PreciseColumnShadowValue<Comparable<?>>> result = new LinkedList<>();
         for (Comparable<?> each : columnValues) {
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/algorithm/HintShadowAlgorithmDeterminer.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminer.java
similarity index 70%
rename from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/algorithm/HintShadowAlgorithmDeterminer.java
rename to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminer.java
index 9bde6cf..4816338 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/algorithm/HintShadowAlgorithmDeterminer.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminer.java
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.shadow.route.engine.determiner.algorithm;
+package org.apache.shardingsphere.shadow.route.engine.determiner;
 
-import lombok.RequiredArgsConstructor;
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.shadow.api.shadow.ShadowOperationType;
 import org.apache.shardingsphere.shadow.api.shadow.hint.HintShadowAlgorithm;
 import org.apache.shardingsphere.shadow.api.shadow.hint.PreciseHintShadowValue;
 import org.apache.shardingsphere.shadow.condition.ShadowDetermineCondition;
-import org.apache.shardingsphere.shadow.route.engine.determiner.ShadowAlgorithmDeterminer;
 import org.apache.shardingsphere.shadow.rule.ShadowRule;
 
 import java.util.Collection;
@@ -32,23 +32,28 @@ import java.util.stream.Collectors;
 /**
  * Hint shadow algorithm determiner.
  */
-@RequiredArgsConstructor
-public final class HintShadowAlgorithmDeterminer implements ShadowAlgorithmDeterminer {
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class HintShadowAlgorithmDeterminer {
     
-    private final HintShadowAlgorithm<Comparable<?>> hintShadowAlgorithm;
-    
-    @Override
-    public boolean isShadow(final ShadowDetermineCondition shadowDetermineCondition, final ShadowRule shadowRule) {
-        Collection<PreciseHintShadowValue<Comparable<?>>> noteShadowValues = createNoteShadowValues(shadowDetermineCondition);
+    /**
+     * Is shadow in hint shadow algorithm.
+     *
+     * @param shadowAlgorithm hint shadow algorithm
+     * @param shadowCondition shadow determine condition
+     * @param shadowRule shadow rule
+     * @return is shadow or not
+     */
+    public static boolean isShadow(final HintShadowAlgorithm<Comparable<?>> shadowAlgorithm, final ShadowDetermineCondition shadowCondition, final ShadowRule shadowRule) {
+        Collection<PreciseHintShadowValue<Comparable<?>>> noteShadowValues = createNoteShadowValues(shadowCondition);
         for (PreciseHintShadowValue<Comparable<?>> each : noteShadowValues) {
-            if (hintShadowAlgorithm.isShadow(shadowRule.getAllShadowTableNames(), each)) {
+            if (shadowAlgorithm.isShadow(shadowRule.getAllShadowTableNames(), each)) {
                 return true;
             }
         }
         return false;
     }
     
-    private Collection<PreciseHintShadowValue<Comparable<?>>> createNoteShadowValues(final ShadowDetermineCondition shadowDetermineCondition) {
+    private static Collection<PreciseHintShadowValue<Comparable<?>>> createNoteShadowValues(final ShadowDetermineCondition shadowDetermineCondition) {
         ShadowOperationType shadowOperationType = shadowDetermineCondition.getShadowOperationType();
         String tableName = shadowDetermineCondition.getTableName();
         return shadowDetermineCondition.getSqlComments().stream().<PreciseHintShadowValue<Comparable<?>>>map(each -> new PreciseHintShadowValue<>(tableName, shadowOperationType, each))
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/ShadowAlgorithmDeterminer.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/ShadowAlgorithmDeterminer.java
deleted file mode 100644
index a4ab776..0000000
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/ShadowAlgorithmDeterminer.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 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.route.engine.determiner;
-
-import org.apache.shardingsphere.shadow.condition.ShadowDetermineCondition;
-import org.apache.shardingsphere.shadow.rule.ShadowRule;
-
-/**
- * Shadow algorithm determiner.
- */
-public interface ShadowAlgorithmDeterminer {
-    
-    /**
-     * Is shadow in shadow algorithm.
-     *
-     * @param shadowDetermineCondition shadow determine condition
-     * @param shadowRule shadow rule
-     * @return is shadow or not
-     */
-    boolean isShadow(ShadowDetermineCondition shadowDetermineCondition, ShadowRule shadowRule);
-}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/ShadowDeterminerFactory.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/ShadowDeterminerFactory.java
deleted file mode 100644
index 17495e6..0000000
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/ShadowDeterminerFactory.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.route.engine.determiner;
-
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.shadow.api.shadow.column.ColumnShadowAlgorithm;
-import org.apache.shardingsphere.shadow.api.shadow.hint.HintShadowAlgorithm;
-import org.apache.shardingsphere.shadow.route.engine.determiner.algorithm.ColumnShadowAlgorithmDeterminer;
-import org.apache.shardingsphere.shadow.route.engine.determiner.algorithm.HintShadowAlgorithmDeterminer;
-
-/**
- * Shadow determiner factory.
- */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class ShadowDeterminerFactory {
-    
-    /**
-     * Create new instance of Shadow algorithm determiner.
-     *
-     * @param hintShadowAlgorithm hint shadow algorithm
-     * @return new instance of Shadow algorithm determiner
-     */
-    public static ShadowAlgorithmDeterminer newInstance(final HintShadowAlgorithm<Comparable<?>> hintShadowAlgorithm) {
-        return new HintShadowAlgorithmDeterminer(hintShadowAlgorithm);
-    }
-    
-    /**
-     * Create new instance of Shadow algorithm determiner.
-     *
-     * @param columnShadowAlgorithm column shadow algorithm
-     * @return new instance of Shadow algorithm determiner
-     */
-    public static ShadowAlgorithmDeterminer newInstance(final ColumnShadowAlgorithm<Comparable<?>> columnShadowAlgorithm) {
-        return new ColumnShadowAlgorithmDeterminer(columnShadowAlgorithm);
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/dml/AbstractShadowDMLStatementRouteEngine.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/dml/AbstractShadowDMLStatementRouteEngine.java
index c1afcda..bda7b3c 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/dml/AbstractShadowDMLStatementRouteEngine.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/dml/AbstractShadowDMLStatementRouteEngine.java
@@ -25,7 +25,8 @@ import org.apache.shardingsphere.shadow.api.shadow.hint.HintShadowAlgorithm;
 import org.apache.shardingsphere.shadow.condition.ShadowColumnCondition;
 import org.apache.shardingsphere.shadow.condition.ShadowDetermineCondition;
 import org.apache.shardingsphere.shadow.route.engine.ShadowRouteEngine;
-import org.apache.shardingsphere.shadow.route.engine.determiner.ShadowDeterminerFactory;
+import org.apache.shardingsphere.shadow.route.engine.determiner.ColumnShadowAlgorithmDeterminer;
+import org.apache.shardingsphere.shadow.route.engine.determiner.HintShadowAlgorithmDeterminer;
 import org.apache.shardingsphere.shadow.rule.ShadowRule;
 import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
@@ -85,7 +86,7 @@ public abstract class AbstractShadowDMLStatementRouteEngine implements ShadowRou
             ShadowAlgorithm shadowAlgorithm = defaultShadowAlgorithm.get();
             if (shadowAlgorithm instanceof HintShadowAlgorithm<?>) {
                 ShadowDetermineCondition shadowDetermineCondition = new ShadowDetermineCondition("", ShadowOperationType.HINT_MATCH);
-                return isMatchHintShadowAlgorithm((HintShadowAlgorithm<Comparable<?>>) shadowAlgorithm, shadowDetermineCondition.initSQLComments(sqlComments.get()), shadowRule);
+                return HintShadowAlgorithmDeterminer.isShadow((HintShadowAlgorithm<Comparable<?>>) shadowAlgorithm, shadowDetermineCondition.initSQLComments(sqlComments.get()), shadowRule);
             }
         }
         return false;
@@ -103,72 +104,48 @@ public abstract class AbstractShadowDMLStatementRouteEngine implements ShadowRou
     }
     
     private boolean isContainsShadowInSQLComments(final String tableName, final ShadowRule shadowRule, final ShadowDetermineCondition shadowCondition) {
-        return parseSQLComments().filter(SQLComments -> shadowRule.getRelatedHintShadowAlgorithms(tableName)
-                .filter(shadowAlgorithms -> isMatchAnyHintShadowAlgorithms(shadowAlgorithms, shadowCondition.initSQLComments(SQLComments), shadowRule)).isPresent()).isPresent();
+        return parseSQLComments()
+                .filter(SQLComments -> isMatchAnyHintShadowAlgorithms(shadowRule.getRelatedHintShadowAlgorithms(tableName), shadowCondition.initSQLComments(SQLComments), shadowRule)).isPresent();
     }
     
     private boolean isMatchAnyHintShadowAlgorithms(final Collection<HintShadowAlgorithm<Comparable<?>>> shadowAlgorithms, final ShadowDetermineCondition shadowCondition, final ShadowRule shadowRule) {
         for (HintShadowAlgorithm<Comparable<?>> each : shadowAlgorithms) {
-            if (isMatchHintShadowAlgorithm(each, shadowCondition, shadowRule)) {
+            if (HintShadowAlgorithmDeterminer.isShadow(each, shadowCondition, shadowRule)) {
                 return true;
             }
         }
         return false;
     }
     
-    private boolean isMatchHintShadowAlgorithm(final HintShadowAlgorithm<Comparable<?>> hintShadowAlgorithm, final ShadowDetermineCondition shadowCondition, final ShadowRule shadowRule) {
-        return ShadowDeterminerFactory.newInstance(hintShadowAlgorithm).isShadow(shadowCondition, shadowRule);
-    }
-    
     private Map<String, String> findByShadowColumn(final Collection<String> relatedShadowTables, final ShadowRule shadowRule, final ShadowOperationType shadowOperationType) {
         Map<String, String> result = new LinkedHashMap<>();
-        Iterator<Optional<ShadowColumnCondition>> iterator = getShadowColumnConditionIterator();
-        while (iterator.hasNext()) {
-            Optional<ShadowColumnCondition> next = iterator.next();
-            if (!next.isPresent()) {
-                continue;
-            }
-            Optional<String> shadowTable = findShadowTableByShadowColumn(relatedShadowTables, shadowRule, next.get(), shadowOperationType);
-            if (!shadowTable.isPresent()) {
-                continue;
+        for (String each : relatedShadowTables) {
+            Collection<ColumnShadowAlgorithm<Comparable<?>>> columnShadowAlgorithms = shadowRule.getRelatedColumnShadowAlgorithms(each, shadowOperationType);
+            if (!columnShadowAlgorithms.isEmpty() && isMatchAnyColumnShadowAlgorithms(each, columnShadowAlgorithms, shadowRule, shadowOperationType)) {
+                return shadowRule.getRelatedShadowDataSourceMappings(each);
             }
-            result.putAll(shadowRule.getRelatedShadowDataSourceMappings(shadowTable.get()));
-            return result;
         }
         return result;
     }
     
-    private Optional<String> findShadowTableByShadowColumn(final Collection<String> relatedShadowTables, final ShadowRule shadowRule, final ShadowColumnCondition shadowColumnCondition,
-                                                           final ShadowOperationType shadowOperationType) {
+    private boolean isMatchAnyColumnShadowAlgorithms(final String shadowTable, final Collection<ColumnShadowAlgorithm<Comparable<?>>> columnShadowAlgorithms, final ShadowRule shadowRule,
+                                                        final ShadowOperationType shadowOperationType) {
+        Iterator<Optional<ShadowColumnCondition>> iterator = getShadowColumnConditionIterator();
         ShadowDetermineCondition shadowDetermineCondition;
-        for (String each : relatedShadowTables) {
-            shadowDetermineCondition = new ShadowDetermineCondition(each, shadowOperationType);
-            if (isContainsShadowInColumn(each, shadowRule, shadowDetermineCondition.initShadowColumnCondition(shadowColumnCondition))) {
-                return Optional.of(each);
-            }
-        }
-        return Optional.empty();
-    }
-    
-    private boolean isContainsShadowInColumn(final String tableName, final ShadowRule shadowRule, final ShadowDetermineCondition shadowCondition) {
-        Optional<Collection<ColumnShadowAlgorithm<Comparable<?>>>> relatedColumnShadowAlgorithms = shadowRule.getRelatedColumnShadowAlgorithms(tableName, shadowCondition.getShadowOperationType());
-        return relatedColumnShadowAlgorithms.isPresent() && isMatchAnyColumnShadowAlgorithms(relatedColumnShadowAlgorithms.get(), shadowCondition, shadowRule);
-    }
-    
-    private boolean isMatchAnyColumnShadowAlgorithms(final Collection<ColumnShadowAlgorithm<Comparable<?>>> shadowAlgorithms, final ShadowDetermineCondition shadowCondition,
-                                                     final ShadowRule shadowRule) {
-        for (ColumnShadowAlgorithm<Comparable<?>> each : shadowAlgorithms) {
-            if (isMatchColumnShadowAlgorithm(each, shadowCondition, shadowRule)) {
-                return true;
+        while (iterator.hasNext()) {
+            Optional<ShadowColumnCondition> next = iterator.next();
+            if (next.isPresent()) {
+                for (ColumnShadowAlgorithm<Comparable<?>> each : columnShadowAlgorithms) {
+                    shadowDetermineCondition = new ShadowDetermineCondition(shadowTable, shadowOperationType);
+                    if (ColumnShadowAlgorithmDeterminer.isShadow(each, shadowDetermineCondition.initShadowColumnCondition(next.get()))) {
+                        return true;
+                    }
+                }
             }
         }
         return false;
     }
     
-    private boolean isMatchColumnShadowAlgorithm(final ColumnShadowAlgorithm<Comparable<?>> columnShadowAlgorithm, final ShadowDetermineCondition shadowCondition, final ShadowRule shadowRule) {
-        return ShadowDeterminerFactory.newInstance(columnShadowAlgorithm).isShadow(shadowCondition, shadowRule);
-    }
-    
     /**
      * Get all tables.
      *
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/impl/ShadowNonDMLStatementRoutingEngine.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/impl/ShadowNonDMLStatementRoutingEngine.java
index 30f14dc..cf216d9 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/impl/ShadowNonDMLStatementRoutingEngine.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/impl/ShadowNonDMLStatementRoutingEngine.java
@@ -24,7 +24,7 @@ import org.apache.shardingsphere.shadow.api.shadow.ShadowOperationType;
 import org.apache.shardingsphere.shadow.api.shadow.hint.HintShadowAlgorithm;
 import org.apache.shardingsphere.shadow.condition.ShadowDetermineCondition;
 import org.apache.shardingsphere.shadow.route.engine.ShadowRouteEngine;
-import org.apache.shardingsphere.shadow.route.engine.determiner.ShadowDeterminerFactory;
+import org.apache.shardingsphere.shadow.route.engine.determiner.HintShadowAlgorithmDeterminer;
 import org.apache.shardingsphere.shadow.rule.ShadowRule;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.CommentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.AbstractSQLStatement;
@@ -55,11 +55,7 @@ public final class ShadowNonDMLStatementRoutingEngine implements ShadowRouteEngi
         if (!sqlComments.isPresent()) {
             return result;
         }
-        Optional<Collection<HintShadowAlgorithm<Comparable<?>>>> noteShadowAlgorithms = shadowRule.getAllHintShadowAlgorithms();
-        if (!noteShadowAlgorithms.isPresent()) {
-            return result;
-        }
-        if (isMatchAnyNoteShadowAlgorithms(noteShadowAlgorithms.get(), createShadowDetermineCondition(sqlComments.get()), shadowRule)) {
+        if (isMatchAnyNoteShadowAlgorithms(shadowRule, createShadowDetermineCondition(sqlComments.get()))) {
             return shadowRule.getAllShadowDataSourceMappings();
         }
         return result;
@@ -76,16 +72,12 @@ public final class ShadowNonDMLStatementRoutingEngine implements ShadowRouteEngi
         return result.initSQLComments(sqlComments);
     }
     
-    private boolean isMatchAnyNoteShadowAlgorithms(final Collection<HintShadowAlgorithm<Comparable<?>>> shadowAlgorithms, final ShadowDetermineCondition shadowCondition, final ShadowRule shadowRule) {
-        for (HintShadowAlgorithm<Comparable<?>> each : shadowAlgorithms) {
-            if (isMatchNoteShadowAlgorithm(each, shadowCondition, shadowRule)) {
+    private boolean isMatchAnyNoteShadowAlgorithms(final ShadowRule shadowRule, final ShadowDetermineCondition shadowCondition) {
+        for (HintShadowAlgorithm<Comparable<?>> each : shadowRule.getAllHintShadowAlgorithms()) {
+            if (HintShadowAlgorithmDeterminer.isShadow(each, shadowCondition, shadowRule)) {
                 return true;
             }
         }
         return false;
     }
-    
-    private boolean isMatchNoteShadowAlgorithm(final HintShadowAlgorithm<Comparable<?>> hintShadowAlgorithm, final ShadowDetermineCondition shadowCondition, final ShadowRule shadowRule) {
-        return ShadowDeterminerFactory.newInstance(hintShadowAlgorithm).isShadow(shadowCondition, shadowRule);
-    }
 }
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 a8b03b9..1de3a9a 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
@@ -38,7 +38,6 @@ import java.util.LinkedList;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
-import java.util.stream.Collectors;
 
 /**
  * Databases shadow rule.
@@ -52,6 +51,8 @@ public final class ShadowRule implements SchemaRule, DataSourceContainedRule {
     
     private ShadowAlgorithm defaultShadowAlgorithm;
     
+    private final Collection<String> hintShadowAlgorithmNames = new LinkedList<>();
+    
     private final Map<String, ShadowDataSourceRule> shadowDataSourceMappings = new LinkedHashMap<>();
     
     private final Map<String, ShadowAlgorithm> shadowAlgorithms = new LinkedHashMap<>();
@@ -77,15 +78,30 @@ public final class ShadowRule implements SchemaRule, DataSourceContainedRule {
     }
     
     private void initShadowAlgorithmConfigurations(final Map<String, ShardingSphereAlgorithmConfiguration> shadowAlgorithmConfigurations) {
-        shadowAlgorithmConfigurations.forEach((key, value) -> shadowAlgorithms.put(key, ShardingSphereAlgorithmFactory.createAlgorithm(value, ShadowAlgorithm.class)));
+        shadowAlgorithmConfigurations.forEach((key, value) -> {
+            ShadowAlgorithm algorithm = ShardingSphereAlgorithmFactory.createAlgorithm(value, ShadowAlgorithm.class);
+            if (algorithm instanceof HintShadowAlgorithm<?>) {
+                hintShadowAlgorithmNames.add(key);
+            }
+            shadowAlgorithms.put(key, algorithm);
+        });
     }
     
     private void initShadowAlgorithms(final Map<String, ShadowAlgorithm> shadowAlgorithms) {
-        this.shadowAlgorithms.putAll(shadowAlgorithms);
+        shadowAlgorithms.forEach((key, value) -> {
+            if (value instanceof HintShadowAlgorithm<?>) {
+                hintShadowAlgorithmNames.add(key);
+            }
+            this.shadowAlgorithms.put(key, value);
+        });
     }
     
     private void initDefaultShadowAlgorithm(final String defaultShadowAlgorithmName) {
-        defaultShadowAlgorithm = shadowAlgorithms.get(defaultShadowAlgorithmName);
+        ShadowAlgorithm shadowAlgorithm = shadowAlgorithms.get(defaultShadowAlgorithmName);
+        if (shadowAlgorithm instanceof HintShadowAlgorithm<?>) {
+            hintShadowAlgorithmNames.add(defaultShadowAlgorithmName);
+        }
+        defaultShadowAlgorithm = shadowAlgorithm;
     }
     
     private void initShadowTableRules(final Map<String, ShadowTableConfiguration> tables) {
@@ -108,7 +124,13 @@ public final class ShadowRule implements SchemaRule, DataSourceContainedRule {
      * @return related shadow tables
      */
     public Collection<String> getRelatedShadowTables(final Collection<String> tableNames) {
-        return tableNames.stream().filter(shadowTableRules.keySet()::contains).collect(Collectors.toCollection(LinkedList::new));
+        Collection<String> result = new LinkedList<>();
+        for (String each : tableNames) {
+            if (shadowTableRules.containsKey(each)) {
+                result.add(each);
+            }
+        }
+        return result;
     }
     
     /**
@@ -126,13 +148,12 @@ public final class ShadowRule implements SchemaRule, DataSourceContainedRule {
      * @return related hint shadow algorithms
      */
     @SuppressWarnings("unchecked")
-    public Optional<Collection<HintShadowAlgorithm<Comparable<?>>>> getAllHintShadowAlgorithms() {
-        Collection<HintShadowAlgorithm<Comparable<?>>> result = shadowAlgorithms.values().stream().filter(each -> each instanceof HintShadowAlgorithm<?>)
-                .map(each -> (HintShadowAlgorithm<Comparable<?>>) each).collect(Collectors.toCollection(LinkedList::new));
-        if (defaultShadowAlgorithm instanceof HintShadowAlgorithm<?>) {
-            result.add((HintShadowAlgorithm<Comparable<?>>) defaultShadowAlgorithm);
+    public Collection<HintShadowAlgorithm<Comparable<?>>> getAllHintShadowAlgorithms() {
+        Collection<HintShadowAlgorithm<Comparable<?>>> result = new LinkedList<>();
+        for (String each : hintShadowAlgorithmNames) {
+            result.add((HintShadowAlgorithm<Comparable<?>>) shadowAlgorithms.get(each));
         }
-        return result.isEmpty() ? Optional.empty() : Optional.of(result);
+        return result;
     }
     
     /**
@@ -142,11 +163,13 @@ public final class ShadowRule implements SchemaRule, DataSourceContainedRule {
      * @return hint shadow algorithms
      */
     @SuppressWarnings("unchecked")
-    public Optional<Collection<HintShadowAlgorithm<Comparable<?>>>> getRelatedHintShadowAlgorithms(final String tableName) {
-        Collection<HintShadowAlgorithm<Comparable<?>>> result = shadowTableRules.get(tableName).getHintShadowAlgorithmNames().stream().map(shadowAlgorithms::get)
-                .filter(shadowAlgorithm -> !Objects.isNull(shadowAlgorithm)).map(shadowAlgorithm -> (HintShadowAlgorithm<Comparable<?>>) shadowAlgorithm)
-                .collect(Collectors.toCollection(LinkedList::new));
-        return result.isEmpty() ? Optional.of(result) : Optional.of(result);
+    public Collection<HintShadowAlgorithm<Comparable<?>>> getRelatedHintShadowAlgorithms(final String tableName) {
+        Collection<HintShadowAlgorithm<Comparable<?>>> result = new LinkedList<>();
+        Collection<String> hintShadowAlgorithmNames = shadowTableRules.get(tableName).getHintShadowAlgorithmNames();
+        for (String each : hintShadowAlgorithmNames) {
+            result.add((HintShadowAlgorithm<Comparable<?>>) shadowAlgorithms.get(each));
+        }
+        return result;
     }
     
     /**
@@ -157,11 +180,17 @@ public final class ShadowRule implements SchemaRule, DataSourceContainedRule {
      * @return column shadow algorithms
      */
     @SuppressWarnings("unchecked")
-    public Optional<Collection<ColumnShadowAlgorithm<Comparable<?>>>> getRelatedColumnShadowAlgorithms(final String tableName, final ShadowOperationType shadowOperationType) {
-        Collection<ColumnShadowAlgorithm<Comparable<?>>> result = shadowTableRules.get(tableName).getColumnShadowAlgorithmNames().get(shadowOperationType).stream().map(shadowAlgorithms::get)
-                .filter(shadowAlgorithm -> !Objects.isNull(shadowAlgorithm)).map(shadowAlgorithm -> (ColumnShadowAlgorithm<Comparable<?>>) shadowAlgorithm)
-                .collect(Collectors.toCollection(LinkedList::new));
-        return result.isEmpty() ? Optional.of(result) : Optional.of(result);
+    public Collection<ColumnShadowAlgorithm<Comparable<?>>> getRelatedColumnShadowAlgorithms(final String tableName, final ShadowOperationType shadowOperationType) {
+        Collection<ColumnShadowAlgorithm<Comparable<?>>> result = new LinkedList<>();
+        Map<ShadowOperationType, Collection<String>> columnShadowAlgorithmNames = shadowTableRules.get(tableName).getColumnShadowAlgorithmNames();
+        Collection<String> names = columnShadowAlgorithmNames.get(shadowOperationType);
+        if (Objects.isNull(names)) {
+            return result;
+        }
+        for (String each : names) {
+            result.add((ColumnShadowAlgorithm<Comparable<?>>) shadowAlgorithms.get(each));
+        }
+        return result;
     }
     
     /**
@@ -171,8 +200,13 @@ public final class ShadowRule implements SchemaRule, DataSourceContainedRule {
      * @return shadow data source rules
      */
     public Map<String, String> getRelatedShadowDataSourceMappings(final String tableName) {
-        return shadowTableRules.get(tableName).getShadowDataSources().stream().map(shadowDataSourceMappings::get).filter(Objects::nonNull)
-                .collect(Collectors.toMap(ShadowDataSourceRule::getSourceDataSource, ShadowDataSourceRule::getShadowDataSource, (a, b) -> b, LinkedHashMap::new));
+        Map<String, String> result = new LinkedHashMap<>();
+        Collection<String> shadowDataSources = shadowTableRules.get(tableName).getShadowDataSources();
+        for (String each : shadowDataSources) {
+            ShadowDataSourceRule shadowDataSourceRule = shadowDataSourceMappings.get(each);
+            result.put(shadowDataSourceRule.getSourceDataSource(), shadowDataSourceRule.getShadowDataSource());
+        }
+        return result;
     }
     
     /**
@@ -181,8 +215,12 @@ public final class ShadowRule implements SchemaRule, DataSourceContainedRule {
      * @return all shadow data source mappings
      */
     public Map<String, String> getAllShadowDataSourceMappings() {
-        return shadowDataSourceMappings.values().stream()
-                .collect(Collectors.toMap(ShadowDataSourceRule::getSourceDataSource, ShadowDataSourceRule::getShadowDataSource, (a, b) -> b, LinkedHashMap::new));
+        Map<String, String> result = new LinkedHashMap<>();
+        for (Map.Entry<String, ShadowDataSourceRule> entry : shadowDataSourceMappings.entrySet()) {
+            ShadowDataSourceRule rule = entry.getValue();
+            result.put(rule.getSourceDataSource(), rule.getShadowDataSource());
+        }
+        return result;
     }
     
     /**
@@ -193,10 +231,7 @@ public final class ShadowRule implements SchemaRule, DataSourceContainedRule {
      */
     public Optional<String> getSourceDataSourceName(final String actualDataSourceName) {
         ShadowDataSourceRule shadowDataSourceRule = shadowDataSourceMappings.get(actualDataSourceName);
-        if (null != shadowDataSourceRule) {
-            return Optional.of(shadowDataSourceRule.getSourceDataSource());
-        }
-        return Optional.empty();
+        return shadowDataSourceRule == null ? Optional.empty() : Optional.of(shadowDataSourceRule.getSourceDataSource());
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/ColumnShadowAlgorithmDeterminerTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/ColumnShadowAlgorithmDeterminerTest.java
new file mode 100644
index 0000000..dc02b12
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/ColumnShadowAlgorithmDeterminerTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.route.engine.determiner;
+
+import org.apache.shardingsphere.shadow.algorithm.shadow.column.ColumnRegexMatchShadowAlgorithm;
+import org.apache.shardingsphere.shadow.api.shadow.ShadowOperationType;
+import org.apache.shardingsphere.shadow.api.shadow.column.ColumnShadowAlgorithm;
+import org.apache.shardingsphere.shadow.condition.ShadowColumnCondition;
+import org.apache.shardingsphere.shadow.condition.ShadowDetermineCondition;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public final class ColumnShadowAlgorithmDeterminerTest {
+    
+    @Test
+    public void assertIsShadow() {
+        assertThat(ColumnShadowAlgorithmDeterminer.isShadow(createColumnShadowAlgorithms(), createShadowDetermineCondition()), is(true));
+    }
+    
+    private ColumnShadowAlgorithm<Comparable<?>> createColumnShadowAlgorithms() {
+        final ColumnShadowAlgorithm<Comparable<?>> result = new ColumnRegexMatchShadowAlgorithm();
+        Properties properties = new Properties();
+        properties.setProperty("column", "user_id");
+        properties.setProperty("operation", "insert");
+        properties.setProperty("regex", "[1]");
+        result.setProps(properties);
+        result.init();
+        return result;
+    }
+    
+    private ShadowDetermineCondition createShadowDetermineCondition() {
+        ShadowDetermineCondition result = new ShadowDetermineCondition("t_order", ShadowOperationType.INSERT);
+        result.initShadowColumnCondition(createColumnValuesMapping());
+        return result;
+    }
+    
+    private ShadowColumnCondition createColumnValuesMapping() {
+        Collection<Comparable<?>> values = new LinkedList<>();
+        values.add(1);
+        return new ShadowColumnCondition("t_order", "user_id", values);
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/algorithm/HintShadowAlgorithmDeterminerTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminerTest.java
similarity index 87%
rename from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/algorithm/HintShadowAlgorithmDeterminerTest.java
rename to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminerTest.java
index 58b83ac..82c7854 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/algorithm/HintShadowAlgorithmDeterminerTest.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminerTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.shadow.route.engine.determiner.algorithm;
+package org.apache.shardingsphere.shadow.route.engine.determiner;
 
 import org.apache.shardingsphere.shadow.algorithm.config.AlgorithmProvidedShadowRuleConfiguration;
 import org.apache.shardingsphere.shadow.algorithm.shadow.hint.SimpleHintShadowAlgorithm;
@@ -24,10 +24,8 @@ import org.apache.shardingsphere.shadow.api.config.table.ShadowTableConfiguratio
 import org.apache.shardingsphere.shadow.api.shadow.ShadowOperationType;
 import org.apache.shardingsphere.shadow.api.shadow.hint.HintShadowAlgorithm;
 import org.apache.shardingsphere.shadow.condition.ShadowDetermineCondition;
-import org.apache.shardingsphere.shadow.route.engine.determiner.ShadowAlgorithmDeterminer;
 import org.apache.shardingsphere.shadow.rule.ShadowRule;
 import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
-import org.junit.Before;
 import org.junit.Test;
 
 import java.util.Collection;
@@ -41,30 +39,22 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 
 public final class HintShadowAlgorithmDeterminerTest {
-
-    private ShadowAlgorithmDeterminer shadowAlgorithmDeterminer;
-
-    @Before
-    public void init() {
-        shadowAlgorithmDeterminer = new HintShadowAlgorithmDeterminer(createHintShadowAlgorithm());
+    
+    @Test
+    public void assertIsShadow() {
+        assertThat(HintShadowAlgorithmDeterminer.isShadow(createHintShadowAlgorithm(), createShadowDetermineCondition(), new ShadowRule(createAlgorithmProvidedShadowRuleConfiguration())), is(true));
     }
-
+    
     @SuppressWarnings({"rawtypes", "unchecked"})
     private HintShadowAlgorithm<Comparable<?>> createHintShadowAlgorithm() {
         HintShadowAlgorithm result = new SimpleHintShadowAlgorithm();
         Properties properties = new Properties();
-        properties.setProperty("shadow", "true");
         properties.setProperty("foo", "bar");
         result.setProps(properties);
         result.init();
         return result;
     }
     
-    @Test
-    public void assertIsShadow() {
-        assertThat(shadowAlgorithmDeterminer.isShadow(createShadowDetermineCondition(), new ShadowRule(createAlgorithmProvidedShadowRuleConfiguration())), is(true));
-    }
-
     private AlgorithmProvidedShadowRuleConfiguration createAlgorithmProvidedShadowRuleConfiguration() {
         AlgorithmProvidedShadowRuleConfiguration result = new AlgorithmProvidedShadowRuleConfiguration();
         result.setDataSources(createDataSources());
@@ -101,7 +91,7 @@ public final class HintShadowAlgorithmDeterminerTest {
     private ShadowDetermineCondition createShadowDetermineCondition() {
         ShadowDetermineCondition result = new ShadowDetermineCondition("t_order", ShadowOperationType.INSERT);
         Collection<String> sqlComments = new LinkedList<>();
-        sqlComments.add("/*foo:bar,shadow:true*/");
+        sqlComments.add("/*foo:bar*/");
         return result.initSQLComments(sqlComments);
     }
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/ShadowDeterminerFactoryTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/ShadowDeterminerFactoryTest.java
deleted file mode 100644
index 660c39e..0000000
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/ShadowDeterminerFactoryTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.route.engine.determiner;
-
-import org.apache.shardingsphere.shadow.api.shadow.column.ColumnShadowAlgorithm;
-import org.apache.shardingsphere.shadow.api.shadow.hint.HintShadowAlgorithm;
-import org.apache.shardingsphere.shadow.route.engine.determiner.algorithm.ColumnShadowAlgorithmDeterminer;
-import org.apache.shardingsphere.shadow.route.engine.determiner.algorithm.HintShadowAlgorithmDeterminer;
-import org.junit.Test;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-
-public final class ShadowDeterminerFactoryTest {
-    
-    @Test
-    public void assertSuccessNewInstance() {
-        assertThat(ShadowDeterminerFactory.newInstance(mock(HintShadowAlgorithm.class)) instanceof HintShadowAlgorithmDeterminer, is(true));
-        assertThat(ShadowDeterminerFactory.newInstance(mock(ColumnShadowAlgorithm.class)) instanceof ColumnShadowAlgorithmDeterminer, is(true));
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/algorithm/ColumnShadowAlgorithmDeterminerTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/algorithm/ColumnShadowAlgorithmDeterminerTest.java
deleted file mode 100644
index 441d634b..0000000
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/algorithm/ColumnShadowAlgorithmDeterminerTest.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * 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.route.engine.determiner.algorithm;
-
-import org.apache.shardingsphere.shadow.algorithm.config.AlgorithmProvidedShadowRuleConfiguration;
-import org.apache.shardingsphere.shadow.algorithm.shadow.column.ColumnRegexMatchShadowAlgorithm;
-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.ShadowOperationType;
-import org.apache.shardingsphere.shadow.api.shadow.column.ColumnShadowAlgorithm;
-import org.apache.shardingsphere.shadow.condition.ShadowColumnCondition;
-import org.apache.shardingsphere.shadow.condition.ShadowDetermineCondition;
-import org.apache.shardingsphere.shadow.route.engine.determiner.ShadowAlgorithmDeterminer;
-import org.apache.shardingsphere.shadow.rule.ShadowRule;
-import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.Properties;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-
-public final class ColumnShadowAlgorithmDeterminerTest {
-    
-    private ShadowAlgorithmDeterminer shadowAlgorithmDeterminer;
-    
-    @Before
-    public void init() {
-        shadowAlgorithmDeterminer = new ColumnShadowAlgorithmDeterminer(createColumnShadowAlgorithms());
-    }
-    
-    private ColumnShadowAlgorithm<Comparable<?>> createColumnShadowAlgorithms() {
-        ColumnShadowAlgorithm<Comparable<?>> result = new ColumnRegexMatchShadowAlgorithm();
-        result.setProps(createProperties());
-        result.init();
-        return result;
-    }
-    
-    private Properties createProperties() {
-        Properties properties = new Properties();
-        properties.setProperty("column", "user_id");
-        properties.setProperty("operation", "insert");
-        properties.setProperty("regex", "[1]");
-        return properties;
-    }
-    
-    @Test
-    public void assertIsShadow() {
-        assertThat(shadowAlgorithmDeterminer.isShadow(createShadowDetermineCondition(), new ShadowRule(createAlgorithmProvidedShadowRuleConfiguration())), is(true));
-    }
-    
-    private AlgorithmProvidedShadowRuleConfiguration createAlgorithmProvidedShadowRuleConfiguration() {
-        AlgorithmProvidedShadowRuleConfiguration result = new AlgorithmProvidedShadowRuleConfiguration();
-        result.setDataSources(createDataSources());
-        result.setTables(createTables());
-        result.setShadowAlgorithms(createShadowAlgorithms());
-        return result;
-    }
-    
-    private Map<String, ShadowAlgorithm> createShadowAlgorithms() {
-        Map<String, ShadowAlgorithm> result = new LinkedHashMap<>();
-        result.put("user_id-insert-regex-algorithm", createColumnShadowAlgorithms());
-        return result;
-    }
-    
-    private Map<String, ShadowTableConfiguration> createTables() {
-        Map<String, ShadowTableConfiguration> result = new LinkedHashMap<>();
-        result.put("t_order", new ShadowTableConfiguration(Collections.singletonList("shadow-data-source-0"), createShadowAlgorithmNames()));
-        return result;
-    }
-    
-    private Collection<String> createShadowAlgorithmNames() {
-        Collection<String> result = new LinkedList<>();
-        result.add("user_id-insert-regex-algorithm");
-        return result;
-    }
-    
-    private Map<String, ShadowDataSourceConfiguration> createDataSources() {
-        Map<String, ShadowDataSourceConfiguration> result = new LinkedHashMap<>();
-        result.put("shadow-data-source-0", new ShadowDataSourceConfiguration("ds", "ds_shadow"));
-        result.put("shadow-data-source-1", new ShadowDataSourceConfiguration("ds1", "ds1_shadow"));
-        return result;
-    }
-    
-    private ShadowDetermineCondition createShadowDetermineCondition() {
-        ShadowDetermineCondition result = new ShadowDetermineCondition("t_order", ShadowOperationType.INSERT);
-        result.initShadowColumnCondition(createColumnValuesMapping());
-        return result;
-    }
-    
-    private ShadowColumnCondition createColumnValuesMapping() {
-        Collection<Comparable<?>> values = new LinkedList<>();
-        values.add(1);
-        return new ShadowColumnCondition("t_order", "user_id", values);
-    }
-}