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/31 04:21:43 UTC

[shardingsphere] branch master updated: Add insert statement shadow route engine (#12112)

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 5444121  Add insert statement shadow route engine (#12112)
5444121 is described below

commit 54441219ef1e8010955084530fb4f952226df715
Author: gin <ja...@163.com>
AuthorDate: Tue Aug 31 12:20:54 2021 +0800

    Add insert statement shadow route engine (#12112)
    
    * Add insert statement shadow route engine.
    
    * Add insert statement shadow route engine test.
    
    * Optimize rule checker method.
---
 .../future/engine/AbstractShadowRouteEngine.java   |  72 ++++++++
 .../future/engine/ShadowRouteEngineFactory.java    |  10 +-
 .../determiner/ShadowAlgorithmDeterminer.java      |  38 +++++
 .../engine/determiner/ShadowDeterminerFactory.java |  70 ++++++++
 .../ShadowTableDeterminer.java}                    |  25 +--
 .../algorithm/ColumnShadowAlgorithmDeterminer.java |  79 +++++++++
 .../algorithm/NoteShadowAlgorithmDeterminer.java   |  55 ++++++
 ...nyAlgorithmApplicableShadowTableDeterminer.java |  46 ++++++
 .../ShadowDeleteStatementRoutingEngine.java        |   2 +-
 .../dml/ShadowInsertStatementRoutingEngine.java    |  68 ++++++++
 .../ShadowSelectStatementRoutingEngine.java        |   2 +-
 .../ShadowUpdateStatementRoutingEngine.java        |   2 +-
 .../shardingsphere/shadow/rule/ShadowRule.java     |   9 +-
 .../rule/checker/ShadowTableRuleChecker.java       |  62 ++++---
 .../engine/AbstractShadowRouteEngineTest.java      |  68 ++++++++
 .../engine/ShadowRouteEngineFactoryTest.java       |  44 +++++
 .../determiner/ShadowDeterminerFactoryTest.java    |  60 +++++++
 .../ColumnShadowAlgorithmDeterminerTest.java       | 105 ++++++++++++
 .../NoteShadowAlgorithmDeterminerTest.java         |  82 +++++++++
 ...gorithmApplicableShadowTableDeterminerTest.java |  95 +++++++++++
 .../ShadowInsertStatementRoutingEngineTest.java    | 184 +++++++++++++++++++++
 21 files changed, 1120 insertions(+), 58 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/AbstractShadowRouteEngine.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/AbstractShadowRouteEngine.java
new file mode 100644
index 0000000..82ed8bd
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/AbstractShadowRouteEngine.java
@@ -0,0 +1,72 @@
+/*
+ * 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.future.engine;
+
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.route.context.RouteMapper;
+import org.apache.shardingsphere.infra.route.context.RouteUnit;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.ShadowTableDeterminer;
+import org.apache.shardingsphere.shadow.rule.ShadowRule;
+
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * Abstract shadow route engine.
+ */
+public abstract class AbstractShadowRouteEngine implements ShadowRouteEngine {
+    
+    private final Map<String, ShadowTableDeterminer> shadowTableDeterminers = new LinkedHashMap<>();
+    
+    protected Map<String, ShadowTableDeterminer> getShadowTableDeterminers() {
+        return shadowTableDeterminers;
+    }
+    
+    /**
+     * Get shadow table determiner by table name.
+     *
+     * @param tableName table name
+     * @return shadow table determiner
+     */
+    protected Optional<ShadowTableDeterminer> getShadowTableDeterminer(final String tableName) {
+        ShadowTableDeterminer shadowTableDeterminer = shadowTableDeterminers.get(tableName);
+        return Objects.isNull(shadowTableDeterminer) ? Optional.empty() : Optional.of(shadowTableDeterminer);
+    }
+    
+    /**
+     * Do shadow decorate in DML statement.
+     *
+     * @param routeContext route context
+     * @param shadowRule shadow rule
+     */
+    protected void shadowDMLStatementRouteDecorate(final RouteContext routeContext, final ShadowRule shadowRule) {
+        Collection<RouteUnit> toBeAdded = new LinkedList<>();
+        routeContext.getRouteUnits().forEach(each -> toBeAdded.add(createActualShadowRouteUnit(each, shadowRule)));
+        routeContext.getRouteUnits().clear();
+        routeContext.getRouteUnits().addAll(toBeAdded);
+    }
+    
+    private RouteUnit createActualShadowRouteUnit(final RouteUnit routeUnit, final ShadowRule shadowRule) {
+        return new RouteUnit(new RouteMapper(routeUnit.getDataSourceMapper().getLogicName(), shadowRule.getShadowDataSourceMappings().get(routeUnit.getDataSourceMapper().getActualName())),
+                routeUnit.getTableMappers());
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/ShadowRouteEngineFactory.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/ShadowRouteEngineFactory.java
index 6ce0bb0..cccd254 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/ShadowRouteEngineFactory.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/ShadowRouteEngineFactory.java
@@ -20,11 +20,11 @@ package org.apache.shardingsphere.shadow.route.future.engine;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.binder.LogicSQL;
-import org.apache.shardingsphere.shadow.route.future.engine.impl.ShadowDeleteStatementRoutingEngine;
-import org.apache.shardingsphere.shadow.route.future.engine.impl.ShadowInsertStatementRoutingEngine;
+import org.apache.shardingsphere.shadow.route.future.engine.dml.ShadowDeleteStatementRoutingEngine;
+import org.apache.shardingsphere.shadow.route.future.engine.dml.ShadowInsertStatementRoutingEngine;
+import org.apache.shardingsphere.shadow.route.future.engine.dml.ShadowSelectStatementRoutingEngine;
+import org.apache.shardingsphere.shadow.route.future.engine.dml.ShadowUpdateStatementRoutingEngine;
 import org.apache.shardingsphere.shadow.route.future.engine.impl.ShadowNonMDLStatementRoutingEngine;
-import org.apache.shardingsphere.shadow.route.future.engine.impl.ShadowSelectStatementRoutingEngine;
-import org.apache.shardingsphere.shadow.route.future.engine.impl.ShadowUpdateStatementRoutingEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
@@ -41,7 +41,7 @@ public final class ShadowRouteEngineFactory {
      * Create new instance of shadow route engine.
      *
      * @param logicSQL logic SQL
-     * @return new instance of shadow route engine.
+     * @return new instance of shadow route engine
      */
     public static ShadowRouteEngine newInstance(final LogicSQL logicSQL) {
         SQLStatement sqlStatement = logicSQL.getSqlStatementContext().getSqlStatement();
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/ShadowAlgorithmDeterminer.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/ShadowAlgorithmDeterminer.java
new file mode 100644
index 0000000..686a2aa
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/ShadowAlgorithmDeterminer.java
@@ -0,0 +1,38 @@
+/*
+ * 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.future.engine.determiner;
+
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+
+import java.util.Collection;
+
+/**
+ * Shadow algorithm determiner.
+ */
+public interface ShadowAlgorithmDeterminer {
+    
+    /**
+     * Is shadow in shadow algorithm.
+     *
+     * @param insertStatementContext insert statement context
+     * @param relatedShadowTables related shadow tables
+     * @param tableName table name
+     * @return is pass or not
+     */
+    boolean isShadow(InsertStatementContext insertStatementContext, Collection<String> relatedShadowTables, String tableName);
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/ShadowDeterminerFactory.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/ShadowDeterminerFactory.java
new file mode 100644
index 0000000..33e077b
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/ShadowDeterminerFactory.java
@@ -0,0 +1,70 @@
+/*
+ * 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.future.engine.determiner;
+
+import org.apache.shardingsphere.shadow.algorithm.shadow.ShadowAlgorithmException;
+import org.apache.shardingsphere.shadow.api.shadow.column.ColumnShadowAlgorithm;
+import org.apache.shardingsphere.shadow.api.shadow.note.NoteShadowAlgorithm;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.algorithm.ColumnShadowAlgorithmDeterminer;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.algorithm.NoteShadowAlgorithmDeterminer;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.table.AnyAlgorithmApplicableShadowTableDeterminer;
+import org.apache.shardingsphere.shadow.rule.ShadowRule;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Shadow determiner factory.
+ */
+public final class ShadowDeterminerFactory {
+    
+    /**
+     * Create new instance of shadow table determiner.
+     *
+     * @param tableName table name
+     * @param shadowRule shadow rule
+     * @return new instance of shadow table determiner
+     */
+    public static Optional<ShadowTableDeterminer> getShadowTableDeterminer(final String tableName, final ShadowRule shadowRule) {
+        return shadowRule.getRelatedShadowAlgorithms(tableName).map(shadowAlgorithms -> new AnyAlgorithmApplicableShadowTableDeterminer(createShadowAlgorithmDeterminers(shadowAlgorithms)));
+    }
+    
+    private static Collection<ShadowAlgorithmDeterminer> createShadowAlgorithmDeterminers(final Collection<ShadowAlgorithm> shadowAlgorithms) {
+        return shadowAlgorithms.stream().map(ShadowDeterminerFactory::getShadowAlgorithmDeterminer).collect(Collectors.toCollection(LinkedList::new));
+    }
+    
+    /**
+     * Create new instance of Shadow algorithm determiner.
+     *
+     * @param shadowAlgorithm shadow algorithm.
+     * @return new instance of Shadow algorithm determiner
+     */
+    @SuppressWarnings(value = "unchecked")
+    public static ShadowAlgorithmDeterminer getShadowAlgorithmDeterminer(final ShadowAlgorithm shadowAlgorithm) {
+        if (shadowAlgorithm instanceof NoteShadowAlgorithm) {
+            return new NoteShadowAlgorithmDeterminer((NoteShadowAlgorithm<Comparable<?>>) shadowAlgorithm);
+        } else if (shadowAlgorithm instanceof ColumnShadowAlgorithm) {
+            return new ColumnShadowAlgorithmDeterminer((ColumnShadowAlgorithm<Comparable<?>>) shadowAlgorithm);
+        } else {
+            throw new ShadowAlgorithmException("Shadow algorithm determiner factory new instance failed shadow algorithm type is `%s`.", shadowAlgorithm.getType());
+        }
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/impl/ShadowInsertStatementRoutingEngine.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/ShadowTableDeterminer.java
similarity index 51%
rename from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/impl/ShadowInsertStatementRoutingEngine.java
rename to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/ShadowTableDeterminer.java
index 96bee8f..de41965 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/impl/ShadowInsertStatementRoutingEngine.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/ShadowTableDeterminer.java
@@ -15,22 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.shadow.route.future.engine.impl;
+package org.apache.shardingsphere.shadow.route.future.engine.determiner;
 
-import org.apache.shardingsphere.infra.binder.LogicSQL;
-import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.route.context.RouteContext;
-import org.apache.shardingsphere.shadow.route.future.engine.ShadowRouteEngine;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
 import org.apache.shardingsphere.shadow.rule.ShadowRule;
 
 /**
- * Shadow insert statement routing engine.
+ * Shadow table determiner.
  */
-public final class ShadowInsertStatementRoutingEngine implements ShadowRouteEngine {
+public interface ShadowTableDeterminer {
     
-    @Override
-    public void route(final RouteContext routeContext, final LogicSQL logicSQL, final ShardingSphereMetaData metaData, final ShadowRule shadowRule, final ConfigurationProperties props) {
-        // TODO decorate route in insert statement case
-    }
+    /**
+     * Is shadow in shadow table.
+     *
+     * @param insertStatementContext insert statement context
+     * @param shadowRule related shadow tables
+     * @param tableName table name
+     * @return is shadow or not
+     */
+    boolean isShadow(InsertStatementContext insertStatementContext, ShadowRule shadowRule, String tableName);
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/algorithm/ColumnShadowAlgorithmDeterminer.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/algorithm/ColumnShadowAlgorithmDeterminer.java
new file mode 100644
index 0000000..a29d543
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/algorithm/ColumnShadowAlgorithmDeterminer.java
@@ -0,0 +1,79 @@
+/*
+ * 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.future.engine.determiner.algorithm;
+
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.shadow.api.shadow.column.ColumnShadowAlgorithm;
+import org.apache.shardingsphere.shadow.api.shadow.column.PreciseColumnShadowValue;
+import org.apache.shardingsphere.shadow.api.shadow.column.ShadowOperationType;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.ShadowAlgorithmDeterminer;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Column shadow algorithm determiner.
+ */
+@RequiredArgsConstructor
+public final class ColumnShadowAlgorithmDeterminer implements ShadowAlgorithmDeterminer {
+    
+    private final ColumnShadowAlgorithm<Comparable<?>> columnShadowAlgorithm;
+    
+    @Override
+    public boolean isShadow(final InsertStatementContext insertStatementContext, final Collection<String> relatedShadowTables, final String tableName) {
+        Collection<String> columnNames = insertStatementContext.getInsertColumnNames();
+        List<List<Object>> groupedParameters = insertStatementContext.getGroupedParameters();
+        Iterator<String> columnNamesIt = columnNames.iterator();
+        Iterator<List<Object>> groupedParametersIt = groupedParameters.iterator();
+        int count = Math.min(columnNames.size(), groupedParameters.size());
+        for (int i = 0; i < count; i++) {
+            Optional<Collection<PreciseColumnShadowValue<Comparable<?>>>> preciseColumnShadowValues = getPreciseColumnShadowValues(columnNamesIt.next(), groupedParametersIt.next(), tableName);
+            if (preciseColumnShadowValues.isPresent()) {
+                if (isPassColumn(preciseColumnShadowValues.get(), columnShadowAlgorithm, relatedShadowTables)) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+    
+    private boolean isPassColumn(final Collection<PreciseColumnShadowValue<Comparable<?>>> preciseColumnShadowValues, final ColumnShadowAlgorithm<Comparable<?>> columnShadowAlgorithm,
+                                 final Collection<String> relatedShadowTables) {
+        for (PreciseColumnShadowValue<Comparable<?>> each : preciseColumnShadowValues) {
+            if (!columnShadowAlgorithm.isShadow(relatedShadowTables, each)) {
+                return false;
+            }
+        }
+        return true;
+    }
+    
+    private Optional<Collection<PreciseColumnShadowValue<Comparable<?>>>> getPreciseColumnShadowValues(final String columnName, final List<Object> groupedParameter, final String tableName) {
+        Collection<PreciseColumnShadowValue<Comparable<?>>> result = new LinkedList<>();
+        for (Object each : groupedParameter) {
+            if (!(each instanceof Comparable<?>)) {
+                return Optional.empty();
+            }
+            result.add(new PreciseColumnShadowValue<>(tableName, ShadowOperationType.INSERT, columnName, (Comparable<?>) each));
+        }
+        return Optional.of(result);
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/algorithm/NoteShadowAlgorithmDeterminer.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/algorithm/NoteShadowAlgorithmDeterminer.java
new file mode 100644
index 0000000..4a3c827
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/algorithm/NoteShadowAlgorithmDeterminer.java
@@ -0,0 +1,55 @@
+/*
+ * 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.future.engine.determiner.algorithm;
+
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.shadow.api.shadow.note.NoteShadowAlgorithm;
+import org.apache.shardingsphere.shadow.api.shadow.note.PreciseNoteShadowValue;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.ShadowAlgorithmDeterminer;
+
+import java.util.Collection;
+import java.util.LinkedList;
+
+/**
+ * Note shadow algorithm determiner.
+ */
+@RequiredArgsConstructor
+public final class NoteShadowAlgorithmDeterminer implements ShadowAlgorithmDeterminer {
+    
+    private final NoteShadowAlgorithm<Comparable<?>> noteShadowAlgorithm;
+    
+    @Override
+    public boolean isShadow(final InsertStatementContext insertStatementContext, final Collection<String> relatedShadowTables, final String tableName) {
+        Collection<PreciseNoteShadowValue<Comparable<?>>> noteShadowValues = createNoteShadowValues(insertStatementContext, tableName);
+        for (PreciseNoteShadowValue<Comparable<?>> noteShadowValue : noteShadowValues) {
+            if (noteShadowAlgorithm.isShadow(relatedShadowTables, noteShadowValue)) {
+                return true;
+            }
+        }
+        return false;
+    }
+    
+    // FIXME refactor the method when sql parses the note and puts it in the statement context
+    private Collection<PreciseNoteShadowValue<Comparable<?>>> createNoteShadowValues(final InsertStatementContext insertStatementContext, final String logicTableName) {
+        Collection<PreciseNoteShadowValue<Comparable<?>>> result = new LinkedList<>();
+        result.add(new PreciseNoteShadowValue<>(logicTableName, "/*table=t_user,shadow=true*/"));
+        result.add(new PreciseNoteShadowValue<>(logicTableName, "/*aaa=bbb*/"));
+        return result;
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/table/AnyAlgorithmApplicableShadowTableDeterminer.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/table/AnyAlgorithmApplicableShadowTableDeterminer.java
new file mode 100644
index 0000000..b110fc1
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/table/AnyAlgorithmApplicableShadowTableDeterminer.java
@@ -0,0 +1,46 @@
+/*
+ * 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.future.engine.determiner.table;
+
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.ShadowAlgorithmDeterminer;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.ShadowTableDeterminer;
+import org.apache.shardingsphere.shadow.rule.ShadowRule;
+
+import java.util.Collection;
+
+/**
+ * any algorithm applicable shadow table determiner.
+ */
+@RequiredArgsConstructor
+public final class AnyAlgorithmApplicableShadowTableDeterminer implements ShadowTableDeterminer {
+    
+    private final Collection<ShadowAlgorithmDeterminer> shadowAlgorithmDeterminers;
+    
+    @Override
+    public boolean isShadow(final InsertStatementContext insertStatementContext, final ShadowRule shadowRule, final String tableName) {
+        Collection<String> shadowTableNames = shadowRule.getAllShadowTableNames();
+        for (ShadowAlgorithmDeterminer each : shadowAlgorithmDeterminers) {
+            if (each.isShadow(insertStatementContext, shadowTableNames, tableName)) {
+                return true;
+            }
+        }
+        return false;
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/impl/ShadowDeleteStatementRoutingEngine.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowDeleteStatementRoutingEngine.java
similarity index 96%
rename from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/impl/ShadowDeleteStatementRoutingEngine.java
rename to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowDeleteStatementRoutingEngine.java
index 5964a73..461d382 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/impl/ShadowDeleteStatementRoutingEngine.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowDeleteStatementRoutingEngine.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.shadow.route.future.engine.impl;
+package org.apache.shardingsphere.shadow.route.future.engine.dml;
 
 import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowInsertStatementRoutingEngine.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowInsertStatementRoutingEngine.java
new file mode 100644
index 0000000..cb8b309
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowInsertStatementRoutingEngine.java
@@ -0,0 +1,68 @@
+/*
+ * 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.future.engine.dml;
+
+import org.apache.shardingsphere.infra.binder.LogicSQL;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.shadow.route.future.engine.AbstractShadowRouteEngine;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.ShadowDeterminerFactory;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.ShadowTableDeterminer;
+import org.apache.shardingsphere.shadow.rule.ShadowRule;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Shadow insert statement routing engine.
+ */
+public final class ShadowInsertStatementRoutingEngine extends AbstractShadowRouteEngine {
+    
+    @Override
+    public void route(final RouteContext routeContext, final LogicSQL logicSQL, final ShardingSphereMetaData metaData, final ShadowRule shadowRule, final ConfigurationProperties props) {
+        if (isShadow(logicSQL, shadowRule)) {
+            shadowDMLStatementRouteDecorate(routeContext, shadowRule);
+        }
+    }
+    
+    private boolean isShadow(final LogicSQL logicSQL, final ShadowRule shadowRule) {
+        InsertStatementContext insertStatementContext = (InsertStatementContext) logicSQL.getSqlStatementContext();
+        Collection<String> relatedShadowTables = getRelatedShadowTables(insertStatementContext, shadowRule);
+        initShadowTableDeterminer(relatedShadowTables, shadowRule);
+        for (String each : relatedShadowTables) {
+            getShadowTableDeterminer(each).ifPresent(tableDeterminer -> tableDeterminer.isShadow(insertStatementContext, shadowRule, each));
+        }
+        return false;
+    }
+    
+    private Collection<String> getRelatedShadowTables(final InsertStatementContext insertStatementContext, final ShadowRule shadowRule) {
+        return shadowRule.getRelatedShadowTables(insertStatementContext.getAllTables().stream().map(each -> each.getTableName().getIdentifier().getValue())
+                .collect(Collectors.toCollection(LinkedList::new)));
+    }
+    
+    private void initShadowTableDeterminer(final Collection<String> relatedShadowTables, final ShadowRule shadowRule) {
+        for (String each : relatedShadowTables) {
+            Optional<ShadowTableDeterminer> shadowTableDeterminer = ShadowDeterminerFactory.getShadowTableDeterminer(each, shadowRule);
+            shadowTableDeterminer.ifPresent(tableDeterminer -> getShadowTableDeterminers().put(each, tableDeterminer));
+        }
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/impl/ShadowSelectStatementRoutingEngine.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowSelectStatementRoutingEngine.java
similarity index 96%
rename from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/impl/ShadowSelectStatementRoutingEngine.java
rename to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowSelectStatementRoutingEngine.java
index 23d22f4..6828c68 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/impl/ShadowSelectStatementRoutingEngine.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowSelectStatementRoutingEngine.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.shadow.route.future.engine.impl;
+package org.apache.shardingsphere.shadow.route.future.engine.dml;
 
 import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/impl/ShadowUpdateStatementRoutingEngine.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowUpdateStatementRoutingEngine.java
similarity index 96%
rename from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/impl/ShadowUpdateStatementRoutingEngine.java
rename to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowUpdateStatementRoutingEngine.java
index 9c01f4c..15f9561 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/impl/ShadowUpdateStatementRoutingEngine.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowUpdateStatementRoutingEngine.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.shadow.route.future.engine.impl;
+package org.apache.shardingsphere.shadow.route.future.engine.dml;
 
 import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
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 59f9183..28b342b 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
@@ -139,12 +139,9 @@ public final class ShadowRule implements FeatureRule, SchemaRule, DataSourceCont
         if (Objects.isNull(shadowTableRule)) {
             return Optional.empty();
         }
-        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);
+        Collection<ShadowAlgorithm> result = shadowTableRule.getShadowAlgorithmNames().stream().map(shadowAlgorithms::get).filter(each -> !Objects.isNull(each))
+                .collect(Collectors.toCollection(LinkedList::new));
+        return result.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/checker/ShadowTableRuleChecker.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowTableRuleChecker.java
index 173b154..0aa2c25 100644
--- 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
@@ -76,41 +76,39 @@ public final class ShadowTableRuleChecker {
     }
     
     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;
-                    }
-                }
-            }
+        int[] operationCount = new int[4];
+        tableShadowAlgorithms.stream().filter(each -> each instanceof ColumnShadowAlgorithm).forEach(each -> checkTableColumnShadowAlgorithm(each, operationCount, tableName));
+    }
+    
+    private static void checkTableColumnShadowAlgorithm(final ShadowAlgorithm shadowAlgorithm, final int[] operationCount, final String tableName) {
+        Optional<ShadowOperationType> shadowOperationTypeOptional = ShadowOperationType.contains(shadowAlgorithm.getProps().get("operation").toString());
+        shadowOperationTypeOptional.ifPresent(shadowOperationType -> checkTableColumnShadowAlgorithmOperation(shadowOperationType, operationCount, tableName));
+    }
+    
+    private static void checkTableColumnShadowAlgorithmOperation(final ShadowOperationType shadowOperationType, final int[] operationCount, final String tableName) {
+        switch (shadowOperationType) {
+            case INSERT:
+                operationCount[0]++;
+                checkOperationCount(shadowOperationType, operationCount[0], tableName);
+                break;
+            case UPDATE:
+                operationCount[1]++;
+                checkOperationCount(shadowOperationType, operationCount[1], tableName);
+                break;
+            case DELETE:
+                operationCount[2]++;
+                checkOperationCount(shadowOperationType, operationCount[2], tableName);
+                break;
+            case SELECT:
+                operationCount[3]++;
+                checkOperationCount(shadowOperationType, operationCount[3], tableName);
+                break;
+            default:
+                break;
         }
     }
     
-    private static void checkOperationCount(final String tableName, final ShadowOperationType shadowOperationType, final int operationCount) {
+    private static void checkOperationCount(final ShadowOperationType shadowOperationType, final int operationCount, final String tableName) {
         Preconditions.checkState(operationCount <= 1, "Column shadow algorithm `%s` operation only supports one column mapping in shadow table `%s`.",
                 shadowOperationType.name(), tableName);
     }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/AbstractShadowRouteEngineTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/AbstractShadowRouteEngineTest.java
new file mode 100644
index 0000000..701aa9b
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/AbstractShadowRouteEngineTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.future.engine;
+
+import com.google.common.collect.Lists;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.route.context.RouteMapper;
+import org.apache.shardingsphere.infra.route.context.RouteUnit;
+import org.apache.shardingsphere.shadow.route.future.engine.dml.ShadowInsertStatementRoutingEngine;
+import org.apache.shardingsphere.shadow.rule.ShadowRule;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.Map;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class AbstractShadowRouteEngineTest {
+    
+    @Test
+    public void assertDoShadowDecorate() {
+        AbstractShadowRouteEngine abstractShadowRouteEngine = new ShadowInsertStatementRoutingEngine();
+        abstractShadowRouteEngine.shadowDMLStatementRouteDecorate(createRouteContext(), createShadowRule());
+    }
+    
+    private ShadowRule createShadowRule() {
+        ShadowRule shadowRule = mock(ShadowRule.class);
+        when(shadowRule.getShadowDataSourceMappings()).thenReturn(createShadowDataSourceMappings());
+        return shadowRule;
+    }
+    
+    private Map<String, String> createShadowDataSourceMappings() {
+        Map<String, String> result = new LinkedHashMap<>();
+        result.put("ds", "ds_shadow");
+        result.put("ds1", "ds1_shadow");
+        return result;
+    }
+    
+    private RouteContext createRouteContext() {
+        RouteContext routeContext = mock(RouteContext.class);
+        when(routeContext.getRouteUnits()).thenReturn(createRouteUnits());
+        return routeContext;
+    }
+    
+    private Collection<RouteUnit> createRouteUnits() {
+        Collection<RouteUnit> result = new LinkedList<>();
+        result.add(new RouteUnit(new RouteMapper("logic_ds", "ds"), Lists.newArrayList()));
+        return result;
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/ShadowRouteEngineFactoryTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/ShadowRouteEngineFactoryTest.java
new file mode 100644
index 0000000..76c3866
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/ShadowRouteEngineFactoryTest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.future.engine;
+
+import com.google.common.collect.Lists;
+import org.apache.shardingsphere.infra.binder.LogicSQL;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.shadow.route.future.engine.dml.ShadowInsertStatementRoutingEngine;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class ShadowRouteEngineFactoryTest {
+    
+    @Test
+    public void assertNewInstance() {
+        assertThat(ShadowRouteEngineFactory.newInstance(new LogicSQL(createSqlStatementContext(), "", Lists.newArrayList())) instanceof ShadowInsertStatementRoutingEngine, is(true));
+    }
+    
+    private SQLStatementContext<InsertStatement> createSqlStatementContext() {
+        SQLStatementContext<InsertStatement> result = mock(SQLStatementContext.class);
+        when(result.getSqlStatement()).thenReturn(mock(InsertStatement.class));
+        return result;
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/ShadowDeterminerFactoryTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/ShadowDeterminerFactoryTest.java
new file mode 100644
index 0000000..9e51022
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/ShadowDeterminerFactoryTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.future.engine.determiner;
+
+import org.apache.shardingsphere.shadow.algorithm.shadow.note.SimpleSQLNoteShadowAlgorithm;
+import org.apache.shardingsphere.shadow.api.shadow.column.ColumnShadowAlgorithm;
+import org.apache.shardingsphere.shadow.api.shadow.note.NoteShadowAlgorithm;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.algorithm.ColumnShadowAlgorithmDeterminer;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.algorithm.NoteShadowAlgorithmDeterminer;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.table.AnyAlgorithmApplicableShadowTableDeterminer;
+import org.apache.shardingsphere.shadow.rule.ShadowRule;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Optional;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class ShadowDeterminerFactoryTest {
+    
+    @Test
+    public void assertSuccessNewInstance() {
+        Optional<ShadowTableDeterminer> shadowTableDeterminer = ShadowDeterminerFactory.getShadowTableDeterminer("t_user", createShadowRule());
+        shadowTableDeterminer.ifPresent(tableDeterminer -> assertThat(tableDeterminer instanceof AnyAlgorithmApplicableShadowTableDeterminer, is(true)));
+        assertThat(ShadowDeterminerFactory.getShadowAlgorithmDeterminer(mock(NoteShadowAlgorithm.class)) instanceof NoteShadowAlgorithmDeterminer, is(true));
+        assertThat(ShadowDeterminerFactory.getShadowAlgorithmDeterminer(mock(ColumnShadowAlgorithm.class)) instanceof ColumnShadowAlgorithmDeterminer, is(true));
+    }
+    
+    private ShadowRule createShadowRule() {
+        ShadowRule shadowRule = mock(ShadowRule.class);
+        when(shadowRule.getRelatedShadowAlgorithms("t_user")).thenReturn(createRelatedShadowAlgorithms());
+        return shadowRule;
+    }
+    
+    private Optional<Collection<ShadowAlgorithm>> createRelatedShadowAlgorithms() {
+        Collection<ShadowAlgorithm> result = new LinkedList<>();
+        result.add(new SimpleSQLNoteShadowAlgorithm());
+        return Optional.of(result);
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/algorithm/ColumnShadowAlgorithmDeterminerTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/algorithm/ColumnShadowAlgorithmDeterminerTest.java
new file mode 100644
index 0000000..97c4a2f
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/algorithm/ColumnShadowAlgorithmDeterminerTest.java
@@ -0,0 +1,105 @@
+/*
+ * 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.future.engine.determiner.algorithm;
+
+import com.google.common.collect.Lists;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.shadow.algorithm.shadow.column.ColumnRegexMatchShadowAlgorithm;
+import org.apache.shardingsphere.shadow.api.shadow.column.ColumnShadowAlgorithm;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class ColumnShadowAlgorithmDeterminerTest {
+    
+    @Test
+    public void assertIsShadow() {
+        assertTrueCase();
+        assertFalseCase();
+    }
+    
+    private void assertFalseCase() {
+        InsertStatementContext insertStatementContext = mock(InsertStatementContext.class);
+        when(insertStatementContext.getInsertColumnNames()).thenReturn(createColumnNames());
+        when(insertStatementContext.getGroupedParameters()).thenReturn(createGroupedParametersFalseCase());
+        ColumnShadowAlgorithmDeterminer columnShadowAlgorithmDeterminer = new ColumnShadowAlgorithmDeterminer(createColumnShadowAlgorithm());
+        assertThat(columnShadowAlgorithmDeterminer.isShadow(insertStatementContext, createRelatedShadowTables(), "t_user"), is(false));
+    }
+    
+    private List<List<Object>> createGroupedParametersFalseCase() {
+        List<List<Object>> result = new LinkedList<>();
+        result.add(Lists.newArrayList(1, 2));
+        result.add(Lists.newArrayList("jack", "rose"));
+        result.add(Lists.newArrayList(1, 2));
+        return result;
+    }
+    
+    private void assertTrueCase() {
+        InsertStatementContext insertStatementContext = mock(InsertStatementContext.class);
+        when(insertStatementContext.getInsertColumnNames()).thenReturn(createColumnNames());
+        when(insertStatementContext.getGroupedParameters()).thenReturn(createGroupedParametersTrueCase());
+        ColumnShadowAlgorithmDeterminer columnShadowAlgorithmDeterminer = new ColumnShadowAlgorithmDeterminer(createColumnShadowAlgorithm());
+        assertThat(columnShadowAlgorithmDeterminer.isShadow(insertStatementContext, createRelatedShadowTables(), "t_user"), is(true));
+    }
+    
+    private Collection<String> createRelatedShadowTables() {
+        Collection<String> result = new LinkedList<>();
+        result.add("t_user");
+        result.add("t_order");
+        return result;
+    }
+    
+    private ColumnShadowAlgorithm<Comparable<?>> createColumnShadowAlgorithm() {
+        ColumnRegexMatchShadowAlgorithm result = new ColumnRegexMatchShadowAlgorithm();
+        result.setProps(createProperties());
+        result.init();
+        return result;
+    }
+    
+    private Properties createProperties() {
+        Properties properties = new Properties();
+        properties.setProperty("column", "age");
+        properties.setProperty("operation", "insert");
+        properties.setProperty("regex", "[1]");
+        return properties;
+    }
+    
+    private List<List<Object>> createGroupedParametersTrueCase() {
+        List<List<Object>> result = new LinkedList<>();
+        result.add(Lists.newArrayList(1, 2));
+        result.add(Lists.newArrayList("jack", "rose"));
+        result.add(Lists.newArrayList(1, 1));
+        return result;
+    }
+    
+    private List<String> createColumnNames() {
+        List<String> result = new LinkedList<>();
+        result.add("id");
+        result.add("name");
+        result.add("age");
+        return result;
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/algorithm/NoteShadowAlgorithmDeterminerTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/algorithm/NoteShadowAlgorithmDeterminerTest.java
new file mode 100644
index 0000000..2b21710
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/algorithm/NoteShadowAlgorithmDeterminerTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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.future.engine.determiner.algorithm;
+
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.shadow.algorithm.shadow.note.SimpleSQLNoteShadowAlgorithm;
+import org.apache.shardingsphere.shadow.api.shadow.note.NoteShadowAlgorithm;
+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;
+import static org.mockito.Mockito.mock;
+
+public final class NoteShadowAlgorithmDeterminerTest {
+    
+    @Test
+    public void assertIsShadow() {
+        assertTrueCase();
+        assertFalseCase();
+    }
+    
+    private void assertTrueCase() {
+        assertThat(new NoteShadowAlgorithmDeterminer(createAlgorithmTrueCase()).isShadow(mock(InsertStatementContext.class), createShadowTablesTrueCase(), "t_user"), is(true));
+    }
+    
+    private Collection<String> createShadowTablesTrueCase() {
+        Collection<String> result = new LinkedList<>();
+        result.add("t_user");
+        result.add("t_order");
+        return result;
+    }
+    
+    private NoteShadowAlgorithm createAlgorithmTrueCase() {
+        SimpleSQLNoteShadowAlgorithm simpleSQLNoteShadowAlgorithm = new SimpleSQLNoteShadowAlgorithm();
+        Properties properties = new Properties();
+        properties.setProperty("shadow", "true");
+        simpleSQLNoteShadowAlgorithm.setProps(properties);
+        simpleSQLNoteShadowAlgorithm.init();
+        return simpleSQLNoteShadowAlgorithm;
+    }
+    
+    private void assertFalseCase() {
+        assertThat(new NoteShadowAlgorithmDeterminer(createAlgorithmFalseCase()).isShadow(mock(InsertStatementContext.class), createShadowTablesTrueCase(), "t_user"), is(false));
+        assertThat(new NoteShadowAlgorithmDeterminer(createAlgorithmTrueCase()).isShadow(mock(InsertStatementContext.class), createShadowTablesFalseCase(), "t_user"), is(false));
+        assertThat(new NoteShadowAlgorithmDeterminer(createAlgorithmTrueCase()).isShadow(mock(InsertStatementContext.class), createShadowTablesTrueCase(), "t_auto"), is(false));
+    }
+    
+    private Collection<String> createShadowTablesFalseCase() {
+        Collection<String> result = new LinkedList<>();
+        result.add("t_auto");
+        result.add("t_order");
+        return result;
+    }
+    
+    private NoteShadowAlgorithm createAlgorithmFalseCase() {
+        SimpleSQLNoteShadowAlgorithm simpleSQLNoteShadowAlgorithm = new SimpleSQLNoteShadowAlgorithm();
+        Properties properties = new Properties();
+        properties.setProperty("shadow", "false");
+        simpleSQLNoteShadowAlgorithm.setProps(properties);
+        simpleSQLNoteShadowAlgorithm.init();
+        return simpleSQLNoteShadowAlgorithm;
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/table/AnyAlgorithmApplicableShadowTableDeterminerTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/table/AnyAlgorithmApplicableShadowTableDeterminerTest.java
new file mode 100644
index 0000000..5062b5c
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/determiner/table/AnyAlgorithmApplicableShadowTableDeterminerTest.java
@@ -0,0 +1,95 @@
+/*
+ * 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.future.engine.determiner.table;
+
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.shadow.algorithm.shadow.note.SimpleSQLNoteShadowAlgorithm;
+import org.apache.shardingsphere.shadow.api.shadow.note.NoteShadowAlgorithm;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.ShadowAlgorithmDeterminer;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.ShadowDeterminerFactory;
+import org.apache.shardingsphere.shadow.route.future.engine.determiner.ShadowTableDeterminer;
+import org.apache.shardingsphere.shadow.rule.ShadowRule;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
+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;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class AnyAlgorithmApplicableShadowTableDeterminerTest {
+    
+    @Test
+    public void assertIsShadow() {
+        assertTrueCase();
+        assertFalseCase();
+    }
+    
+    private void assertTrueCase() {
+        ShadowTableDeterminer shadowTableDeterminer = new AnyAlgorithmApplicableShadowTableDeterminer(createShadowAlgorithmDeterminersTrueCase());
+        assertThat(shadowTableDeterminer.isShadow(mock(InsertStatementContext.class), createShadowRule(), "t_user"), is(true));
+    }
+    
+    private Collection<ShadowAlgorithmDeterminer> createShadowAlgorithmDeterminersTrueCase() {
+        Collection<ShadowAlgorithmDeterminer> result = new LinkedList<>();
+        result.add(ShadowDeterminerFactory.getShadowAlgorithmDeterminer(createShadowAlgorithmsTrueCase()));
+        return result;
+    }
+    
+    private ShadowAlgorithm createShadowAlgorithmsTrueCase() {
+        NoteShadowAlgorithm<String> result = new SimpleSQLNoteShadowAlgorithm();
+        Properties properties = new Properties();
+        properties.setProperty("shadow", "true");
+        result.setProps(properties);
+        result.init();
+        return result;
+    }
+    
+    private ShadowRule createShadowRule() {
+        ShadowRule result = mock(ShadowRule.class);
+        Collection<String> tables = new LinkedList<>();
+        tables.add("t_user");
+        tables.add("t_order");
+        when(result.getAllShadowTableNames()).thenReturn(tables);
+        return result;
+    }
+    
+    private void assertFalseCase() {
+        ShadowTableDeterminer shadowTableDeterminer = new AnyAlgorithmApplicableShadowTableDeterminer(createShadowAlgorithmDeterminersFalseCase());
+        assertThat(shadowTableDeterminer.isShadow(mock(InsertStatementContext.class), createShadowRule(), "t_user"), is(false));
+    }
+    
+    private Collection<ShadowAlgorithmDeterminer> createShadowAlgorithmDeterminersFalseCase() {
+        Collection<ShadowAlgorithmDeterminer> result = new LinkedList<>();
+        result.add(ShadowDeterminerFactory.getShadowAlgorithmDeterminer(createShadowAlgorithmsFalseCase()));
+        return result;
+    }
+    
+    private ShadowAlgorithm createShadowAlgorithmsFalseCase() {
+        NoteShadowAlgorithm<String> result = new SimpleSQLNoteShadowAlgorithm();
+        Properties properties = new Properties();
+        properties.setProperty("shadow", "false");
+        result.setProps(properties);
+        result.init();
+        return result;
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowInsertStatementRoutingEngineTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowInsertStatementRoutingEngineTest.java
new file mode 100644
index 0000000..a159b30
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/route/future/engine/dml/ShadowInsertStatementRoutingEngineTest.java
@@ -0,0 +1,184 @@
+/*
+ * 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.future.engine.dml;
+
+import com.google.common.collect.Lists;
+import org.apache.shardingsphere.infra.binder.LogicSQL;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.route.context.RouteMapper;
+import org.apache.shardingsphere.infra.route.context.RouteUnit;
+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.datasource.ShadowDataSourceConfiguration;
+import org.apache.shardingsphere.shadow.api.config.table.ShadowTableConfiguration;
+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;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class ShadowInsertStatementRoutingEngineTest {
+    
+    private ShadowInsertStatementRoutingEngine shadowRoutingEngine;
+    
+    @Before
+    public void init() {
+        shadowRoutingEngine = new ShadowInsertStatementRoutingEngine();
+    }
+    
+    @Test
+    public void assertRoute() {
+        ShadowRule shadowRule = new ShadowRule(createAlgorithmProvidedShadowRuleConfiguration());
+        shadowRoutingEngine.route(createRouteContext(), new LogicSQL(createInsertStatementContext(), "", Lists.newArrayList()), mock(ShardingSphereMetaData.class), shadowRule,
+                mock(ConfigurationProperties.class));
+    }
+    
+    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 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 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 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<>();
+        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");
+            result.add("simple-note-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;
+    }
+    
+    private InsertStatementContext createInsertStatementContext() {
+        InsertStatementContext result = mock(InsertStatementContext.class);
+        when(result.getInsertColumnNames()).thenReturn(createInsertColumnNames());
+        when(result.getGroupedParameters()).thenReturn(createGroupedParameters());
+        Collection<SimpleTableSegment> allTables = createAllTables();
+        when(result.getAllTables()).thenReturn(allTables);
+        return result;
+    }
+    
+    private List<String> createInsertColumnNames() {
+        return Lists.newArrayList("user_id", "user_name", "user_pwd");
+    }
+    
+    private List<List<Object>> createGroupedParameters() {
+        List<List<Object>> result = new LinkedList<>();
+        result.add(Lists.newArrayList(1, 1));
+        result.add(Lists.newArrayList("jack", "rose"));
+        result.add(Lists.newArrayList("123456", "123456"));
+        return result;
+    }
+    
+    private Collection<SimpleTableSegment> createAllTables() {
+        Collection<SimpleTableSegment> result = new LinkedList<>();
+        result.add(createSimpleTableSegment());
+        return result;
+    }
+    
+    private SimpleTableSegment createSimpleTableSegment() {
+        IdentifierValue identifierValue = mock(IdentifierValue.class);
+        when(identifierValue.getValue()).thenReturn("t_user");
+        return new SimpleTableSegment(new TableNameSegment(0, 10, identifierValue));
+    }
+    
+    private RouteContext createRouteContext() {
+        RouteContext routeContext = mock(RouteContext.class);
+        when(routeContext.getRouteUnits()).thenReturn(createRouteUnits());
+        return routeContext;
+    }
+    
+    private Collection<RouteUnit> createRouteUnits() {
+        Collection<RouteUnit> result = new LinkedList<>();
+        result.add(new RouteUnit(new RouteMapper("logic_ds", "ds"), Lists.newArrayList()));
+        return result;
+    }
+}