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 2020/10/08 05:52:20 UTC

[shardingsphere] branch master updated: Move route strategy to route module (#7708)

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 5f609a83 Move route strategy to route module (#7708)
5f609a83 is described below

commit 5f609a83b99c64b29205a95fd5ea81f8c9845e7c
Author: Liang Zhang <te...@163.com>
AuthorDate: Thu Oct 8 13:52:00 2020 +0800

    Move route strategy to route module (#7708)
    
    * Fix java doc
    
    * Fix java doc
    
    * Simplify WhereClauseShardingConditionEngine.createShardingConditions
    
    * Simplify WhereClauseShardingConditionEngine.createShardingConditions
    
    * For code style
    
    * Add ShardingRule.defaultDatabaseShardingStrategyConfig & defaultTableShardingStrategyConfig
    
    * Refactor ShardingRule.isShardingColumn
    
    * Remove ShardingRule.defaultDatabaseShardingStrategy & defaultTableShardingStrategy
    
    * Remove TableRule.databaseShardingStrategy
    
    * Remove TableRule.tableShardingStrategy
    
    * Revise ShardingRule
    
    * Revise ShardingRule
    
    * Revise ShardingRule
    
    * Revise TableRule
    
    * fix AutoIntervalShardingAlgorithmTest
    
    * fix IntervalShardingAlgorithmTest
    
    * fix InlineShardingAlgorithmTest
    
    * fix HashModShardingAlgorithmTest
    
    * fix ModShardingAlgorithmTest
    
    * fix BoundaryBasedRangeShardingAlgorithmTest
    
    * fix VolumeBasedRangeShardingAlgorithmTest
    
    * Move route strategy to route module
    
    * Move strategy package
    
    * Move RouteValue to condition package
    
    * Rename ShardingConditionValue
    
    * Rename ShardingConditionValue
    
    * Fix checkstyle
---
 .../rewrite/condition/EncryptConditionEngine.java  |   4 +-
 .../impl/EncryptPredicateColumnTokenGenerator.java |   4 +-
 .../shadow/condition/ShadowConditionEngine.java    |   4 +-
 .../impl/ShadowPredicateColumnTokenGenerator.java  |   4 +-
 .../standard/StandardShardingAlgorithm.java        |   8 +-
 .../shardingsphere/sharding/rule/ShardingRule.java | 109 +++++++++----------
 .../shardingsphere/sharding/rule/TableRule.java    |  48 +++-----
 .../AutoIntervalShardingAlgorithmTest.java         |  52 ++-------
 .../datetime/IntervalShardingAlgorithmTest.java    |  90 ++++++---------
 .../inline/InlineShardingAlgorithmTest.java        |  66 ++++-------
 .../sharding/mod/HashModShardingAlgorithmTest.java |  25 +----
 .../sharding/mod/ModShardingAlgorithmTest.java     |  28 ++---
 .../BoundaryBasedRangeShardingAlgorithmTest.java   |  23 +---
 .../VolumeBasedRangeShardingAlgorithmTest.java     |  30 ++---
 .../sharding/rule/BindingTableRuleTest.java        |   4 +-
 .../sharding/rule/ShardingRuleTest.java            |  43 +-------
 .../sharding/rule/TableRuleTest.java               |  36 ++----
 ...e.shardingsphere.sharding.spi.ShardingAlgorithm |   3 -
 .../sharding/route/engine/ShardingSQLRouter.java   |  29 ++---
 .../route/engine/condition/ShardingCondition.java  |   4 +-
 .../impl/InsertClauseShardingConditionEngine.java  |  10 +-
 .../impl/WhereClauseShardingConditionEngine.java   |  87 +++++++--------
 .../generator/ConditionValueGenerator.java         |   4 +-
 .../generator/ConditionValueGeneratorFactory.java  |   4 +-
 .../ConditionValueBetweenOperatorGenerator.java    |  26 ++---
 .../ConditionValueCompareOperatorGenerator.java    |  26 ++---
 .../impl/ConditionValueInOperatorGenerator.java    |  18 +--
 .../AlwaysFalseShardingConditionValue.java}        |   8 +-
 .../value/ListShardingConditionValue.java}         |   6 +-
 .../value/RangeShardingConditionValue.java}        |   6 +-
 .../condition/value/ShardingConditionValue.java}   |   6 +-
 .../standard/ShardingStandardRoutingEngine.java    | 121 +++++++++++++--------
 .../sharding/route}/strategy/ShardingStrategy.java |  16 +--
 .../route}/strategy/ShardingStrategyFactory.java   |  10 +-
 .../type}/complex/ComplexShardingStrategy.java     |  30 ++---
 .../strategy/type}/hint/HintShardingStrategy.java  |  16 +--
 .../strategy/type}/none/NoneShardingStrategy.java  |  12 +-
 .../type}/standard/StandardShardingStrategy.java   |  38 +++----
 ...ConditionValueBetweenOperatorGeneratorTest.java |  56 +++++-----
 ...ConditionValueCompareOperatorGeneratorTest.java |  42 +++----
 .../ConditionValueInOperatorGeneratorTest.java     |  10 +-
 .../value/ListShardingConditionValueTest.java}     |   8 +-
 .../route/fixture/AbstractRoutingEngineTest.java   |  12 +-
 .../fixture/HintShardingAlgorithmFixture.java      |  53 ---------
 .../route}/strategy/ShardingStrategyTest.java      |  26 ++---
 .../ComplexKeysShardingAlgorithmFixture.java       |   2 +-
 .../fixture/HintShardingAlgorithmFixture.java      |   2 +-
 .../fixture/StandardShardingAlgorithmFixture.java  |   2 +-
 ...e.shardingsphere.sharding.spi.ShardingAlgorithm |   5 +-
 .../spring/boot/SpringBootStarterTest.java         |   9 --
 .../shardingsphere/spring/SpringNamespaceTest.java |   2 -
 .../GovernanceSpringBootRegistryShardingTest.java  |  10 --
 .../type/GovernanceSpringBootShardingTest.java     |  10 --
 .../GovernanceShardingNamespaceTest.java           |   9 --
 ...actFromExpression.java => ColumnExtractor.java} |  17 ++-
 .../parser/sql/common/util/ExpressionBuilder.java  |   7 +-
 56 files changed, 538 insertions(+), 802 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/condition/EncryptConditionEngine.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/condition/EncryptConditionEngine.java
index 33a2d72..c23113d 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/condition/EncryptConditionEngine.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/condition/EncryptConditionEngine.java
@@ -34,7 +34,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.S
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.AndPredicate;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.util.ExpressionBuilder;
-import org.apache.shardingsphere.sql.parser.sql.common.util.ColumnExtractFromExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.util.ColumnExtractor;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -96,7 +96,7 @@ public final class EncryptConditionEngine {
     }
     
     private Optional<EncryptCondition> createEncryptCondition(final SQLStatementContext sqlStatementContext, final ExpressionSegment expression) {
-        Optional<ColumnSegment> column = ColumnExtractFromExpression.extract(expression);
+        Optional<ColumnSegment> column = ColumnExtractor.extract(expression);
         if (!column.isPresent()) {
             return Optional.empty();
         }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptPredicateColumnTokenGenerator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptPredicateColumnTokenGenerator.java
index 662c092..1c95418 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptPredicateColumnTokenGenerator.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptPredicateColumnTokenGenerator.java
@@ -32,7 +32,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.Column
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.AndPredicate;
 import org.apache.shardingsphere.sql.parser.sql.common.util.ExpressionBuilder;
-import org.apache.shardingsphere.sql.parser.sql.common.util.ColumnExtractFromExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.util.ColumnExtractor;
 
 import java.util.Collection;
 import java.util.LinkedHashSet;
@@ -70,7 +70,7 @@ public final class EncryptPredicateColumnTokenGenerator extends BaseEncryptSQLTo
     private Collection<SubstitutableColumnNameToken> generateSQLTokens(final SQLStatementContext sqlStatementContext, final AndPredicate andPredicate) {
         Collection<SubstitutableColumnNameToken> result = new LinkedList<>();
         for (ExpressionSegment each : andPredicate.getPredicates()) {
-            Optional<ColumnSegment> column = ColumnExtractFromExpression.extract(each);
+            Optional<ColumnSegment> column = ColumnExtractor.extract(each);
             if (!column.isPresent()) {
                 continue;
             }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/main/java/org/apache/shardingsphere/shadow/condition/ShadowConditionEngine.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/main/java/org/apache/shardingsphere/shadow/condition/ShadowConditionEngine.java
index b475c27..106c95d 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/main/java/org/apache/shardingsphere/shadow/condition/ShadowConditionEngine.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/main/java/org/apache/shardingsphere/shadow/condition/ShadowConditionEngine.java
@@ -31,7 +31,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.S
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.AndPredicate;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.util.ExpressionBuilder;
-import org.apache.shardingsphere.sql.parser.sql.common.util.ColumnExtractFromExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.util.ColumnExtractor;
 
 import java.util.Collection;
 import java.util.HashSet;
@@ -83,7 +83,7 @@ public final class ShadowConditionEngine {
     
     private Optional<ShadowCondition> createShadowCondition(final AndPredicate andPredicate) {
         for (ExpressionSegment predicate : andPredicate.getPredicates()) {
-            Optional<ColumnSegment> column = ColumnExtractFromExpression.extract(predicate);
+            Optional<ColumnSegment> column = ColumnExtractor.extract(predicate);
             if (!column.isPresent()) {
                 continue;
             }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-rewrite/src/main/java/org/apache/shardingsphere/shadow/rewrite/token/generator/impl/ShadowPredicateColumnTokenGenerator.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-rewrite/src/main/java/org/apache/shardingsphere/shadow/rewrite/token/generator/impl/ShadowPredicateColumnTokenGenerator.java
index 3789d1f..40e83fb 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-rewrite/src/main/java/org/apache/shardingsphere/shadow/rewrite/token/generator/impl/ShadowPredicateColumnTokenGenerator.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-rewrite/src/main/java/org/apache/shardingsphere/shadow/rewrite/token/generator/impl/ShadowPredicateColumnTokenGenerator.java
@@ -30,7 +30,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.Expressi
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.AndPredicate;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.util.ExpressionBuilder;
-import org.apache.shardingsphere.sql.parser.sql.common.util.ColumnExtractFromExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.util.ColumnExtractor;
 
 import java.util.Collection;
 import java.util.LinkedList;
@@ -66,7 +66,7 @@ public final class ShadowPredicateColumnTokenGenerator extends BaseShadowSQLToke
         List<ExpressionSegment> predicates = (LinkedList<ExpressionSegment>) andPredicate.getPredicates();
         for (int i = 0; i < predicates.size(); i++) {
             ExpressionSegment expression = predicates.get(i);
-            Optional<ColumnSegment> column = ColumnExtractFromExpression.extract(expression);
+            Optional<ColumnSegment> column = ColumnExtractor.extract(expression);
             if (!column.isPresent()) {
                 continue;
             }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/api/sharding/standard/StandardShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/api/sharding/standard/StandardShardingAlgorithm.java
index 6254cc0..75fdd5b 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/api/sharding/standard/StandardShardingAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/api/sharding/standard/StandardShardingAlgorithm.java
@@ -31,18 +31,18 @@ public interface StandardShardingAlgorithm<T extends Comparable<?>> extends Shar
     /**
      * Sharding.
      * 
-     * @param availableTargetNames available data sources or tables's names
+     * @param availableTargetNames available data sources or table names
      * @param shardingValue sharding value
-     * @return sharding result for data source or table's name
+     * @return sharding result for data source or table name
      */
     String doSharding(Collection<String> availableTargetNames, PreciseShardingValue<T> shardingValue);
     
     /**
      * Sharding.
      *
-     * @param availableTargetNames available data sources or tables's names
+     * @param availableTargetNames available data sources or table names
      * @param shardingValue sharding value
-     * @return sharding results for data sources or tables's names
+     * @return sharding results for data sources or table names
      */
     Collection<String> doSharding(Collection<String> availableTargetNames, RangeShardingValue<T> shardingValue);
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
index ff784dd..ac25206 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.sharding.rule;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Splitter;
+import lombok.AccessLevel;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmFactory;
 import org.apache.shardingsphere.infra.config.exception.ShardingSphereConfigurationException;
@@ -26,18 +27,19 @@ import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.rule.DataNodeRoutedRule;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.type.TypedSPIRegistry;
+import org.apache.shardingsphere.sharding.algorithm.config.AlgorithmProvidedShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ComplexShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.NoneShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.sharding.ShardingAutoTableAlgorithm;
 import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
 import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
-import org.apache.shardingsphere.sharding.algorithm.config.AlgorithmProvidedShardingRuleConfiguration;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
-import org.apache.shardingsphere.sharding.strategy.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.ShardingStrategyFactory;
-import org.apache.shardingsphere.sharding.strategy.none.NoneShardingStrategy;
 
 import java.util.Collection;
 import java.util.LinkedHashMap;
@@ -72,14 +74,16 @@ public final class ShardingRule implements DataNodeRoutedRule {
     
     private final Collection<String> broadcastTables;
     
-    private final ShardingStrategy defaultDatabaseShardingStrategy;
+    @Getter(AccessLevel.NONE)
+    private final ShardingStrategyConfiguration defaultDatabaseShardingStrategyConfig;
     
-    private final ShardingStrategy defaultTableShardingStrategy;
+    @Getter(AccessLevel.NONE)
+    private final ShardingStrategyConfiguration defaultTableShardingStrategyConfig;
     
     private final KeyGenerateAlgorithm defaultKeyGenerateAlgorithm;
     
     public ShardingRule(final ShardingRuleConfiguration config, final Collection<String> dataSourceNames) {
-        Preconditions.checkArgument(null != config, "ShardingRuleConfig cannot be null.");
+        Preconditions.checkArgument(null != config, "Sharding rule configuration cannot be null.");
         Preconditions.checkArgument(null != dataSourceNames && !dataSourceNames.isEmpty(), "Data sources cannot be empty.");
         this.dataSourceNames = getDataSourceNames(config.getTables(), dataSourceNames);
         config.getShardingAlgorithms().forEach((key, value) -> shardingAlgorithms.put(key, ShardingSphereAlgorithmFactory.createAlgorithm(value, ShardingAlgorithm.class)));
@@ -88,14 +92,14 @@ public final class ShardingRule implements DataNodeRoutedRule {
         tableRules.addAll(createAutoTableRules(config.getAutoTables(), config.getDefaultKeyGenerateStrategy()));
         broadcastTables = config.getBroadcastTables();
         bindingTableRules = createBindingTableRules(config.getBindingTableGroups());
-        defaultDatabaseShardingStrategy = createDefaultShardingStrategy(config.getDefaultDatabaseShardingStrategy());
-        defaultTableShardingStrategy = createDefaultShardingStrategy(config.getDefaultTableShardingStrategy());
+        defaultDatabaseShardingStrategyConfig = null == config.getDefaultDatabaseShardingStrategy() ? new NoneShardingStrategyConfiguration() : config.getDefaultDatabaseShardingStrategy();
+        defaultTableShardingStrategyConfig = null == config.getDefaultTableShardingStrategy() ? new NoneShardingStrategyConfiguration() : config.getDefaultTableShardingStrategy();
         defaultKeyGenerateAlgorithm = null == config.getDefaultKeyGenerateStrategy()
                 ? TypedSPIRegistry.getRegisteredService(KeyGenerateAlgorithm.class) : keyGenerators.get(config.getDefaultKeyGenerateStrategy().getKeyGeneratorName());
     }
     
     public ShardingRule(final AlgorithmProvidedShardingRuleConfiguration config, final Collection<String> dataSourceNames) {
-        Preconditions.checkArgument(null != config, "ShardingRuleConfig cannot be null.");
+        Preconditions.checkArgument(null != config, "Sharding rule configuration cannot be null.");
         Preconditions.checkArgument(null != dataSourceNames && !dataSourceNames.isEmpty(), "Data sources cannot be empty.");
         this.dataSourceNames = getDataSourceNames(config.getTables(), dataSourceNames);
         shardingAlgorithms.putAll(config.getShardingAlgorithms());
@@ -104,8 +108,8 @@ public final class ShardingRule implements DataNodeRoutedRule {
         tableRules.addAll(createAutoTableRules(config.getAutoTables(), config.getDefaultKeyGenerateStrategy()));
         broadcastTables = config.getBroadcastTables();
         bindingTableRules = createBindingTableRules(config.getBindingTableGroups());
-        defaultDatabaseShardingStrategy = createDefaultShardingStrategy(config.getDefaultDatabaseShardingStrategy());
-        defaultTableShardingStrategy = createDefaultShardingStrategy(config.getDefaultTableShardingStrategy());
+        defaultDatabaseShardingStrategyConfig = null == config.getDefaultDatabaseShardingStrategy() ? new NoneShardingStrategyConfiguration() : config.getDefaultDatabaseShardingStrategy();
+        defaultTableShardingStrategyConfig = null == config.getDefaultTableShardingStrategy() ? new NoneShardingStrategyConfiguration() : config.getDefaultTableShardingStrategy();
         defaultKeyGenerateAlgorithm = null == config.getDefaultKeyGenerateStrategy()
                 ? TypedSPIRegistry.getRegisteredService(KeyGenerateAlgorithm.class) : keyGenerators.get(config.getDefaultKeyGenerateStrategy().getKeyGeneratorName());
     }
@@ -133,23 +137,17 @@ public final class ShardingRule implements DataNodeRoutedRule {
         return result;
     }
     
-    private Collection<TableRule> createTableRules(
-            final Collection<ShardingTableRuleConfiguration> tableRuleConfigurations, final KeyGenerateStrategyConfiguration defaultKeyGenerateStrategyConfig) {
-        return tableRuleConfigurations.stream().map(
-            each -> {
-                ShardingAlgorithm databaseShardingAlgorithm = null == each.getDatabaseShardingStrategy()
-                        ? null : shardingAlgorithms.get(each.getDatabaseShardingStrategy().getShardingAlgorithmName());
-                ShardingAlgorithm tableShardingAlgorithm = null == each.getTableShardingStrategy() ? null : shardingAlgorithms.get(each.getTableShardingStrategy().getShardingAlgorithmName());
-                return new TableRule(each, dataSourceNames, databaseShardingAlgorithm, tableShardingAlgorithm, getDefaultGenerateKeyColumn(defaultKeyGenerateStrategyConfig));
-            }).collect(Collectors.toList());
+    private Collection<TableRule> createTableRules(final Collection<ShardingTableRuleConfiguration> tableRuleConfigurations, final KeyGenerateStrategyConfiguration defaultKeyGenerateStrategyConfig) {
+        return tableRuleConfigurations.stream().map(each -> new TableRule(each, dataSourceNames, getDefaultGenerateKeyColumn(defaultKeyGenerateStrategyConfig))).collect(Collectors.toList());
     }
     
-    private Collection<TableRule> createAutoTableRules(
-            final Collection<ShardingAutoTableRuleConfiguration> autoTableRuleConfigurations, final KeyGenerateStrategyConfiguration defaultKeyGenerateStrategyConfig) {
+    private Collection<TableRule> createAutoTableRules(final Collection<ShardingAutoTableRuleConfiguration> autoTableRuleConfigurations, 
+                                                       final KeyGenerateStrategyConfiguration defaultKeyGenerateStrategyConfig) {
         return autoTableRuleConfigurations.stream().map(
             each -> {
                 ShardingAlgorithm shardingAlgorithm = null == each.getShardingStrategy() ? null : shardingAlgorithms.get(each.getShardingStrategy().getShardingAlgorithmName());
-                return new TableRule(each, dataSourceNames, shardingAlgorithm, getDefaultGenerateKeyColumn(defaultKeyGenerateStrategyConfig));
+                Preconditions.checkState(shardingAlgorithm instanceof ShardingAutoTableAlgorithm, "Sharding auto table rule configuration must match sharding auto table algorithm.");
+                return new TableRule(each, dataSourceNames, (ShardingAutoTableAlgorithm) shardingAlgorithm, getDefaultGenerateKeyColumn(defaultKeyGenerateStrategyConfig));
             }).collect(Collectors.toList());
     }
     
@@ -165,9 +163,24 @@ public final class ShardingRule implements DataNodeRoutedRule {
         return new BindingTableRule(Splitter.on(",").trimResults().splitToList(bindingTableGroup).stream().map(this::getTableRule).collect(Collectors.toList()));
     }
     
-    private ShardingStrategy createDefaultShardingStrategy(final ShardingStrategyConfiguration shardingStrategyConfig) {
-        return null == shardingStrategyConfig ? new NoneShardingStrategy()
-                : ShardingStrategyFactory.newInstance(shardingStrategyConfig, shardingAlgorithms.get(shardingStrategyConfig.getShardingAlgorithmName()));
+    /**
+     * Get database sharding strategy configuration.
+     * 
+     * @param tableRule table rule
+     * @return database sharding strategy configuration
+     */
+    public ShardingStrategyConfiguration getDatabaseShardingStrategyConfiguration(final TableRule tableRule) {
+        return null == tableRule.getDatabaseShardingStrategyConfig() ? defaultDatabaseShardingStrategyConfig : tableRule.getDatabaseShardingStrategyConfig();
+    }
+    
+    /**
+     * Get table sharding strategy configuration.
+     *
+     * @param tableRule table rule
+     * @return table sharding strategy configuration
+     */
+    public ShardingStrategyConfiguration getTableShardingStrategyConfiguration(final TableRule tableRule) {
+        return null == tableRule.getTableShardingStrategyConfig() ? defaultTableShardingStrategyConfig : tableRule.getTableShardingStrategyConfig();
     }
     
     /**
@@ -208,34 +221,6 @@ public final class ShardingRule implements DataNodeRoutedRule {
     }
     
     /**
-     * Get database sharding strategy.
-     *
-     * <p>
-     * Use default database sharding strategy if not found.
-     * </p>
-     *
-     * @param tableRule table rule
-     * @return database sharding strategy
-     */
-    public ShardingStrategy getDatabaseShardingStrategy(final TableRule tableRule) {
-        return null == tableRule.getDatabaseShardingStrategy() ? defaultDatabaseShardingStrategy : tableRule.getDatabaseShardingStrategy();
-    }
-    
-    /**
-     * Get table sharding strategy.
-     *
-     * <p>
-     * Use default table sharding strategy if not found.
-     * </p>
-     *
-     * @param tableRule table rule
-     * @return table sharding strategy
-     */
-    public ShardingStrategy getTableShardingStrategy(final TableRule tableRule) {
-        return null == tableRule.getTableShardingStrategy() ? defaultTableShardingStrategy : tableRule.getTableShardingStrategy();
-    }
-    
-    /**
      * Judge logic tables is all belong to binding encryptors.
      *
      * @param logicTableNames logic table names
@@ -310,9 +295,19 @@ public final class ShardingRule implements DataNodeRoutedRule {
     }
     
     private boolean isShardingColumn(final TableRule tableRule, final String columnName) {
-        return getDatabaseShardingStrategy(tableRule).getShardingColumns().contains(columnName) || getTableShardingStrategy(tableRule).getShardingColumns().contains(columnName);
+        return isShardingColumn(getDatabaseShardingStrategyConfiguration(tableRule), columnName) || isShardingColumn(getTableShardingStrategyConfiguration(tableRule), columnName);
     }
     
+    private boolean isShardingColumn(final ShardingStrategyConfiguration shardingStrategyConfig, final String columnName) {
+        if (shardingStrategyConfig instanceof StandardShardingStrategyConfiguration) {
+            return ((StandardShardingStrategyConfiguration) shardingStrategyConfig).getShardingColumn().equalsIgnoreCase(columnName);
+        }
+        if (shardingStrategyConfig instanceof ComplexShardingStrategyConfiguration) {
+            return ((ComplexShardingStrategyConfiguration) shardingStrategyConfig).getShardingColumns().contains(columnName);
+        }
+        return false;
+    } 
+    
     /**
      * Judge is generate key column or not.
      *
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/TableRule.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/TableRule.java
index 13551bf..4cfb5cf0 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/TableRule.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/TableRule.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.sharding.rule;
 
-import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import lombok.AccessLevel;
 import lombok.Getter;
@@ -26,16 +25,13 @@ import org.apache.shardingsphere.infra.config.exception.ShardingSphereConfigurat
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.datanode.DataNodeUtil;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.NoneShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.sharding.ShardingAutoTableAlgorithm;
-import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
-import org.apache.shardingsphere.sharding.strategy.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.ShardingStrategyFactory;
-import org.apache.shardingsphere.sharding.strategy.none.NoneShardingStrategy;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -66,9 +62,9 @@ public final class TableRule {
     @Getter(AccessLevel.NONE)
     private final Map<DataNode, Integer> dataNodeIndexMap;
     
-    private final ShardingStrategy databaseShardingStrategy;
+    private final ShardingStrategyConfiguration databaseShardingStrategyConfig;
     
-    private final ShardingStrategy tableShardingStrategy;
+    private final ShardingStrategyConfiguration tableShardingStrategyConfig;
     
     @Getter(AccessLevel.NONE)
     private final String generateKeyColumn;
@@ -84,21 +80,20 @@ public final class TableRule {
         dataNodeIndexMap = new HashMap<>(dataSourceNames.size(), 1);
         actualDataNodes = generateDataNodes(logicTableName, dataSourceNames);
         actualTables = getActualTables();
-        databaseShardingStrategy = null;
-        tableShardingStrategy = null;
+        databaseShardingStrategyConfig = null;
+        tableShardingStrategyConfig = null;
         generateKeyColumn = null;
         keyGeneratorName = null;
     }
     
-    public TableRule(final ShardingTableRuleConfiguration tableRuleConfig, final Collection<String> dataSourceNames, 
-                     final ShardingAlgorithm databaseShardingAlgorithm, final ShardingAlgorithm tableShardingAlgorithm, final String defaultGenerateKeyColumn) {
+    public TableRule(final ShardingTableRuleConfiguration tableRuleConfig, final Collection<String> dataSourceNames, final String defaultGenerateKeyColumn) {
         logicTable = tableRuleConfig.getLogicTable().toLowerCase();
         List<String> dataNodes = new InlineExpressionParser(tableRuleConfig.getActualDataNodes()).splitAndEvaluate();
         dataNodeIndexMap = new HashMap<>(dataNodes.size(), 1);
         actualDataNodes = isEmptyDataNodes(dataNodes) ? generateDataNodes(tableRuleConfig.getLogicTable(), dataSourceNames) : generateDataNodes(dataNodes, dataSourceNames);
         actualTables = getActualTables();
-        databaseShardingStrategy = createShardingStrategy(tableRuleConfig.getDatabaseShardingStrategy(), databaseShardingAlgorithm);
-        tableShardingStrategy = createShardingStrategy(tableRuleConfig.getTableShardingStrategy(), tableShardingAlgorithm);
+        databaseShardingStrategyConfig = tableRuleConfig.getDatabaseShardingStrategy();
+        tableShardingStrategyConfig = tableRuleConfig.getTableShardingStrategy();
         KeyGenerateStrategyConfiguration keyGeneratorConfig = tableRuleConfig.getKeyGenerateStrategy();
         generateKeyColumn = null != keyGeneratorConfig && !Strings.isNullOrEmpty(keyGeneratorConfig.getColumn()) ? keyGeneratorConfig.getColumn() : defaultGenerateKeyColumn;
         keyGeneratorName = null == keyGeneratorConfig ? null : keyGeneratorConfig.getKeyGeneratorName();
@@ -106,13 +101,11 @@ public final class TableRule {
     }
     
     public TableRule(final ShardingAutoTableRuleConfiguration tableRuleConfig, 
-                     final Collection<String> dataSourceNames, final ShardingAlgorithm shardingAlgorithm, final String defaultGenerateKeyColumn) {
+                     final Collection<String> dataSourceNames, final ShardingAutoTableAlgorithm shardingAutoTableAlgorithm, final String defaultGenerateKeyColumn) {
         logicTable = tableRuleConfig.getLogicTable().toLowerCase();
-        databaseShardingStrategy = new NoneShardingStrategy();
-        tableShardingStrategy = createShardingStrategy(tableRuleConfig.getShardingStrategy(), shardingAlgorithm);
-        Preconditions.checkArgument(null == tableRuleConfig.getShardingStrategy() || tableShardingStrategy.getShardingAlgorithm() instanceof ShardingAutoTableAlgorithm,
-                "ShardingAutoTableAlgorithm is required.");
-        List<String> dataNodes = getDataNodes(tableRuleConfig, dataSourceNames);
+        databaseShardingStrategyConfig = new NoneShardingStrategyConfiguration();
+        tableShardingStrategyConfig = tableRuleConfig.getShardingStrategy();
+        List<String> dataNodes = getDataNodes(tableRuleConfig, shardingAutoTableAlgorithm, dataSourceNames);
         dataNodeIndexMap = new HashMap<>(dataNodes.size(), 1);
         actualDataNodes = isEmptyDataNodes(dataNodes) ? generateDataNodes(tableRuleConfig.getLogicTable(), dataSourceNames) : generateDataNodes(dataNodes, dataSourceNames);
         actualTables = getActualTables();
@@ -122,20 +115,15 @@ public final class TableRule {
         checkRule(dataNodes);
     }
     
-    private ShardingStrategy createShardingStrategy(final ShardingStrategyConfiguration shardingStrategyConfig, final ShardingAlgorithm shardingAlgorithm) {
-        return null == shardingStrategyConfig ? null : ShardingStrategyFactory.newInstance(shardingStrategyConfig, shardingAlgorithm);
-    }
-    
-    private List<String> getDataNodes(final ShardingAutoTableRuleConfiguration tableRuleConfig, final Collection<String> dataSourceNames) {
-        if (null == tableShardingStrategy) {
+    private List<String> getDataNodes(final ShardingAutoTableRuleConfiguration tableRuleConfig, final ShardingAutoTableAlgorithm shardingAlgorithm, final Collection<String> dataSourceNames) {
+        if (null == tableShardingStrategyConfig) {
             return new LinkedList<>();
         }
         List<String> result = new LinkedList<>();
         List<String> dataSources = Strings.isNullOrEmpty(tableRuleConfig.getActualDataSources()) ? new LinkedList<>(dataSourceNames)
                 : new InlineExpressionParser(tableRuleConfig.getActualDataSources()).splitAndEvaluate();
-        ShardingAutoTableAlgorithm tableAlgorithm = (ShardingAutoTableAlgorithm) tableShardingStrategy.getShardingAlgorithm();
-        Iterator iterator = dataSources.iterator();
-        for (int i = 0; i < tableAlgorithm.getAutoTablesAmount(); i++) {
+        Iterator<String> iterator = dataSources.iterator();
+        for (int i = 0; i < shardingAlgorithm.getAutoTablesAmount(); i++) {
             if (!iterator.hasNext()) {
                 iterator = dataSources.iterator();
             }
@@ -224,7 +212,7 @@ public final class TableRule {
     }
     
     private void checkRule(final List<String> dataNodes) {
-        if (isEmptyDataNodes(dataNodes) && null != tableShardingStrategy && !(tableShardingStrategy instanceof NoneShardingStrategy)) {
+        if (isEmptyDataNodes(dataNodes) && null != tableShardingStrategyConfig && !(tableShardingStrategyConfig instanceof NoneShardingStrategyConfiguration)) {
             throw new ShardingSphereConfigurationException("ActualDataNodes must be configured if want to shard tables for logicTable [%s]", logicTable);
         }
     }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/AutoIntervalShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/AutoIntervalShardingAlgorithmTest.java
index 72a8957..725ad80 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/AutoIntervalShardingAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/AutoIntervalShardingAlgorithmTest.java
@@ -18,19 +18,14 @@
 package org.apache.shardingsphere.sharding.algorithm.sharding.datetime;
 
 import com.google.common.collect.Range;
-import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.RangeShardingValue;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
-import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
@@ -38,61 +33,40 @@ import static org.junit.Assert.assertTrue;
 
 public final class AutoIntervalShardingAlgorithmTest {
     
-    private StandardShardingStrategy shardingStrategy;
+    private AutoIntervalShardingAlgorithm shardingAlgorithm;
     
     @Before
     public void setup() {
-        AutoIntervalShardingAlgorithm shardingAlgorithm = new AutoIntervalShardingAlgorithm();
+        shardingAlgorithm = new AutoIntervalShardingAlgorithm();
         shardingAlgorithm.getProps().setProperty("datetime-lower", "2020-01-01 00:00:00");
         shardingAlgorithm.getProps().setProperty("datetime-upper", "2020-01-01 00:00:16");
         shardingAlgorithm.getProps().setProperty("sharding-seconds", "4");
         shardingAlgorithm.init();
-        shardingStrategy = new StandardShardingStrategy("create_time", shardingAlgorithm);
     }
     
     @Test
     public void assertPreciseDoSharding() {
         List<String> availableTargetNames = Arrays.asList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
-        List<RouteValue> shardingValues = Collections.singletonList(new ListRouteValue<>("create_time", "t_order", Arrays.asList("2020-01-01 00:00:01", "2020-01-01 00:00:02")));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
-        assertThat(actual.size(), is(1));
-        assertTrue(actual.contains("t_order_1"));
-    }
-    
-    @Test
-    public void assertPreciseDoShardingWithFirstPartition() {
-        List<String> availableTargetNames = Arrays.asList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
-        List<RouteValue> shardingValues = Collections.singletonList(new ListRouteValue<>("create_time", "t_order", Arrays.asList("2019-12-01 00:00:01", "2020-01-01 00:00:02")));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
-        assertThat(actual.size(), is(2));
-        assertTrue(actual.contains("t_order_0"));
-        assertTrue(actual.contains("t_order_1"));
+        assertThat(shardingAlgorithm.doSharding(availableTargetNames, new PreciseShardingValue<>("t_order", "create_time", "2020-01-01 00:00:01")), is("t_order_1"));
     }
     
     @Test
     public void assertPreciseDoShardingBeyondTheLastOne() {
         List<String> availableTargetNames = Arrays.asList("t_order_0", "t_order_1", "t_order_2", "t_order_3", "t_order_4", "t_order_5");
-        List<RouteValue> shardingValues = Collections.singletonList(new ListRouteValue<>("create_time", "t_order", Collections.singletonList("2021-01-01 00:00:02")));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
-        assertThat(actual.size(), is(1));
-        assertTrue(actual.contains("t_order_5"));
+        assertThat(shardingAlgorithm.doSharding(availableTargetNames, new PreciseShardingValue<>("t_order", "create_time", "2021-01-01 00:00:02")), is("t_order_5"));
     }
     
     @Test
     public void assertRangeDoShardingWithAllRange() {
         List<String> availableTargetNames = Arrays.asList("t_order_0", "t_order_1", "t_order_2", "t_order_3", "t_order_4");
-        Range<String> rangeValue = Range.closed("2019-01-01 00:00:00", "2020-01-01 00:00:15");
-        List<RouteValue> shardingValues = Collections.singletonList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new RangeShardingValue<>("t_order", "create_time", Range.closed("2019-01-01 00:00:00", "2020-01-01 00:00:15")));
         assertThat(actual.size(), is(5));
     }
     
     @Test
     public void assertRangeDoShardingWithPartRange() {
-        List<String> availableTargetNames = Arrays.asList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
-        Range<String> rangeValue = Range.closed("2020-01-01 00:00:04", "2020-01-01 00:00:10");
-        List<RouteValue> shardingValues = Collections.singletonList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
+        List<String> availableTargetNames = Arrays.asList("t_order_0", "t_order_1", "t_order_2", "t_order_3", "t_order_4");
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new RangeShardingValue<>("t_order", "create_time", Range.closed("2020-01-01 00:00:04", "2020-01-01 00:00:10")));
         assertThat(actual.size(), is(3));
         assertTrue(actual.contains("t_order_1"));
         assertTrue(actual.contains("t_order_2"));
@@ -102,9 +76,7 @@ public final class AutoIntervalShardingAlgorithmTest {
     @Test
     public void assertRangeDoShardingWithoutLowerBound() {
         List<String> availableTargetNames = Arrays.asList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
-        Range<String> rangeValue = Range.lessThan("2020-01-01 00:00:11");
-        List<RouteValue> shardingValues = Collections.singletonList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new RangeShardingValue<>("t_order", "create_time", Range.lessThan("2020-01-01 00:00:11")));
         assertThat(actual.size(), is(4));
         assertTrue(actual.contains("t_order_0"));
         assertTrue(actual.contains("t_order_1"));
@@ -115,9 +87,7 @@ public final class AutoIntervalShardingAlgorithmTest {
     @Test
     public void assertRangeDoShardingWithoutUpperBound() {
         List<String> availableTargetNames = Arrays.asList("t_order_0", "t_order_1", "t_order_2", "t_order_3", "t_order_4", "t_order_5");
-        Range<String> rangeValue = Range.greaterThan("2020-01-01 00:00:09");
-        List<RouteValue> shardingValues = Collections.singletonList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new RangeShardingValue<>("t_order", "create_time", Range.greaterThan("2020-01-01 00:00:09")));
         assertThat(actual.size(), is(3));
         assertTrue(actual.contains("t_order_3"));
         assertTrue(actual.contains("t_order_4"));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/IntervalShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/IntervalShardingAlgorithmTest.java
index ad98c07..8feaccc 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/IntervalShardingAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/IntervalShardingAlgorithmTest.java
@@ -18,36 +18,28 @@
 package org.apache.shardingsphere.sharding.algorithm.sharding.datetime;
 
 import com.google.common.collect.Range;
-import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.RangeShardingValue;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.time.LocalDateTime;
 import java.time.format.DateTimeFormatter;
-import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 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.junit.Assert.assertTrue;
 
 public final class IntervalShardingAlgorithmTest {
     
-    private final Collection<String> availableTablesForMonthStrategy = new LinkedList<>();
+    private final Collection<String> availableTablesForQuarterDataSources = new LinkedList<>();
     
-    private final Collection<String> availableTablesForQuarterStrategy = new LinkedList<>();
+    private final Collection<String> availableTablesForMonthDataSources = new LinkedList<>();
     
-    private StandardShardingStrategy shardingStrategyByMonth;
+    private IntervalShardingAlgorithm shardingAlgorithmByQuarter;
     
-    private StandardShardingStrategy shardingStrategyByQuarter;
+    private IntervalShardingAlgorithm shardingAlgorithmByMonth;
     
     @Before
     public void setup() {
@@ -56,86 +48,72 @@ public final class IntervalShardingAlgorithmTest {
     }
     
     private void initShardStrategyByQuarter() {
-        IntervalShardingAlgorithm shardingAlgorithm = new IntervalShardingAlgorithm();
-        shardingAlgorithm.getProps().setProperty("datetime-pattern", "yyyy-MM-dd HH:mm:ss");
-        shardingAlgorithm.getProps().setProperty("datetime-lower", "2016-01-01 00:00:00");
-        shardingAlgorithm.getProps().setProperty("datetime-upper", "2021-12-31 00:00:00");
-        shardingAlgorithm.getProps().setProperty("sharding-suffix-pattern", "yyyyQQ");
-        shardingAlgorithm.getProps().setProperty("datetime-interval-amount", "3");
-        shardingAlgorithm.getProps().setProperty("datetime-interval-unit", "Months");
-        shardingAlgorithm.init();
-        shardingStrategyByQuarter = new StandardShardingStrategy("create_time", shardingAlgorithm);
+        shardingAlgorithmByQuarter = new IntervalShardingAlgorithm();
+        shardingAlgorithmByQuarter.getProps().setProperty("datetime-pattern", "yyyy-MM-dd HH:mm:ss");
+        shardingAlgorithmByQuarter.getProps().setProperty("datetime-lower", "2016-01-01 00:00:00");
+        shardingAlgorithmByQuarter.getProps().setProperty("datetime-upper", "2021-12-31 00:00:00");
+        shardingAlgorithmByQuarter.getProps().setProperty("sharding-suffix-pattern", "yyyyQQ");
+        shardingAlgorithmByQuarter.getProps().setProperty("datetime-interval-amount", "3");
+        shardingAlgorithmByQuarter.getProps().setProperty("datetime-interval-unit", "Months");
+        shardingAlgorithmByQuarter.init();
         for (int i = 2016; i <= 2020; i++) {
             for (int j = 1; j <= 4; j++) {
-                availableTablesForQuarterStrategy.add(String.format("t_order_%04d%02d", i, j));
+                availableTablesForQuarterDataSources.add(String.format("t_order_%04d%02d", i, j));
             }
         }
     }
     
     private void initShardStrategyByMonth() {
-        IntervalShardingAlgorithm shardingAlgorithm = new IntervalShardingAlgorithm();
-        shardingAlgorithm.getProps().setProperty("datetime-pattern", "yyyy-MM-dd HH:mm:ss");
-        shardingAlgorithm.getProps().setProperty("datetime-lower", "2016-01-01 00:00:00");
-        shardingAlgorithm.getProps().setProperty("datetime-upper", "2021-12-31 00:00:00");
-        shardingAlgorithm.getProps().setProperty("sharding-suffix-pattern", "yyyyMM");
-        shardingAlgorithm.getProps().setProperty("datetime-interval-amount", "1");
-        shardingAlgorithm.getProps().setProperty("datetime-interval-unit", "Months");
-        shardingAlgorithm.init();
-        shardingStrategyByMonth = new StandardShardingStrategy("create_time", shardingAlgorithm);
+        shardingAlgorithmByMonth = new IntervalShardingAlgorithm();
+        shardingAlgorithmByMonth.getProps().setProperty("datetime-pattern", "yyyy-MM-dd HH:mm:ss");
+        shardingAlgorithmByMonth.getProps().setProperty("datetime-lower", "2016-01-01 00:00:00");
+        shardingAlgorithmByMonth.getProps().setProperty("datetime-upper", "2021-12-31 00:00:00");
+        shardingAlgorithmByMonth.getProps().setProperty("sharding-suffix-pattern", "yyyyMM");
+        shardingAlgorithmByMonth.getProps().setProperty("datetime-interval-amount", "1");
+        shardingAlgorithmByMonth.getProps().setProperty("datetime-interval-unit", "Months");
+        shardingAlgorithmByMonth.init();
         for (int i = 2016; i <= 2020; i++) {
             for (int j = 1; j <= 12; j++) {
-                availableTablesForMonthStrategy.add(String.format("t_order_%04d%02d", i, j));
+                availableTablesForMonthDataSources.add(String.format("t_order_%04d%02d", i, j));
             }
         }
     }
     
     @Test
     public void assertPreciseDoShardingByQuarter() {
-        List<RouteValue> shardingValues = Collections.singletonList(new ListRouteValue<>("create_time", "t_order", Arrays.asList("2020-01-01 00:00:01", "2020-01-01 00:00:02", "2020-04-15 10:59:08")));
-        Collection<String> actual = shardingStrategyByQuarter.doSharding(availableTablesForQuarterStrategy, shardingValues, new ConfigurationProperties(new Properties()));
-        assertThat(actual.size(), is(2));
-        assertTrue(actual.contains("t_order_202001"));
-        assertTrue(actual.contains("t_order_202002"));
+        assertThat(shardingAlgorithmByQuarter.doSharding(availableTablesForQuarterDataSources, new PreciseShardingValue<>("t_order", "create_time", "2020-01-01 00:00:01")), is("t_order_202001"));
     }
     
     @Test
     public void assertRangeDoShardingByQuarter() {
-        Range<String> rangeValue = Range.closed("2019-10-15 10:59:08", "2020-04-08 10:59:08");
-        List<RouteValue> shardingValues = Collections.singletonList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
-        Collection<String> actual = shardingStrategyByQuarter.doSharding(availableTablesForQuarterStrategy, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithmByQuarter.doSharding(
+                availableTablesForQuarterDataSources, new RangeShardingValue<>("t_order", "create_time", Range.closed("2019-10-15 10:59:08", "2020-04-08 10:59:08")));
         assertThat(actual.size(), is(3));
     }
     
     @Test
     public void assertPreciseDoShardingByMonth() {
-        List<RouteValue> shardingValues = Collections.singletonList(new ListRouteValue<>("create_time", "t_order", Arrays.asList("2020-01-01 00:00:01", "2020-01-01 00:00:02", "2020-04-15 10:59:08")));
-        Collection<String> actual = shardingStrategyByMonth.doSharding(availableTablesForMonthStrategy, shardingValues, new ConfigurationProperties(new Properties()));
-        assertThat(actual.size(), is(2));
-        assertTrue(actual.contains("t_order_202001"));
-        assertTrue(actual.contains("t_order_202004"));
+        assertThat(shardingAlgorithmByMonth.doSharding(availableTablesForMonthDataSources, new PreciseShardingValue<>("t_order", "create_time", "2020-01-01 00:00:01")), is("t_order_202001"));
     }
     
     @Test
     public void assertRangeDoShardingByMonth() {
-        Range<String> rangeValue = Range.closed("2019-10-15 10:59:08", "2020-04-08 10:59:08");
-        List<RouteValue> shardingValues = Collections.singletonList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
-        Collection<String> actual = shardingStrategyByMonth.doSharding(availableTablesForMonthStrategy, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithmByMonth.doSharding(
+                availableTablesForMonthDataSources, new RangeShardingValue<>("t_order", "create_time", Range.closed("2019-10-15 10:59:08", "2020-04-08 10:59:08")));
         assertThat(actual.size(), is(7));
     }
     
     @Test
     public void assertLowerHalfRangeDoSharding() {
-        Range<String> rangeValue = Range.atLeast("2018-10-15 10:59:08");
-        List<RouteValue> shardingValues = Collections.singletonList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
-        Collection<String> actual = shardingStrategyByQuarter.doSharding(availableTablesForQuarterStrategy, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithmByQuarter.doSharding(
+                availableTablesForQuarterDataSources, new RangeShardingValue<>("t_order", "create_time", Range.atLeast("2018-10-15 10:59:08")));
         assertThat(actual.size(), is(9));
     }
     
     @Test
     public void assertUpperHalfRangeDoSharding() {
-        Range<String> rangeValue = Range.atMost("2019-09-01 00:00:00");
-        List<RouteValue> shardingValues = Collections.singletonList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
-        Collection<String> actual = shardingStrategyByQuarter.doSharding(availableTablesForQuarterStrategy, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithmByQuarter.doSharding(
+                availableTablesForQuarterDataSources, new RangeShardingValue<>("t_order", "create_time", Range.atMost("2019-09-01 00:00:00")));
         assertThat(actual.size(), is(15));
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineShardingAlgorithmTest.java
index 0a8ab5d..ab80621 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineShardingAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineShardingAlgorithmTest.java
@@ -19,17 +19,13 @@ package org.apache.shardingsphere.sharding.algorithm.sharding.inline;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Range;
-import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.RangeShardingValue;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.util.Collection;
 import java.util.List;
-import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
@@ -37,65 +33,47 @@ import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 
 public final class InlineShardingAlgorithmTest {
-
-    private StandardShardingStrategy shardingStrategy;
-
-    private StandardShardingStrategy shardingStrategyWithSimplified;
+    
+    private InlineShardingAlgorithm inlineShardingAlgorithm;
+    
+    private InlineShardingAlgorithm inlineShardingAlgorithmWithSimplified;
     
     @Before
     public void setUp() {
-        shardingStrategy = createShardingStrategy();
-        shardingStrategyWithSimplified = createShardingStrategyWithSimplified();
+        initInlineShardingAlgorithm();
+        initInlineShardingAlgorithmWithSimplified();
     }
     
-    private StandardShardingStrategy createShardingStrategy() {
-        InlineShardingAlgorithm shardingAlgorithm = new InlineShardingAlgorithm();
-        shardingAlgorithm.getProps().setProperty("algorithm-expression", "t_order_$->{order_id % 4}");
-        shardingAlgorithm.getProps().setProperty("allow-range-query-with-inline-sharding", "true");
-        shardingAlgorithm.init();
-        return new StandardShardingStrategy("order_id", shardingAlgorithm);
+    private void initInlineShardingAlgorithm() {
+        inlineShardingAlgorithm = new InlineShardingAlgorithm();
+        inlineShardingAlgorithm.getProps().setProperty("algorithm-expression", "t_order_$->{order_id % 4}");
+        inlineShardingAlgorithm.getProps().setProperty("allow-range-query-with-inline-sharding", "true");
+        inlineShardingAlgorithm.init();
     }
     
-    private StandardShardingStrategy createShardingStrategyWithSimplified() {
-        InlineShardingAlgorithm shardingAlgorithmWithSimplified = new InlineShardingAlgorithm();
-        shardingAlgorithmWithSimplified.getProps().setProperty("algorithm-expression", "t_order_${order_id % 4}");
-        shardingAlgorithmWithSimplified.init();
-        return new StandardShardingStrategy("order_id", shardingAlgorithmWithSimplified);
+    private void initInlineShardingAlgorithmWithSimplified() {
+        inlineShardingAlgorithmWithSimplified = new InlineShardingAlgorithm();
+        inlineShardingAlgorithmWithSimplified.getProps().setProperty("algorithm-expression", "t_order_${order_id % 4}");
+        inlineShardingAlgorithmWithSimplified.init();
     }
     
     @Test
     public void assertDoSharding() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
-        List<RouteValue> shardingValues = Lists.newArrayList(new ListRouteValue<>("order_id", "t_order", Lists.newArrayList(0, 1, 2, 3)));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
-        assertThat(actual.size(), is(4));
-        Collection<String> actualWithSimplified = shardingStrategyWithSimplified.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
-        assertThat(actualWithSimplified.size(), is(4));
+        assertThat(inlineShardingAlgorithm.doSharding(availableTargetNames, new PreciseShardingValue<>("t_order", "order_id", 0)), is("t_order_0"));
     }
     
     @Test
-    public void assertDoShardingWithRangeRouteValue() {
+    public void assertDoShardingWithRangeShardingConditionValue() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
-        List<RouteValue> shardingValues = Lists.newArrayList(new RangeRouteValue<>("order_id", "t_order", mock(Range.class)));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = inlineShardingAlgorithm.doSharding(availableTargetNames, new RangeShardingValue<>("t_order", "order_id", mock(Range.class)));
         assertTrue(actual.containsAll(availableTargetNames));
     }
     
     @Test
     public void assertDoShardingWithNonExistNodes() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1");
-        List<RouteValue> shardingValues = Lists.newArrayList(new ListRouteValue<>("order_id", "t_order", Lists.newArrayList(0, 1, 2, 3)));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
-        assertThat(actual.size(), is(2));
-        Collection<String> actualWithSimplified = shardingStrategyWithSimplified.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
-        assertThat(actualWithSimplified.size(), is(2));
-    }
-    
-    @Test
-    public void assertGetShardingColumns() {
-        assertThat(shardingStrategy.getShardingColumns().size(), is(1));
-        assertThat(shardingStrategy.getShardingColumns().iterator().next(), is("order_id"));
-        assertThat(shardingStrategyWithSimplified.getShardingColumns().size(), is(1));
-        assertThat(shardingStrategyWithSimplified.getShardingColumns().iterator().next(), is("order_id"));
+        assertThat(inlineShardingAlgorithm.doSharding(availableTargetNames, new PreciseShardingValue<>("t_order", "order_id", 0)), is("t_order_0"));
+        assertThat(inlineShardingAlgorithmWithSimplified.doSharding(availableTargetNames, new PreciseShardingValue<>("t_order", "order_id", 0)), is("t_order_0"));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/HashModShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/HashModShardingAlgorithmTest.java
index 88eb0ce..7441e0f 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/HashModShardingAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/HashModShardingAlgorithmTest.java
@@ -19,51 +19,38 @@ package org.apache.shardingsphere.sharding.algorithm.sharding.mod;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Range;
-import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.RangeShardingValue;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.util.Collection;
 import java.util.List;
-import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
 
 public final class HashModShardingAlgorithmTest {
     
-    private StandardShardingStrategy shardingStrategy;
+    private HashModShardingAlgorithm shardingAlgorithm;
     
     @Before
     public void setup() {
-        HashModShardingAlgorithm shardingAlgorithm = new HashModShardingAlgorithm();
+        shardingAlgorithm = new HashModShardingAlgorithm();
         shardingAlgorithm.getProps().setProperty("sharding-count", "4");
         shardingAlgorithm.init();
-        shardingStrategy = new StandardShardingStrategy("order_type", shardingAlgorithm);
     }
     
     @Test
     public void assertPreciseDoSharding() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
-        List<RouteValue> shardingValues = Lists.newArrayList(new ListRouteValue<>("order_type", "t_order", Lists.newArrayList("a", "b", "c")));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
-        assertThat(actual.size(), is(3));
-        assertTrue(actual.contains("t_order_1"));
-        assertTrue(actual.contains("t_order_2"));
-        assertTrue(actual.contains("t_order_3"));
+        assertThat(shardingAlgorithm.doSharding(availableTargetNames, new PreciseShardingValue<>("t_order", "order_type", "a")), is("t_order_1"));
     }
     
     @Test
     public void assertRangeDoSharding() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
-        Range<String> rangeValue = Range.closed("a", "f");
-        List<RouteValue> shardingValues = Lists.newArrayList(new RangeRouteValue<>("order_type", "t_order", rangeValue));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new RangeShardingValue<>("t_order", "create_time", Range.closed("a", "f")));
         assertThat(actual.size(), is(4));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/ModShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/ModShardingAlgorithmTest.java
index 4ff1541..816939e 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/ModShardingAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/ModShardingAlgorithmTest.java
@@ -19,17 +19,13 @@ package org.apache.shardingsphere.sharding.algorithm.sharding.mod;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Range;
-import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.RangeShardingValue;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.util.Collection;
 import java.util.List;
-import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
@@ -37,42 +33,32 @@ import static org.junit.Assert.assertTrue;
 
 public final class ModShardingAlgorithmTest {
     
-    private StandardShardingStrategy shardingStrategy;
+    private ModShardingAlgorithm shardingAlgorithm;
     
     @Before
     public void setup() {
-        ModShardingAlgorithm shardingAlgorithm = new ModShardingAlgorithm();
+        shardingAlgorithm = new ModShardingAlgorithm();
         shardingAlgorithm.getProps().setProperty("sharding-count", "4");
         shardingAlgorithm.init();
-        shardingStrategy = new StandardShardingStrategy("order_id", shardingAlgorithm);
     }
     
     @Test
     public void assertPreciseDoSharding() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
-        List<RouteValue> shardingValues = Lists.newArrayList(new ListRouteValue<>("order_id", "t_order", Lists.newArrayList(10L, 11L, 13L)));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
-        assertThat(actual.size(), is(3));
-        assertTrue(actual.contains("t_order_1"));
-        assertTrue(actual.contains("t_order_2"));
-        assertTrue(actual.contains("t_order_3"));
+        assertThat(shardingAlgorithm.doSharding(availableTargetNames, new PreciseShardingValue<>("t_order", "order_id", 13L)), is("t_order_1"));
     }
     
     @Test
     public void assertRangeDoShardingWithAllTargets() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
-        Range<Long> rangeValue = Range.closed(11L, 14L);
-        List<RouteValue> shardingValues = Lists.newArrayList(new RangeRouteValue<>("order_id", "t_order", rangeValue));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new RangeShardingValue<>("t_order", "order_id", Range.closed(11L, 14L)));
         assertThat(actual.size(), is(4));
     }
     
     @Test
     public void assertRangeDoShardingWithPartTargets() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
-        Range<Long> rangeValue = Range.closed(11L, 12L);
-        List<RouteValue> shardingValues = Lists.newArrayList(new RangeRouteValue<>("order_id", "t_order", rangeValue));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new RangeShardingValue<>("t_order", "order_id", Range.closed(11L, 12L)));
         assertThat(actual.size(), is(2));
         assertTrue(actual.contains("t_order_3"));
         assertTrue(actual.contains("t_order_0"));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/BoundaryBasedRangeShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/BoundaryBasedRangeShardingAlgorithmTest.java
index ed8f65e..9a0f7e1 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/BoundaryBasedRangeShardingAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/BoundaryBasedRangeShardingAlgorithmTest.java
@@ -19,17 +19,13 @@ package org.apache.shardingsphere.sharding.algorithm.sharding.range;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Range;
-import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.RangeShardingValue;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.util.Collection;
 import java.util.List;
-import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
@@ -37,32 +33,25 @@ import static org.junit.Assert.assertTrue;
 
 public final class BoundaryBasedRangeShardingAlgorithmTest {
     
-    private StandardShardingStrategy shardingStrategy;
+    private BoundaryBasedRangeShardingAlgorithm shardingAlgorithm;
     
     @Before
     public void setUp() {
-        BoundaryBasedRangeShardingAlgorithm shardingAlgorithm = new BoundaryBasedRangeShardingAlgorithm();
+        shardingAlgorithm = new BoundaryBasedRangeShardingAlgorithm();
         shardingAlgorithm.getProps().setProperty("sharding-ranges", "1,5,10");
         shardingAlgorithm.init();
-        shardingStrategy = new StandardShardingStrategy("order_id", shardingAlgorithm);
     }
     
     @Test
     public void assertPreciseDoSharding() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
-        List<RouteValue> shardingValues = Lists.newArrayList(new ListRouteValue<>("order_id", "t_order", Lists.newArrayList(0L, 1L, 2L, 4L, 17L)));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
-        assertThat(actual.size(), is(3));
-        assertTrue(actual.contains("t_order_0"));
-        assertTrue(actual.contains("t_order_1"));
-        assertTrue(actual.contains("t_order_3"));
+        assertThat(shardingAlgorithm.doSharding(availableTargetNames, new PreciseShardingValue<>("t_order", "order_id", 0L)), is("t_order_0"));
     }
     
     @Test
     public void assertRangeDoSharding() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
-        List<RouteValue> shardingValues = Lists.newArrayList(new RangeRouteValue<>("order_id", "t_order", Range.closed(2L, 15L)));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new RangeShardingValue<>("t_order", "order_id", Range.closed(2L, 15L)));
         assertThat(actual.size(), is(3));
         assertTrue(actual.contains("t_order_1"));
         assertTrue(actual.contains("t_order_2"));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/VolumeBasedRangeShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/VolumeBasedRangeShardingAlgorithmTest.java
index 4782f64..2750e94 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/VolumeBasedRangeShardingAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/VolumeBasedRangeShardingAlgorithmTest.java
@@ -19,17 +19,13 @@ package org.apache.shardingsphere.sharding.algorithm.sharding.range;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Range;
-import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.RangeShardingValue;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.util.Collection;
 import java.util.List;
-import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
@@ -37,35 +33,27 @@ import static org.junit.Assert.assertTrue;
 
 public final class VolumeBasedRangeShardingAlgorithmTest {
     
-    private StandardShardingStrategy shardingStrategy;
+    private VolumeBasedRangeShardingAlgorithm shardingAlgorithm;
     
     @Before
     public void setUp() {
-        VolumeBasedRangeShardingAlgorithm shardingAlgorithm = new VolumeBasedRangeShardingAlgorithm();
+        shardingAlgorithm = new VolumeBasedRangeShardingAlgorithm();
         shardingAlgorithm.getProps().setProperty("range-lower", "10");
         shardingAlgorithm.getProps().setProperty("range-upper", "45");
         shardingAlgorithm.getProps().setProperty("sharding-volume", "10");
         shardingAlgorithm.init();
-        shardingStrategy = new StandardShardingStrategy("order_id", shardingAlgorithm);
     }
     
     @Test
     public void assertPreciseDoSharding() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3", "t_order_4", "t_order_5");
-        List<RouteValue> shardingValues = Lists.newArrayList(new ListRouteValue<>("order_id", "t_order", Lists.newArrayList(0L, 1L, 2L, 4L, 17L, 25L, 45L)));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
-        assertThat(actual.size(), is(4));
-        assertTrue(actual.contains("t_order_0"));
-        assertTrue(actual.contains("t_order_1"));
-        assertTrue(actual.contains("t_order_2"));
-        assertTrue(actual.contains("t_order_5"));
+        assertThat(shardingAlgorithm.doSharding(availableTargetNames, new PreciseShardingValue<>("t_order", "order_id", 0L)), is("t_order_0"));
     }
     
     @Test
     public void assertRangeDoShardingWithoutLowerBound() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3", "t_order_4", "t_order_5");
-        List<RouteValue> shardingValues = Lists.newArrayList(new RangeRouteValue<>("order_id", "t_order", Range.lessThan(12L)));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new RangeShardingValue<>("t_order", "order_id", Range.lessThan(12L)));
         assertThat(actual.size(), is(2));
         assertTrue(actual.contains("t_order_0"));
         assertTrue(actual.contains("t_order_1"));
@@ -74,8 +62,7 @@ public final class VolumeBasedRangeShardingAlgorithmTest {
     @Test
     public void assertRangeDoShardingWithoutUpperBound() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3", "t_order_4", "t_order_5");
-        List<RouteValue> shardingValues = Lists.newArrayList(new RangeRouteValue<>("order_id", "t_order", Range.greaterThan(40L)));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new RangeShardingValue<>("t_order", "order_id", Range.greaterThan(40L)));
         assertThat(actual.size(), is(2));
         assertTrue(actual.contains("t_order_4"));
         assertTrue(actual.contains("t_order_5"));
@@ -84,8 +71,7 @@ public final class VolumeBasedRangeShardingAlgorithmTest {
     @Test
     public void assertRangeDoSharding() {
         List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3", "t_order_4", "t_order_5");
-        List<RouteValue> shardingValues = Lists.newArrayList(new RangeRouteValue<>("order_id", "t_order", Range.closed(12L, 55L)));
-        Collection<String> actual = shardingStrategy.doSharding(availableTargetNames, shardingValues, new ConfigurationProperties(new Properties()));
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new RangeShardingValue<>("t_order", "order_id", Range.closed(12L, 55L)));
         assertThat(actual.size(), is(5));
         assertTrue(actual.contains("t_order_1"));
         assertTrue(actual.contains("t_order_2"));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/BindingTableRuleTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/BindingTableRuleTest.java
index cdaebb6..77541d0 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/BindingTableRuleTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/BindingTableRuleTest.java
@@ -74,10 +74,10 @@ public final class BindingTableRuleTest {
     }
     
     private TableRule createTableRule() {
-        return new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..1}"), Arrays.asList("ds0", "ds1"), null, null, null);
+        return new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..1}"), Arrays.asList("ds0", "ds1"), null);
     }
     
     private TableRule createSubTableRule() {
-        return new TableRule(new ShardingTableRuleConfiguration("SUB_LOGIC_TABLE", "ds${0..1}.sub_table_${0..1}"), Arrays.asList("ds0", "ds1"), null, null, null);
+        return new TableRule(new ShardingTableRuleConfiguration("SUB_LOGIC_TABLE", "ds${0..1}.sub_table_${0..1}"), Arrays.asList("ds0", "ds1"), null);
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
index d58ce0e..d29e7aa 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
@@ -20,17 +20,14 @@ package org.apache.shardingsphere.sharding.rule;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.exception.ShardingSphereConfigurationException;
 import org.apache.shardingsphere.infra.datanode.DataNode;
+import org.apache.shardingsphere.sharding.algorithm.keygen.SnowflakeKeyGenerateAlgorithm;
+import org.apache.shardingsphere.sharding.algorithm.keygen.fixture.IncrementKeyGenerateAlgorithm;
+import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineShardingAlgorithm;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.NoneShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
-import org.apache.shardingsphere.sharding.algorithm.keygen.SnowflakeKeyGenerateAlgorithm;
-import org.apache.shardingsphere.sharding.algorithm.keygen.fixture.IncrementKeyGenerateAlgorithm;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.none.NoneShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
 import org.junit.Test;
 
 import java.util.Arrays;
@@ -43,8 +40,6 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 public final class ShardingRuleTest {
     
@@ -60,8 +55,6 @@ public final class ShardingRuleTest {
         assertThat(actual.getBindingTableRules().size(), is(1));
         assertThat(actual.getBindingTableRules().iterator().next().getTableRules().size(), is(2));
         assertThat(actual.getBroadcastTables(), is(Collections.singletonList("BROADCAST_TABLE")));
-        assertThat(actual.getDefaultDatabaseShardingStrategy(), instanceOf(StandardShardingStrategy.class));
-        assertThat(actual.getDefaultTableShardingStrategy(), instanceOf(StandardShardingStrategy.class));
         assertThat(actual.getDefaultKeyGenerateAlgorithm(), instanceOf(IncrementKeyGenerateAlgorithm.class));
     }
     
@@ -71,8 +64,6 @@ public final class ShardingRuleTest {
         assertThat(actual.getTableRules().size(), is(1));
         assertTrue(actual.getBindingTableRules().isEmpty());
         assertTrue(actual.getBroadcastTables().isEmpty());
-        assertThat(actual.getDefaultDatabaseShardingStrategy(), instanceOf(NoneShardingStrategy.class));
-        assertThat(actual.getDefaultTableShardingStrategy(), instanceOf(NoneShardingStrategy.class));
         assertThat(actual.getDefaultKeyGenerateAlgorithm(), instanceOf(SnowflakeKeyGenerateAlgorithm.class));
     }
     
@@ -124,34 +115,6 @@ public final class ShardingRuleTest {
     }
     
     @Test
-    public void assertGetDatabaseShardingStrategyFromTableRule() {
-        TableRule tableRule = mock(TableRule.class);
-        when(tableRule.getDatabaseShardingStrategy()).thenReturn(new NoneShardingStrategy());
-        assertThat(createMaximumShardingRule().getDatabaseShardingStrategy(tableRule), instanceOf(NoneShardingStrategy.class));
-    }
-    
-    @Test
-    public void assertGetDatabaseShardingStrategyFromDefault() {
-        ShardingStrategy actual = createMaximumShardingRule().getDatabaseShardingStrategy(mock(TableRule.class));
-        assertThat(actual, instanceOf(StandardShardingStrategy.class));
-        assertThat(actual.getShardingColumns().iterator().next(), is("ds_id"));
-    }
-    
-    @Test
-    public void assertGetTableShardingStrategyFromTableRule() {
-        TableRule tableRule = mock(TableRule.class);
-        when(tableRule.getTableShardingStrategy()).thenReturn(new NoneShardingStrategy());
-        assertThat(createMaximumShardingRule().getTableShardingStrategy(tableRule), instanceOf(NoneShardingStrategy.class));
-    }
-    
-    @Test
-    public void assertGetTableShardingStrategyFromDefault() {
-        ShardingStrategy actual = createMaximumShardingRule().getTableShardingStrategy(mock(TableRule.class));
-        assertThat(actual, instanceOf(StandardShardingStrategy.class));
-        assertThat(actual.getShardingColumns().iterator().next(), is("table_id"));
-    }
-    
-    @Test
     public void assertIsAllBindingTableWhenLogicTablesIsEmpty() {
         assertFalse(createMaximumShardingRule().isAllBindingTables(Collections.emptyList()));
     }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/TableRuleTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/TableRuleTest.java
index e0d1ee1..4ea67fd 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/TableRuleTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/TableRuleTest.java
@@ -21,26 +21,21 @@ import com.google.common.collect.Sets;
 import org.apache.shardingsphere.infra.config.exception.ShardingSphereConfigurationException;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.sharding.algorithm.sharding.mod.ModShardingAlgorithm;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.NoneShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
-import org.apache.shardingsphere.sharding.algorithm.sharding.mod.ModShardingAlgorithm;
-import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.none.NoneShardingStrategy;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.util.Arrays;
 import java.util.Collections;
 
-import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
@@ -54,13 +49,11 @@ public final class TableRuleTest {
     @Test
     public void assertCreateMinTableRule() {
         ShardingTableRuleConfiguration tableRuleConfig = new ShardingTableRuleConfiguration("LOGIC_TABLE");
-        TableRule actual = new TableRule(tableRuleConfig, Arrays.asList("ds0", "ds1"), null, null, null);
+        TableRule actual = new TableRule(tableRuleConfig, Arrays.asList("ds0", "ds1"), null);
         assertThat(actual.getLogicTable(), is("logic_table"));
         assertThat(actual.getActualDataNodes().size(), is(2));
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds0", "LOGIC_TABLE")));
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds1", "LOGIC_TABLE")));
-        assertNull(actual.getDatabaseShardingStrategy());
-        assertNull(actual.getTableShardingStrategy());
     }
     
     @Test
@@ -69,7 +62,7 @@ public final class TableRuleTest {
         tableRuleConfig.setDatabaseShardingStrategy(new NoneShardingStrategyConfiguration());
         tableRuleConfig.setTableShardingStrategy(new NoneShardingStrategyConfiguration());
         tableRuleConfig.setKeyGenerateStrategy(new KeyGenerateStrategyConfiguration("col_1", "increment"));
-        TableRule actual = new TableRule(tableRuleConfig, Arrays.asList("ds0", "ds1"), null, null, null);
+        TableRule actual = new TableRule(tableRuleConfig, Arrays.asList("ds0", "ds1"), null);
         assertThat(actual.getLogicTable(), is("logic_table"));
         assertThat(actual.getActualDataNodes().size(), is(6));
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds0", "table_0")));
@@ -78,8 +71,6 @@ public final class TableRuleTest {
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds1", "table_0")));
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds1", "table_1")));
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds1", "table_2")));
-        assertNotNull(actual.getDatabaseShardingStrategy());
-        assertNotNull(actual.getTableShardingStrategy());
         assertTrue(actual.getGenerateKeyColumn().isPresent());
         assertThat(actual.getGenerateKeyColumn().get(), is("col_1"));
         assertThat(actual.getKeyGeneratorName(), is("increment"));
@@ -100,8 +91,6 @@ public final class TableRuleTest {
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds1", "logic_table_1")));
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds0", "logic_table_2")));
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds1", "logic_table_3")));
-        assertThat(actual.getDatabaseShardingStrategy(), instanceOf(NoneShardingStrategy.class));
-        assertNotNull(actual.getTableShardingStrategy());
     }
     
     @Test
@@ -119,19 +108,17 @@ public final class TableRuleTest {
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds1", "logic_table_1")));
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds2", "logic_table_2")));
         assertTrue(actual.getActualDataNodes().contains(new DataNode("ds0", "logic_table_3")));
-        assertThat(actual.getDatabaseShardingStrategy(), instanceOf(NoneShardingStrategy.class));
-        assertNotNull(actual.getTableShardingStrategy());
     }
     
     @Test
     public void assertGetActualDatasourceNames() {
-        TableRule actual = new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..2}"), Arrays.asList("ds0", "ds1"), null, null, null);
+        TableRule actual = new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..2}"), Arrays.asList("ds0", "ds1"), null);
         assertThat(actual.getActualDatasourceNames(), is(Sets.newLinkedHashSet(Arrays.asList("ds0", "ds1"))));
     }
     
     @Test
     public void assertGetActualTableNames() {
-        TableRule actual = new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..2}"), Arrays.asList("ds0", "ds1"), null, null, null);
+        TableRule actual = new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..2}"), Arrays.asList("ds0", "ds1"), null);
         assertThat(actual.getActualTableNames("ds0"), is(Sets.newLinkedHashSet(Arrays.asList("table_0", "table_1", "table_2"))));
         assertThat(actual.getActualTableNames("ds1"), is(Sets.newLinkedHashSet(Arrays.asList("table_0", "table_1", "table_2"))));
         assertThat(actual.getActualTableNames("ds2"), is(Collections.emptySet()));
@@ -139,25 +126,25 @@ public final class TableRuleTest {
     
     @Test
     public void assertFindActualTableIndex() {
-        TableRule actual = new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..2}"), Arrays.asList("ds0", "ds1"), null, null, null);
+        TableRule actual = new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..2}"), Arrays.asList("ds0", "ds1"), null);
         assertThat(actual.findActualTableIndex("ds1", "table_1"), is(4));
     }
     
     @Test
     public void assertNotFindActualTableIndex() {
-        TableRule actual = new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..2}"), Arrays.asList("ds0", "ds1"), null, null, null);
+        TableRule actual = new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..2}"), Arrays.asList("ds0", "ds1"), null);
         assertThat(actual.findActualTableIndex("ds2", "table_2"), is(-1));
     }
     
     @Test
     public void assertActualTableNameExisted() {
-        TableRule actual = new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..2}"), Arrays.asList("ds0", "ds1"), null, null, null);
+        TableRule actual = new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..2}"), Arrays.asList("ds0", "ds1"), null);
         assertTrue(actual.isExisted("table_2"));
     }
     
     @Test
     public void assertActualTableNameNotExisted() {
-        TableRule actual = new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..2}"), Arrays.asList("ds0", "ds1"), null, null, null);
+        TableRule actual = new TableRule(new ShardingTableRuleConfiguration("LOGIC_TABLE", "ds${0..1}.table_${0..2}"), Arrays.asList("ds0", "ds1"), null);
         assertFalse(actual.isExisted("table_3"));
     }
     
@@ -165,9 +152,6 @@ public final class TableRuleTest {
     public void assertActualDataNodesNotConfigured() {
         ShardingTableRuleConfiguration shardingTableRuleConfig = new ShardingTableRuleConfiguration("LOGIC_TABLE", "");
         shardingTableRuleConfig.setTableShardingStrategy(new StandardShardingStrategyConfiguration("shardingColumn", "INLINE"));
-        InlineShardingAlgorithm inlineShardingAlgorithm = new InlineShardingAlgorithm();
-        inlineShardingAlgorithm.getProps().setProperty("algorithm-expression", "xxx");
-        inlineShardingAlgorithm.init();
-        new TableRule(shardingTableRuleConfig, Arrays.asList("ds0", "ds1"), null, inlineShardingAlgorithm, null);
+        new TableRule(shardingTableRuleConfig, Arrays.asList("ds0", "ds1"), null);
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.ShardingAlgorithm b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.ShardingAlgorithm
index c1e4bc7..083e71d 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.ShardingAlgorithm
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.ShardingAlgorithm
@@ -16,8 +16,5 @@
 #
  
 org.apache.shardingsphere.sharding.fixture.ComplexKeysShardingAlgorithmFixture
-org.apache.shardingsphere.sharding.strategy.fixture.ComplexKeysShardingAlgorithmFixture
 org.apache.shardingsphere.sharding.fixture.HintShardingAlgorithmFixture
-org.apache.shardingsphere.sharding.strategy.fixture.HintShardingAlgorithmFixture
 org.apache.shardingsphere.sharding.fixture.StandardShardingAlgorithmFixture
-org.apache.shardingsphere.sharding.strategy.fixture.StandardShardingAlgorithmFixture
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
index 85731fa..b2c08b2 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.hint.HintManager;
 import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.sql.LogicSQL;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.HintShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingCondition;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingConditions;
@@ -34,9 +35,8 @@ import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStateme
 import org.apache.shardingsphere.sharding.rule.BindingTableRule;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
-import org.apache.shardingsphere.sharding.strategy.hint.HintShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.binder.statement.dml.InsertStatementContext;
 import org.apache.shardingsphere.sql.parser.binder.statement.dml.SelectStatementContext;
@@ -105,7 +105,8 @@ public final class ShardingSQLRouter implements SQLRouter<ShardingRule> {
     }
     
     private boolean isRoutingByHint(final ShardingRule shardingRule, final TableRule tableRule) {
-        return shardingRule.getDatabaseShardingStrategy(tableRule) instanceof HintShardingStrategy && shardingRule.getTableShardingStrategy(tableRule) instanceof HintShardingStrategy;
+        return shardingRule.getDatabaseShardingStrategyConfiguration(tableRule) instanceof HintShardingStrategyConfiguration
+                && shardingRule.getTableShardingStrategyConfiguration(tableRule) instanceof HintShardingStrategyConfiguration;
     }
     
     private boolean isSameShardingCondition(final ShardingRule shardingRule, final ShardingConditions shardingConditions) {
@@ -119,29 +120,29 @@ public final class ShardingSQLRouter implements SQLRouter<ShardingRule> {
     }
     
     private boolean isSameShardingCondition(final ShardingRule shardingRule, final ShardingCondition shardingCondition1, final ShardingCondition shardingCondition2) {
-        if (shardingCondition1.getRouteValues().size() != shardingCondition2.getRouteValues().size()) {
+        if (shardingCondition1.getValues().size() != shardingCondition2.getValues().size()) {
             return false;
         }
-        for (int i = 0; i < shardingCondition1.getRouteValues().size(); i++) {
-            RouteValue shardingValue1 = shardingCondition1.getRouteValues().get(i);
-            RouteValue shardingValue2 = shardingCondition2.getRouteValues().get(i);
-            if (!isSameRouteValue(shardingRule, (ListRouteValue) shardingValue1, (ListRouteValue) shardingValue2)) {
+        for (int i = 0; i < shardingCondition1.getValues().size(); i++) {
+            ShardingConditionValue shardingConditionValue1 = shardingCondition1.getValues().get(i);
+            ShardingConditionValue shardingConditionValue2 = shardingCondition2.getValues().get(i);
+            if (!isSameShardingConditionValue(shardingRule, (ListShardingConditionValue) shardingConditionValue1, (ListShardingConditionValue) shardingConditionValue2)) {
                 return false;
             }
         }
         return true;
     }
     
-    private boolean isSameRouteValue(final ShardingRule shardingRule, final ListRouteValue routeValue1, final ListRouteValue routeValue2) {
-        return isSameLogicTable(shardingRule, routeValue1, routeValue2) && routeValue1.getColumnName().equals(routeValue2.getColumnName()) 
-                && SafeNumberOperationUtils.safeEquals(routeValue1.getValues(), routeValue2.getValues());
+    private boolean isSameShardingConditionValue(final ShardingRule shardingRule, final ListShardingConditionValue shardingConditionValue1, final ListShardingConditionValue shardingConditionValue2) {
+        return isSameLogicTable(shardingRule, shardingConditionValue1, shardingConditionValue2) && shardingConditionValue1.getColumnName().equals(shardingConditionValue2.getColumnName()) 
+                && SafeNumberOperationUtils.safeEquals(shardingConditionValue1.getValues(), shardingConditionValue2.getValues());
     }
     
-    private boolean isSameLogicTable(final ShardingRule shardingRule, final ListRouteValue shardingValue1, final ListRouteValue shardingValue2) {
+    private boolean isSameLogicTable(final ShardingRule shardingRule, final ListShardingConditionValue shardingValue1, final ListShardingConditionValue shardingValue2) {
         return shardingValue1.getTableName().equals(shardingValue2.getTableName()) || isBindingTable(shardingRule, shardingValue1, shardingValue2);
     }
     
-    private boolean isBindingTable(final ShardingRule shardingRule, final ListRouteValue shardingValue1, final ListRouteValue shardingValue2) {
+    private boolean isBindingTable(final ShardingRule shardingRule, final ListShardingConditionValue shardingValue1, final ListShardingConditionValue shardingValue2) {
         Optional<BindingTableRule> bindingRule = shardingRule.findBindingTableRule(shardingValue1.getTableName());
         return bindingRule.isPresent() && bindingRule.get().hasLogicTable(shardingValue2.getTableName());
     }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/ShardingCondition.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/ShardingCondition.java
index d2a8524..af0de20 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/ShardingCondition.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/ShardingCondition.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.sharding.route.engine.condition;
 
 import lombok.Getter;
 import lombok.ToString;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 
 import java.util.LinkedList;
 import java.util.List;
@@ -31,5 +31,5 @@ import java.util.List;
 @ToString
 public class ShardingCondition {
     
-    private final List<RouteValue> routeValues = new LinkedList<>();
+    private final List<ShardingConditionValue> values = new LinkedList<>();
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/impl/InsertClauseShardingConditionEngine.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/impl/InsertClauseShardingConditionEngine.java
index 9e44990..1d7d144 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/impl/InsertClauseShardingConditionEngine.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/impl/InsertClauseShardingConditionEngine.java
@@ -25,7 +25,7 @@ import org.apache.shardingsphere.sharding.route.engine.condition.ShardingConditi
 import org.apache.shardingsphere.sharding.route.engine.condition.engine.ShardingConditionEngine;
 import org.apache.shardingsphere.sharding.route.spi.SPITimeService;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
 import org.apache.shardingsphere.sql.parser.binder.metadata.schema.SchemaMetaData;
 import org.apache.shardingsphere.sql.parser.binder.segment.insert.keygen.GeneratedKeyContext;
 import org.apache.shardingsphere.sql.parser.binder.segment.insert.values.InsertValueContext;
@@ -90,9 +90,9 @@ public final class InsertClauseShardingConditionEngine implements ShardingCondit
             String columnName = columnNames.next();
             if (shardingRule.isShardingColumn(columnName, tableName)) {
                 if (each instanceof SimpleExpressionSegment) {
-                    result.getRouteValues().add(new ListRouteValue<>(columnName, tableName, Collections.singletonList(getRouteValue((SimpleExpressionSegment) each, parameters))));
+                    result.getValues().add(new ListShardingConditionValue<>(columnName, tableName, Collections.singletonList(getShardingValue((SimpleExpressionSegment) each, parameters))));
                 } else if (ExpressionConditionUtils.isNowExpression(each)) {
-                    result.getRouteValues().add(new ListRouteValue<>(columnName, tableName, Collections.singletonList(timeService.getTime())));
+                    result.getValues().add(new ListShardingConditionValue<>(columnName, tableName, Collections.singletonList(timeService.getTime())));
                 } else if (ExpressionConditionUtils.isNullExpression(each)) {
                     throw new ShardingSphereException("Insert clause sharding column can't be null.");
                 }
@@ -101,7 +101,7 @@ public final class InsertClauseShardingConditionEngine implements ShardingCondit
         return result;
     }
     
-    private Comparable<?> getRouteValue(final SimpleExpressionSegment expressionSegment, final List<Object> parameters) {
+    private Comparable<?> getShardingValue(final SimpleExpressionSegment expressionSegment, final List<Object> parameters) {
         Object result;
         if (expressionSegment instanceof ParameterMarkerExpressionSegment) {
             result = parameters.get(((ParameterMarkerExpressionSegment) expressionSegment).getParameterMarkerIndex());
@@ -135,7 +135,7 @@ public final class InsertClauseShardingConditionEngine implements ShardingCondit
     private void appendGeneratedKeyCondition(final GeneratedKeyContext generatedKey, final String tableName, final List<ShardingCondition> shardingConditions) {
         Iterator<Comparable<?>> generatedValuesIterator = generatedKey.getGeneratedValues().iterator();
         for (ShardingCondition each : shardingConditions) {
-            each.getRouteValues().add(new ListRouteValue<>(generatedKey.getColumnName(), tableName, Collections.<Comparable<?>>singletonList(generatedValuesIterator.next())));
+            each.getValues().add(new ListShardingConditionValue<>(generatedKey.getColumnName(), tableName, Collections.<Comparable<?>>singletonList(generatedValuesIterator.next())));
         }
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/impl/WhereClauseShardingConditionEngine.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/impl/WhereClauseShardingConditionEngine.java
index 1e3c1c7..fa8a11e 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/impl/WhereClauseShardingConditionEngine.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/impl/WhereClauseShardingConditionEngine.java
@@ -20,16 +20,16 @@ package org.apache.shardingsphere.sharding.route.engine.condition.engine.impl;
 import com.google.common.collect.Range;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
-import org.apache.shardingsphere.sharding.route.engine.condition.AlwaysFalseRouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.AlwaysFalseShardingConditionValue;
 import org.apache.shardingsphere.sharding.route.engine.condition.AlwaysFalseShardingCondition;
 import org.apache.shardingsphere.sharding.route.engine.condition.Column;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingCondition;
 import org.apache.shardingsphere.sharding.route.engine.condition.engine.ShardingConditionEngine;
 import org.apache.shardingsphere.sharding.route.engine.condition.generator.ConditionValueGeneratorFactory;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.RangeShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.apache.shardingsphere.sql.parser.binder.metadata.schema.SchemaMetaData;
 import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.binder.type.WhereAvailable;
@@ -38,7 +38,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.Expressi
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.AndPredicate;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
-import org.apache.shardingsphere.sql.parser.sql.common.util.ColumnExtractFromExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.util.ColumnExtractor;
 import org.apache.shardingsphere.sql.parser.sql.common.util.ExpressionBuilder;
 import org.apache.shardingsphere.sql.parser.sql.common.util.SafeNumberOperationUtils;
 import org.apache.shardingsphere.sql.parser.sql.common.util.WhereSegmentExtractUtils;
@@ -69,10 +69,7 @@ public final class WhereClauseShardingConditionEngine implements ShardingConditi
             return Collections.emptyList();
         }
         List<ShardingCondition> result = new ArrayList<>();
-        Optional<WhereSegment> whereSegment = ((WhereAvailable) sqlStatementContext).getWhere();
-        if (whereSegment.isPresent()) {
-            result.addAll(createShardingConditions(sqlStatementContext, whereSegment.get().getExpr(), parameters));
-        }
+        ((WhereAvailable) sqlStatementContext).getWhere().ifPresent(segment -> result.addAll(createShardingConditions(sqlStatementContext, segment.getExpr(), parameters)));
         Collection<WhereSegment> subqueryWhereSegments = sqlStatementContext.getSqlStatement() instanceof SelectStatement
                 ? WhereSegmentExtractUtils.getSubqueryWhereSegments((SelectStatement) sqlStatementContext.getSqlStatement()) : Collections.emptyList();
         for (WhereSegment each : subqueryWhereSegments) {
@@ -86,23 +83,21 @@ public final class WhereClauseShardingConditionEngine implements ShardingConditi
     
     private Collection<ShardingCondition> createShardingConditions(final SQLStatementContext<?> sqlStatementContext, final ExpressionSegment expressionSegment, final List<Object> parameters) {
         Collection<ShardingCondition> result = new LinkedList<>();
-    
-        ExpressionBuilder expressionBuilder = new ExpressionBuilder(expressionSegment);
-        Collection<AndPredicate> andPredicates = new LinkedList<>(expressionBuilder.extractAndPredicates().getAndPredicates());
-        for (AndPredicate each : andPredicates) {
-            Map<Column, Collection<RouteValue>> routeValueMap = createRouteValueMap(sqlStatementContext, each, parameters);
-            if (routeValueMap.isEmpty()) {
+        for (AndPredicate each : new ExpressionBuilder(expressionSegment).extractAndPredicates().getAndPredicates()) {
+            Map<Column, Collection<ShardingConditionValue>> shardingConditionValues = createShardingConditionValueMap(sqlStatementContext, each, parameters);
+            if (shardingConditionValues.isEmpty()) {
                 return Collections.emptyList();
             }
-            result.add(createShardingCondition(routeValueMap));
+            result.add(createShardingCondition(shardingConditionValues));
         }
         return result;
     }
     
-    private Map<Column, Collection<RouteValue>> createRouteValueMap(final SQLStatementContext<?> sqlStatementContext, final AndPredicate expressions, final List<Object> parameters) {
-        Map<Column, Collection<RouteValue>> result = new HashMap<>();
-        for (ExpressionSegment each : expressions.getPredicates()) {
-            Optional<ColumnSegment> columnSegment = ColumnExtractFromExpression.extract(each);
+    private Map<Column, Collection<ShardingConditionValue>> createShardingConditionValueMap(final SQLStatementContext<?> sqlStatementContext, 
+                                                                                            final AndPredicate andPredicate, final List<Object> parameters) {
+        Map<Column, Collection<ShardingConditionValue>> result = new HashMap<>(andPredicate.getPredicates().size(), 1);
+        for (ExpressionSegment each : andPredicate.getPredicates()) {
+            Optional<ColumnSegment> columnSegment = ColumnExtractor.extract(each);
             if (!columnSegment.isPresent()) {
                 continue;
             }
@@ -111,29 +106,29 @@ public final class WhereClauseShardingConditionEngine implements ShardingConditi
                 continue;
             }
             Column column = new Column(columnSegment.get().getIdentifier().getValue(), tableName.get());
-            Optional<RouteValue> routeValue = ConditionValueGeneratorFactory.generate(each, column, parameters);
-            if (routeValue.isPresent()) {
+            Optional<ShardingConditionValue> shardingConditionValue = ConditionValueGeneratorFactory.generate(each, column, parameters);
+            if (shardingConditionValue.isPresent()) {
                 if (!result.containsKey(column)) {
-                    Collection<RouteValue> routeValues = new LinkedList<>();
-                    routeValues.add(routeValue.get());
-                    result.put(column, routeValues);
+                    Collection<ShardingConditionValue> shardingConditionValues = new LinkedList<>();
+                    shardingConditionValues.add(shardingConditionValue.get());
+                    result.put(column, shardingConditionValues);
                 } else {
-                    result.get(column).add(routeValue.get());
+                    result.get(column).add(shardingConditionValue.get());
                 }
             }
         }
         return result;
     }
     
-    private ShardingCondition createShardingCondition(final Map<Column, Collection<RouteValue>> routeValueMap) {
+    private ShardingCondition createShardingCondition(final Map<Column, Collection<ShardingConditionValue>> shardingConditionValues) {
         ShardingCondition result = new ShardingCondition();
-        for (Entry<Column, Collection<RouteValue>> entry : routeValueMap.entrySet()) {
+        for (Entry<Column, Collection<ShardingConditionValue>> entry : shardingConditionValues.entrySet()) {
             try {
-                RouteValue routeValue = mergeRouteValues(entry.getKey(), entry.getValue());
-                if (routeValue instanceof AlwaysFalseRouteValue) {
+                ShardingConditionValue shardingConditionValue = mergeShardingConditionValues(entry.getKey(), entry.getValue());
+                if (shardingConditionValue instanceof AlwaysFalseShardingConditionValue) {
                     return new AlwaysFalseShardingCondition();
                 }
-                result.getRouteValues().add(routeValue);
+                result.getValues().add(shardingConditionValue);
             } catch (final ClassCastException ex) {
                 throw new ShardingSphereException("Found different types for sharding value `%s`.", entry.getKey());
             }
@@ -142,34 +137,34 @@ public final class WhereClauseShardingConditionEngine implements ShardingConditi
     }
     
     @SuppressWarnings("unchecked")
-    private RouteValue mergeRouteValues(final Column column, final Collection<RouteValue> routeValues) {
+    private ShardingConditionValue mergeShardingConditionValues(final Column column, final Collection<ShardingConditionValue> shardingConditionValues) {
         Collection<Comparable<?>> listValue = null;
         Range<Comparable<?>> rangeValue = null;
-        for (RouteValue each : routeValues) {
-            if (each instanceof ListRouteValue) {
-                listValue = mergeListRouteValues(((ListRouteValue) each).getValues(), listValue);
+        for (ShardingConditionValue each : shardingConditionValues) {
+            if (each instanceof ListShardingConditionValue) {
+                listValue = mergeListShardingValues(((ListShardingConditionValue) each).getValues(), listValue);
                 if (listValue.isEmpty()) {
-                    return new AlwaysFalseRouteValue();
+                    return new AlwaysFalseShardingConditionValue();
                 }
-            } else if (each instanceof RangeRouteValue) {
+            } else if (each instanceof RangeShardingConditionValue) {
                 try {
-                    rangeValue = mergeRangeRouteValues(((RangeRouteValue) each).getValueRange(), rangeValue);
+                    rangeValue = mergeRangeShardingValues(((RangeShardingConditionValue) each).getValueRange(), rangeValue);
                 } catch (final IllegalArgumentException ex) {
-                    return new AlwaysFalseRouteValue();
+                    return new AlwaysFalseShardingConditionValue();
                 }
             }
         }
         if (null == listValue) {
-            return new RangeRouteValue<>(column.getName(), column.getTableName(), rangeValue);
+            return new RangeShardingConditionValue<>(column.getName(), column.getTableName(), rangeValue);
         }
         if (null == rangeValue) {
-            return new ListRouteValue<>(column.getName(), column.getTableName(), listValue);
+            return new ListShardingConditionValue<>(column.getName(), column.getTableName(), listValue);
         }
-        listValue = mergeListAndRangeRouteValues(listValue, rangeValue);
-        return listValue.isEmpty() ? new AlwaysFalseRouteValue() : new ListRouteValue<>(column.getName(), column.getTableName(), listValue);
+        listValue = mergeListAndRangeShardingValues(listValue, rangeValue);
+        return listValue.isEmpty() ? new AlwaysFalseShardingConditionValue() : new ListShardingConditionValue<>(column.getName(), column.getTableName(), listValue);
     }
     
-    private Collection<Comparable<?>> mergeListRouteValues(final Collection<Comparable<?>> value1, final Collection<Comparable<?>> value2) {
+    private Collection<Comparable<?>> mergeListShardingValues(final Collection<Comparable<?>> value1, final Collection<Comparable<?>> value2) {
         if (null == value2) {
             return value1;
         }
@@ -177,11 +172,11 @@ public final class WhereClauseShardingConditionEngine implements ShardingConditi
         return value1;
     }
     
-    private Range<Comparable<?>> mergeRangeRouteValues(final Range<Comparable<?>> value1, final Range<Comparable<?>> value2) {
+    private Range<Comparable<?>> mergeRangeShardingValues(final Range<Comparable<?>> value1, final Range<Comparable<?>> value2) {
         return null == value2 ? value1 : SafeNumberOperationUtils.safeIntersection(value1, value2);
     }
     
-    private Collection<Comparable<?>> mergeListAndRangeRouteValues(final Collection<Comparable<?>> listValue, final Range<Comparable<?>> rangeValue) {
+    private Collection<Comparable<?>> mergeListAndRangeShardingValues(final Collection<Comparable<?>> listValue, final Range<Comparable<?>> rangeValue) {
         Collection<Comparable<?>> result = new LinkedList<>();
         for (Comparable<?> each : listValue) {
             if (SafeNumberOperationUtils.safeContains(rangeValue, each)) {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/ConditionValueGenerator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/ConditionValueGenerator.java
index 1504bb1..5756fde 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/ConditionValueGenerator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/ConditionValueGenerator.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.sharding.route.engine.condition.generator;
 
 import org.apache.shardingsphere.sharding.route.engine.condition.Column;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
 
 import java.util.List;
@@ -39,5 +39,5 @@ public interface ConditionValueGenerator<T extends ExpressionSegment> {
      * @param parameters SQL parameters
      * @return route value
      */
-    Optional<RouteValue> generate(T predicateRightValue, Column column, List<Object> parameters);
+    Optional<ShardingConditionValue> generate(T predicateRightValue, Column column, List<Object> parameters);
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/ConditionValueGeneratorFactory.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/ConditionValueGeneratorFactory.java
index bf26ee0..2d0a7ec 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/ConditionValueGeneratorFactory.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/ConditionValueGeneratorFactory.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.sharding.route.engine.condition.Column;
 import org.apache.shardingsphere.sharding.route.engine.condition.generator.impl.ConditionValueBetweenOperatorGenerator;
 import org.apache.shardingsphere.sharding.route.engine.condition.generator.impl.ConditionValueCompareOperatorGenerator;
 import org.apache.shardingsphere.sharding.route.engine.condition.generator.impl.ConditionValueInOperatorGenerator;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
@@ -46,7 +46,7 @@ public final class ConditionValueGeneratorFactory {
      * @param parameters SQL parameters
      * @return route value
      */
-    public static Optional<RouteValue> generate(final ExpressionSegment predicate, final Column column, final List<Object> parameters) {
+    public static Optional<ShardingConditionValue> generate(final ExpressionSegment predicate, final Column column, final List<Object> parameters) {
         if (predicate instanceof BinaryOperationExpression) {
             return new ConditionValueCompareOperatorGenerator().generate((BinaryOperationExpression) predicate, column, parameters);
         }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueBetweenOperatorGenerator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueBetweenOperatorGenerator.java
index 0bf69c9..35e7efa 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueBetweenOperatorGenerator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueBetweenOperatorGenerator.java
@@ -23,8 +23,8 @@ import org.apache.shardingsphere.sharding.route.engine.condition.ExpressionCondi
 import org.apache.shardingsphere.sharding.route.engine.condition.generator.ConditionValue;
 import org.apache.shardingsphere.sharding.route.engine.condition.generator.ConditionValueGenerator;
 import org.apache.shardingsphere.sharding.route.spi.SPITimeService;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.RangeShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.util.SafeNumberOperationUtils;
 
@@ -38,21 +38,21 @@ import java.util.Optional;
 public final class ConditionValueBetweenOperatorGenerator implements ConditionValueGenerator<BetweenExpression> {
     
     @Override
-    public Optional<RouteValue> generate(final BetweenExpression predicate, final Column column, final List<Object> parameters) {
-        Optional<Comparable<?>> betweenRouteValue = new ConditionValue(predicate.getBetweenExpr(), parameters).getValue();
-        Optional<Comparable<?>> andRouteValue = new ConditionValue(predicate.getAndExpr(), parameters).getValue();
-        if (betweenRouteValue.isPresent() && andRouteValue.isPresent()) {
-            return Optional.of(new RangeRouteValue<>(column.getName(), column.getTableName(), SafeNumberOperationUtils.safeClosed(betweenRouteValue.get(), andRouteValue.get())));
+    public Optional<ShardingConditionValue> generate(final BetweenExpression predicate, final Column column, final List<Object> parameters) {
+        Optional<Comparable<?>> betweenConditionValue = new ConditionValue(predicate.getBetweenExpr(), parameters).getValue();
+        Optional<Comparable<?>> andConditionValue = new ConditionValue(predicate.getAndExpr(), parameters).getValue();
+        if (betweenConditionValue.isPresent() && andConditionValue.isPresent()) {
+            return Optional.of(new RangeShardingConditionValue<>(column.getName(), column.getTableName(), SafeNumberOperationUtils.safeClosed(betweenConditionValue.get(), andConditionValue.get())));
         }
         Date date = new SPITimeService().getTime();
-        if (!betweenRouteValue.isPresent() && ExpressionConditionUtils.isNowExpression(predicate.getBetweenExpr())) {
-            betweenRouteValue = Optional.of(date);
+        if (!betweenConditionValue.isPresent() && ExpressionConditionUtils.isNowExpression(predicate.getBetweenExpr())) {
+            betweenConditionValue = Optional.of(date);
         }
-        if (!andRouteValue.isPresent() && ExpressionConditionUtils.isNowExpression(predicate.getAndExpr())) {
-            andRouteValue = Optional.of(date);
+        if (!andConditionValue.isPresent() && ExpressionConditionUtils.isNowExpression(predicate.getAndExpr())) {
+            andConditionValue = Optional.of(date);
         }
-        return betweenRouteValue.isPresent() && andRouteValue.isPresent()
-                ? Optional.of(new RangeRouteValue<>(column.getName(), column.getTableName(), Range.closed(betweenRouteValue.get(), andRouteValue.get())))
+        return betweenConditionValue.isPresent() && andConditionValue.isPresent()
+                ? Optional.of(new RangeShardingConditionValue<>(column.getName(), column.getTableName(), Range.closed(betweenConditionValue.get(), andConditionValue.get())))
                 : Optional.empty();
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueCompareOperatorGenerator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueCompareOperatorGenerator.java
index e2609bf..2457fbc 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueCompareOperatorGenerator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueCompareOperatorGenerator.java
@@ -24,9 +24,9 @@ import org.apache.shardingsphere.sharding.route.engine.condition.ExpressionCondi
 import org.apache.shardingsphere.sharding.route.engine.condition.generator.ConditionValue;
 import org.apache.shardingsphere.sharding.route.engine.condition.generator.ConditionValueGenerator;
 import org.apache.shardingsphere.sharding.route.spi.SPITimeService;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.RangeShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
 
 import java.util.Arrays;
@@ -51,14 +51,14 @@ public final class ConditionValueCompareOperatorGenerator implements ConditionVa
     private static final List<String> OPERATORS = Arrays.asList(EQUAL, GREATER_THAN, LESS_THAN, AT_LEAST, AT_MOST);
     
     @Override
-    public Optional<RouteValue> generate(final BinaryOperationExpression predicate, final Column column, final List<Object> parameters) {
+    public Optional<ShardingConditionValue> generate(final BinaryOperationExpression predicate, final Column column, final List<Object> parameters) {
         String operator = predicate.getOperator();
         if (!isSupportedOperator(operator)) {
             return Optional.empty();
         }
-        Optional<Comparable<?>> routeValue = new ConditionValue(predicate.getRight(), parameters).getValue();
-        if (routeValue.isPresent()) {
-            return generate(routeValue.get(), column, operator);
+        Optional<Comparable<?>> conditionValue = new ConditionValue(predicate.getRight(), parameters).getValue();
+        if (conditionValue.isPresent()) {
+            return generate(conditionValue.get(), column, operator);
         }
         if (ExpressionConditionUtils.isNowExpression(predicate.getRight())) {
             return generate(new SPITimeService().getTime(), column, operator);
@@ -66,20 +66,20 @@ public final class ConditionValueCompareOperatorGenerator implements ConditionVa
         return Optional.empty();
     }
     
-    private Optional<RouteValue> generate(final Comparable<?> comparable, final Column column, final String operator) {
+    private Optional<ShardingConditionValue> generate(final Comparable<?> comparable, final Column column, final String operator) {
         String columnName = column.getName();
         String tableName = column.getTableName();
         switch (operator) {
             case EQUAL:
-                return Optional.of(new ListRouteValue<>(columnName, tableName, Lists.newArrayList(comparable)));
+                return Optional.of(new ListShardingConditionValue<>(columnName, tableName, Lists.newArrayList(comparable)));
             case GREATER_THAN:
-                return Optional.of(new RangeRouteValue<>(columnName, tableName, Range.greaterThan(comparable)));
+                return Optional.of(new RangeShardingConditionValue<>(columnName, tableName, Range.greaterThan(comparable)));
             case LESS_THAN:
-                return Optional.of(new RangeRouteValue<>(columnName, tableName, Range.lessThan(comparable)));
+                return Optional.of(new RangeShardingConditionValue<>(columnName, tableName, Range.lessThan(comparable)));
             case AT_MOST:
-                return Optional.of(new RangeRouteValue<>(columnName, tableName, Range.atMost(comparable)));
+                return Optional.of(new RangeShardingConditionValue<>(columnName, tableName, Range.atMost(comparable)));
             case AT_LEAST:
-                return Optional.of(new RangeRouteValue<>(columnName, tableName, Range.atLeast(comparable)));
+                return Optional.of(new RangeShardingConditionValue<>(columnName, tableName, Range.atLeast(comparable)));
             default:
                 return Optional.empty();
         }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueInOperatorGenerator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueInOperatorGenerator.java
index 07d25f5..774f10f 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueInOperatorGenerator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueInOperatorGenerator.java
@@ -22,8 +22,8 @@ import org.apache.shardingsphere.sharding.route.engine.condition.ExpressionCondi
 import org.apache.shardingsphere.sharding.route.engine.condition.generator.ConditionValue;
 import org.apache.shardingsphere.sharding.route.engine.condition.generator.ConditionValueGenerator;
 import org.apache.shardingsphere.sharding.route.spi.SPITimeService;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
 
@@ -37,19 +37,19 @@ import java.util.Optional;
 public final class ConditionValueInOperatorGenerator implements ConditionValueGenerator<InExpression> {
     
     @Override
-    public Optional<RouteValue> generate(final InExpression predicate, final Column column, final List<Object> parameters) {
-        List<Comparable<?>> routeValues = new LinkedList<>();
+    public Optional<ShardingConditionValue> generate(final InExpression predicate, final Column column, final List<Object> parameters) {
+        List<Comparable<?>> shardingConditionValues = new LinkedList<>();
         SPITimeService timeService = new SPITimeService();
         for (ExpressionSegment each : predicate.getExpressionList()) {
-            Optional<Comparable<?>> routeValue = new ConditionValue(each, parameters).getValue();
-            if (routeValue.isPresent()) {
-                routeValues.add(routeValue.get());
+            Optional<Comparable<?>> shardingConditionValue = new ConditionValue(each, parameters).getValue();
+            if (shardingConditionValue.isPresent()) {
+                shardingConditionValues.add(shardingConditionValue.get());
                 continue;
             }
             if (ExpressionConditionUtils.isNowExpression(each)) {
-                routeValues.add(timeService.getTime());
+                shardingConditionValues.add(timeService.getTime());
             }
         }
-        return routeValues.isEmpty() ? Optional.empty() : Optional.of(new ListRouteValue<>(column.getName(), column.getTableName(), routeValues));
+        return shardingConditionValues.isEmpty() ? Optional.empty() : Optional.of(new ListShardingConditionValue<>(column.getName(), column.getTableName(), shardingConditionValues));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/AlwaysFalseRouteValue.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/value/AlwaysFalseShardingConditionValue.java
similarity index 86%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/AlwaysFalseRouteValue.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/value/AlwaysFalseShardingConditionValue.java
index 63de889..323df1c 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/AlwaysFalseRouteValue.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/value/AlwaysFalseShardingConditionValue.java
@@ -15,14 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.condition;
-
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+package org.apache.shardingsphere.sharding.route.engine.condition.value;
 
 /**
- * Always false route value.
+ * Always false sharding condition value.
  */
-public final class AlwaysFalseRouteValue implements RouteValue {
+public final class AlwaysFalseShardingConditionValue implements ShardingConditionValue {
     
     @Override
     public String getColumnName() {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/ListRouteValue.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/value/ListShardingConditionValue.java
similarity index 85%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/ListRouteValue.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/value/ListShardingConditionValue.java
index ec548ae..62ca8a9 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/ListRouteValue.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/value/ListShardingConditionValue.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.value;
+package org.apache.shardingsphere.sharding.route.engine.condition.value;
 
 import com.google.common.base.Joiner;
 import lombok.Getter;
@@ -25,11 +25,11 @@ import java.util.ArrayList;
 import java.util.Collection;
 
 /**
- * Route value for list values.
+ * Sharding condition value for list values.
  */
 @RequiredArgsConstructor
 @Getter
-public final class ListRouteValue<T extends Comparable<?>> implements RouteValue {
+public final class ListShardingConditionValue<T extends Comparable<?>> implements ShardingConditionValue {
     
     private final String columnName;
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/RangeRouteValue.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/value/RangeShardingConditionValue.java
similarity index 83%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/RangeRouteValue.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/value/RangeShardingConditionValue.java
index 464a423..6f6f7c2 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/RangeRouteValue.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/value/RangeShardingConditionValue.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.value;
+package org.apache.shardingsphere.sharding.route.engine.condition.value;
 
 import com.google.common.collect.Range;
 import lombok.Getter;
@@ -23,12 +23,12 @@ import lombok.RequiredArgsConstructor;
 import lombok.ToString;
 
 /**
- * Route value for range.
+ * Sharding condition value for range.
  */
 @RequiredArgsConstructor
 @Getter
 @ToString
-public final class RangeRouteValue<T extends Comparable<?>> implements RouteValue {
+public final class RangeShardingConditionValue<T extends Comparable<?>> implements ShardingConditionValue {
     
     private final String columnName;
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/RouteValue.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/value/ShardingConditionValue.java
similarity index 87%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/RouteValue.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/value/ShardingConditionValue.java
index 066e957..09165fd 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/RouteValue.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/value/ShardingConditionValue.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.value;
+package org.apache.shardingsphere.sharding.route.engine.condition.value;
 
 /**
- * Route value.
+ * Sharding condition value.
  */
-public interface RouteValue {
+public interface ShardingConditionValue {
     
     /**
      * Get column name.
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/standard/ShardingStandardRoutingEngine.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/standard/ShardingStandardRoutingEngine.java
index f279565..72ff2b5 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/standard/ShardingStandardRoutingEngine.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/standard/ShardingStandardRoutingEngine.java
@@ -25,16 +25,21 @@ import org.apache.shardingsphere.infra.hint.HintManager;
 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.sharding.api.config.strategy.sharding.HintShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingCondition;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingConditions;
 import org.apache.shardingsphere.sharding.route.engine.type.ShardingRouteEngine;
 import org.apache.shardingsphere.sharding.rule.BindingTableRule;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
-import org.apache.shardingsphere.sharding.strategy.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.hint.HintShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
+import org.apache.shardingsphere.sharding.route.strategy.ShardingStrategy;
+import org.apache.shardingsphere.sharding.route.strategy.ShardingStrategyFactory;
+import org.apache.shardingsphere.sharding.route.strategy.type.hint.HintShardingStrategy;
+import org.apache.shardingsphere.sharding.route.strategy.type.none.NoneShardingStrategy;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -42,6 +47,7 @@ import java.util.Collections;
 import java.util.LinkedHashSet;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Optional;
 
 /**
@@ -69,73 +75,84 @@ public final class ShardingStandardRoutingEngine implements ShardingRouteEngine
     }
     
     private Collection<DataNode> getDataNodes(final ShardingRule shardingRule, final TableRule tableRule) {
+        ShardingStrategy databaseShardingStrategy = createShardingStrategy(shardingRule.getDatabaseShardingStrategyConfiguration(tableRule), shardingRule.getShardingAlgorithms());
+        ShardingStrategy tableShardingStrategy = createShardingStrategy(shardingRule.getTableShardingStrategyConfiguration(tableRule), shardingRule.getShardingAlgorithms());
         if (isRoutingByHint(shardingRule, tableRule)) {
-            return routeByHint(shardingRule, tableRule);
+            return routeByHint(tableRule, databaseShardingStrategy, tableShardingStrategy);
         }
         if (isRoutingByShardingConditions(shardingRule, tableRule)) {
-            return routeByShardingConditions(shardingRule, tableRule);
+            return routeByShardingConditions(shardingRule, tableRule, databaseShardingStrategy, tableShardingStrategy);
         }
-        return routeByMixedConditions(shardingRule, tableRule);
+        return routeByMixedConditions(shardingRule, tableRule, databaseShardingStrategy, tableShardingStrategy);
     }
     
     private boolean isRoutingByHint(final ShardingRule shardingRule, final TableRule tableRule) {
-        return shardingRule.getDatabaseShardingStrategy(tableRule) instanceof HintShardingStrategy && shardingRule.getTableShardingStrategy(tableRule) instanceof HintShardingStrategy;
+        return shardingRule.getDatabaseShardingStrategyConfiguration(tableRule) instanceof HintShardingStrategyConfiguration
+                && shardingRule.getTableShardingStrategyConfiguration(tableRule) instanceof HintShardingStrategyConfiguration;
     }
     
-    private Collection<DataNode> routeByHint(final ShardingRule shardingRule, final TableRule tableRule) {
-        return route0(shardingRule, tableRule, getDatabaseShardingValuesFromHint(), getTableShardingValuesFromHint());
+    private Collection<DataNode> routeByHint(final TableRule tableRule, final ShardingStrategy databaseShardingStrategy, final ShardingStrategy tableShardingStrategy) {
+        return route0(tableRule, databaseShardingStrategy, getDatabaseShardingValuesFromHint(), tableShardingStrategy, getTableShardingValuesFromHint());
     }
     
     private boolean isRoutingByShardingConditions(final ShardingRule shardingRule, final TableRule tableRule) {
-        return !(shardingRule.getDatabaseShardingStrategy(tableRule) instanceof HintShardingStrategy || shardingRule.getTableShardingStrategy(tableRule) instanceof HintShardingStrategy);
+        return !(shardingRule.getDatabaseShardingStrategyConfiguration(tableRule) instanceof HintShardingStrategyConfiguration
+                || shardingRule.getTableShardingStrategyConfiguration(tableRule) instanceof HintShardingStrategyConfiguration);
     }
     
-    private Collection<DataNode> routeByShardingConditions(final ShardingRule shardingRule, final TableRule tableRule) {
+    private Collection<DataNode> routeByShardingConditions(final ShardingRule shardingRule, final TableRule tableRule, 
+                                                           final ShardingStrategy databaseShardingStrategy, final ShardingStrategy tableShardingStrategy) {
         return shardingConditions.getConditions().isEmpty()
-                ? route0(shardingRule, tableRule, Collections.emptyList(), Collections.emptyList()) : routeByShardingConditionsWithCondition(shardingRule, tableRule);
+                ? route0(tableRule, databaseShardingStrategy, Collections.emptyList(), tableShardingStrategy, Collections.emptyList())
+                : routeByShardingConditionsWithCondition(shardingRule, tableRule, databaseShardingStrategy, tableShardingStrategy);
     }
     
-    private Collection<DataNode> routeByShardingConditionsWithCondition(final ShardingRule shardingRule, final TableRule tableRule) {
+    private Collection<DataNode> routeByShardingConditionsWithCondition(final ShardingRule shardingRule, final TableRule tableRule, 
+                                                                        final ShardingStrategy databaseShardingStrategy, final ShardingStrategy tableShardingStrategy) {
         Collection<DataNode> result = new LinkedList<>();
         for (ShardingCondition each : shardingConditions.getConditions()) {
-            Collection<DataNode> dataNodes = route0(shardingRule, tableRule, 
-                    getShardingValuesFromShardingConditions(shardingRule, shardingRule.getDatabaseShardingStrategy(tableRule).getShardingColumns(), each),
-                    getShardingValuesFromShardingConditions(shardingRule, shardingRule.getTableShardingStrategy(tableRule).getShardingColumns(), each));
+            Collection<DataNode> dataNodes = route0(tableRule, 
+                    databaseShardingStrategy, getShardingValuesFromShardingConditions(shardingRule, databaseShardingStrategy.getShardingColumns(), each),
+                    tableShardingStrategy, getShardingValuesFromShardingConditions(shardingRule, tableShardingStrategy.getShardingColumns(), each));
             result.addAll(dataNodes);
             originalDataNodes.add(dataNodes);
         }
         return result;
     }
     
-    private Collection<DataNode> routeByMixedConditions(final ShardingRule shardingRule, final TableRule tableRule) {
-        return shardingConditions.getConditions().isEmpty() ? routeByMixedConditionsWithHint(shardingRule, tableRule) : routeByMixedConditionsWithCondition(shardingRule, tableRule);
+    private Collection<DataNode> routeByMixedConditions(final ShardingRule shardingRule, final TableRule tableRule, 
+                                                        final ShardingStrategy databaseShardingStrategy, final ShardingStrategy tableShardingStrategy) {
+        return shardingConditions.getConditions().isEmpty()
+                ? routeByMixedConditionsWithHint(shardingRule, tableRule, databaseShardingStrategy, tableShardingStrategy)
+                : routeByMixedConditionsWithCondition(shardingRule, tableRule, databaseShardingStrategy, tableShardingStrategy);
     }
     
-    private Collection<DataNode> routeByMixedConditionsWithCondition(final ShardingRule shardingRule, final TableRule tableRule) {
+    private Collection<DataNode> routeByMixedConditionsWithCondition(final ShardingRule shardingRule, final TableRule tableRule, 
+                                                                     final ShardingStrategy databaseShardingStrategy, final ShardingStrategy tableShardingStrategy) {
         Collection<DataNode> result = new LinkedList<>();
         for (ShardingCondition each : shardingConditions.getConditions()) {
-            Collection<DataNode> dataNodes = route0(shardingRule, tableRule, getDatabaseShardingValues(shardingRule, tableRule, each), getTableShardingValues(shardingRule, tableRule, each));
+            Collection<DataNode> dataNodes = route0(tableRule, databaseShardingStrategy, 
+                    getDatabaseShardingValues(shardingRule, databaseShardingStrategy, each), tableShardingStrategy, getTableShardingValues(shardingRule, tableShardingStrategy, each));
             result.addAll(dataNodes);
             originalDataNodes.add(dataNodes);
         }
         return result;
     }
     
-    private Collection<DataNode> routeByMixedConditionsWithHint(final ShardingRule shardingRule, final TableRule tableRule) {
-        if (shardingRule.getDatabaseShardingStrategy(tableRule) instanceof HintShardingStrategy) {
-            return route0(shardingRule, tableRule, getDatabaseShardingValuesFromHint(), Collections.emptyList());
+    private Collection<DataNode> routeByMixedConditionsWithHint(final ShardingRule shardingRule, final TableRule tableRule, 
+                                                                final ShardingStrategy databaseShardingStrategy, final ShardingStrategy tableShardingStrategy) {
+        if (shardingRule.getDatabaseShardingStrategyConfiguration(tableRule) instanceof HintShardingStrategyConfiguration) {
+            return route0(tableRule, databaseShardingStrategy, getDatabaseShardingValuesFromHint(), tableShardingStrategy, Collections.emptyList());
         }
-        return route0(shardingRule, tableRule, Collections.emptyList(), getTableShardingValuesFromHint());
+        return route0(tableRule, databaseShardingStrategy, Collections.emptyList(), tableShardingStrategy, getTableShardingValuesFromHint());
     }
     
-    private List<RouteValue> getDatabaseShardingValues(final ShardingRule shardingRule, final TableRule tableRule, final ShardingCondition shardingCondition) {
-        ShardingStrategy dataBaseShardingStrategy = shardingRule.getDatabaseShardingStrategy(tableRule);
-        return isGettingShardingValuesFromHint(dataBaseShardingStrategy)
-                ? getDatabaseShardingValuesFromHint() : getShardingValuesFromShardingConditions(shardingRule, dataBaseShardingStrategy.getShardingColumns(), shardingCondition);
+    private List<ShardingConditionValue> getDatabaseShardingValues(final ShardingRule shardingRule, final ShardingStrategy databaseShardingStrategy, final ShardingCondition shardingCondition) {
+        return isGettingShardingValuesFromHint(databaseShardingStrategy)
+                ? getDatabaseShardingValuesFromHint() : getShardingValuesFromShardingConditions(shardingRule, databaseShardingStrategy.getShardingColumns(), shardingCondition);
     }
     
-    private List<RouteValue> getTableShardingValues(final ShardingRule shardingRule, final TableRule tableRule, final ShardingCondition shardingCondition) {
-        ShardingStrategy tableShardingStrategy = shardingRule.getTableShardingStrategy(tableRule);
+    private List<ShardingConditionValue> getTableShardingValues(final ShardingRule shardingRule, final ShardingStrategy tableShardingStrategy, final ShardingCondition shardingCondition) {
         return isGettingShardingValuesFromHint(tableShardingStrategy)
                 ? getTableShardingValuesFromHint() : getShardingValuesFromShardingConditions(shardingRule, tableShardingStrategy.getShardingColumns(), shardingCondition);
     }
@@ -144,21 +161,21 @@ public final class ShardingStandardRoutingEngine implements ShardingRouteEngine
         return shardingStrategy instanceof HintShardingStrategy;
     }
     
-    private List<RouteValue> getDatabaseShardingValuesFromHint() {
-        return getRouteValues(HintManager.isDatabaseShardingOnly() ? HintManager.getDatabaseShardingValues() : HintManager.getDatabaseShardingValues(logicTableName));
+    private List<ShardingConditionValue> getDatabaseShardingValuesFromHint() {
+        return getShardingConditions(HintManager.isDatabaseShardingOnly() ? HintManager.getDatabaseShardingValues() : HintManager.getDatabaseShardingValues(logicTableName));
     }
     
-    private List<RouteValue> getTableShardingValuesFromHint() {
-        return getRouteValues(HintManager.getTableShardingValues(logicTableName));
+    private List<ShardingConditionValue> getTableShardingValuesFromHint() {
+        return getShardingConditions(HintManager.getTableShardingValues(logicTableName));
     }
     
-    private List<RouteValue> getRouteValues(final Collection<Comparable<?>> shardingValue) {
-        return shardingValue.isEmpty() ? Collections.emptyList() : Collections.singletonList(new ListRouteValue<>("", logicTableName, shardingValue));
+    private List<ShardingConditionValue> getShardingConditions(final Collection<Comparable<?>> shardingValue) {
+        return shardingValue.isEmpty() ? Collections.emptyList() : Collections.singletonList(new ListShardingConditionValue<>("", logicTableName, shardingValue));
     }
     
-    private List<RouteValue> getShardingValuesFromShardingConditions(final ShardingRule shardingRule, final Collection<String> shardingColumns, final ShardingCondition shardingCondition) {
-        List<RouteValue> result = new ArrayList<>(shardingColumns.size());
-        for (RouteValue each : shardingCondition.getRouteValues()) {
+    private List<ShardingConditionValue> getShardingValuesFromShardingConditions(final ShardingRule shardingRule, final Collection<String> shardingColumns, final ShardingCondition shardingCondition) {
+        List<ShardingConditionValue> result = new ArrayList<>(shardingColumns.size());
+        for (ShardingConditionValue each : shardingCondition.getValues()) {
             Optional<BindingTableRule> bindingTableRule = shardingRule.findBindingTableRule(logicTableName);
             if ((logicTableName.equals(each.getTableName()) || bindingTableRule.isPresent() && bindingTableRule.get().hasLogicTable(logicTableName)) 
                     && shardingColumns.contains(each.getColumnName())) {
@@ -168,34 +185,42 @@ public final class ShardingStandardRoutingEngine implements ShardingRouteEngine
         return result;
     }
     
-    private Collection<DataNode> route0(final ShardingRule shardingRule, final TableRule tableRule, final List<RouteValue> databaseShardingValues, final List<RouteValue> tableShardingValues) {
-        Collection<String> routedDataSources = routeDataSources(shardingRule, tableRule, databaseShardingValues);
+    private Collection<DataNode> route0(final TableRule tableRule, 
+                                        final ShardingStrategy databaseShardingStrategy, final List<ShardingConditionValue> databaseShardingValues, 
+                                        final ShardingStrategy tableShardingStrategy, final List<ShardingConditionValue> tableShardingValues) {
+        Collection<String> routedDataSources = routeDataSources(tableRule, databaseShardingStrategy, databaseShardingValues);
         Collection<DataNode> result = new LinkedList<>();
         for (String each : routedDataSources) {
-            result.addAll(routeTables(shardingRule, tableRule, each, tableShardingValues));
+            result.addAll(routeTables(tableRule, each, tableShardingStrategy, tableShardingValues));
         }
         return result;
     }
     
-    private Collection<String> routeDataSources(final ShardingRule shardingRule, final TableRule tableRule, final List<RouteValue> databaseShardingValues) {
+    private Collection<String> routeDataSources(final TableRule tableRule, final ShardingStrategy databaseShardingStrategy, final List<ShardingConditionValue> databaseShardingValues) {
         if (databaseShardingValues.isEmpty()) {
             return tableRule.getActualDatasourceNames();
         }
-        Collection<String> result = new LinkedHashSet<>(shardingRule.getDatabaseShardingStrategy(tableRule).doSharding(tableRule.getActualDatasourceNames(), databaseShardingValues, properties));
+        Collection<String> result = new LinkedHashSet<>(databaseShardingStrategy.doSharding(tableRule.getActualDatasourceNames(), databaseShardingValues, properties));
         Preconditions.checkState(!result.isEmpty(), "no database route info");
         Preconditions.checkState(tableRule.getActualDatasourceNames().containsAll(result), 
                 "Some routed data sources do not belong to configured data sources. routed data sources: `%s`, configured data sources: `%s`", result, tableRule.getActualDatasourceNames());
         return result;
     }
     
-    private Collection<DataNode> routeTables(final ShardingRule shardingRule, final TableRule tableRule, final String routedDataSource, final List<RouteValue> tableShardingValues) {
+    private Collection<DataNode> routeTables(final TableRule tableRule, final String routedDataSource, 
+                                             final ShardingStrategy tableShardingStrategy, final List<ShardingConditionValue> tableShardingValues) {
         Collection<String> availableTargetTables = tableRule.getActualTableNames(routedDataSource);
-        Collection<String> routedTables = new LinkedHashSet<>(tableShardingValues.isEmpty() ? availableTargetTables
-                : shardingRule.getTableShardingStrategy(tableRule).doSharding(availableTargetTables, tableShardingValues, properties));
+        Collection<String> routedTables = new LinkedHashSet<>(tableShardingValues.isEmpty()
+                ? availableTargetTables : tableShardingStrategy.doSharding(availableTargetTables, tableShardingValues, properties));
         Collection<DataNode> result = new LinkedList<>();
         for (String each : routedTables) {
             result.add(new DataNode(routedDataSource, each));
         }
         return result;
     }
+    
+    private ShardingStrategy createShardingStrategy(final ShardingStrategyConfiguration shardingStrategyConfig, final Map<String, ShardingAlgorithm> shardingAlgorithms) {
+        return null == shardingStrategyConfig ? new NoneShardingStrategy()
+                : ShardingStrategyFactory.newInstance(shardingStrategyConfig, shardingAlgorithms.get(shardingStrategyConfig.getShardingAlgorithmName()));
+    }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategy.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/ShardingStrategy.java
similarity index 73%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategy.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/ShardingStrategy.java
index 5c5d92d..1e5e901 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategy.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/ShardingStrategy.java
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy;
+package org.apache.shardingsphere.sharding.route.strategy;
 
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 
 import java.util.Collection;
 
@@ -34,7 +34,7 @@ public interface ShardingStrategy {
      * @return sharding columns
      */
     Collection<String> getShardingColumns();
-
+    
     /**
      * Get sharding algorithm.
      *
@@ -45,10 +45,10 @@ public interface ShardingStrategy {
     /**
      * Sharding.
      *
-     * @param availableTargetNames available data sources or tables's names
-     * @param shardingValues sharding values
-     * @param props ShardingSphere properties
-     * @return sharding results for data sources or tables's names
+     * @param availableTargetNames available data source or table names
+     * @param shardingConditionValues sharding condition values
+     * @param props configuration properties
+     * @return sharding results for data source or table names
      */
-    Collection<String> doSharding(Collection<String> availableTargetNames, Collection<RouteValue> shardingValues, ConfigurationProperties props);
+    Collection<String> doSharding(Collection<String> availableTargetNames, Collection<ShardingConditionValue> shardingConditionValues, ConfigurationProperties props);
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategyFactory.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/ShardingStrategyFactory.java
similarity index 86%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategyFactory.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/ShardingStrategyFactory.java
index 9f5d60e..3d629fb 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategyFactory.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/ShardingStrategyFactory.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy;
+package org.apache.shardingsphere.sharding.route.strategy;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
@@ -26,10 +26,10 @@ import org.apache.shardingsphere.sharding.api.sharding.complex.ComplexKeysShardi
 import org.apache.shardingsphere.sharding.api.sharding.hint.HintShardingAlgorithm;
 import org.apache.shardingsphere.sharding.api.sharding.standard.StandardShardingAlgorithm;
 import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.complex.ComplexShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.hint.HintShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.none.NoneShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
+import org.apache.shardingsphere.sharding.route.strategy.type.complex.ComplexShardingStrategy;
+import org.apache.shardingsphere.sharding.route.strategy.type.hint.HintShardingStrategy;
+import org.apache.shardingsphere.sharding.route.strategy.type.none.NoneShardingStrategy;
+import org.apache.shardingsphere.sharding.route.strategy.type.standard.StandardShardingStrategy;
 
 /**
  * Sharding strategy factory.
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/complex/ComplexShardingStrategy.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/type/complex/ComplexShardingStrategy.java
similarity index 71%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/complex/ComplexShardingStrategy.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/type/complex/ComplexShardingStrategy.java
index 05a9d1d..9845ed9 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/complex/ComplexShardingStrategy.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/type/complex/ComplexShardingStrategy.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.complex;
+package org.apache.shardingsphere.sharding.route.strategy.type.complex;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Splitter;
@@ -24,10 +24,10 @@ import lombok.Getter;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.sharding.api.sharding.complex.ComplexKeysShardingAlgorithm;
 import org.apache.shardingsphere.sharding.api.sharding.complex.ComplexKeysShardingValue;
-import org.apache.shardingsphere.sharding.strategy.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.strategy.ShardingStrategy;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.RangeShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 
 import java.util.Collection;
 import java.util.HashMap;
@@ -42,9 +42,9 @@ public final class ComplexShardingStrategy implements ShardingStrategy {
     
     private final Collection<String> shardingColumns;
     
-    private final ComplexKeysShardingAlgorithm shardingAlgorithm;
+    private final ComplexKeysShardingAlgorithm<?> shardingAlgorithm;
     
-    public ComplexShardingStrategy(final String shardingColumns, final ComplexKeysShardingAlgorithm shardingAlgorithm) {
+    public ComplexShardingStrategy(final String shardingColumns, final ComplexKeysShardingAlgorithm<?> shardingAlgorithm) {
         Preconditions.checkNotNull(shardingColumns, "Sharding columns cannot be null.");
         Preconditions.checkNotNull(shardingAlgorithm, "Sharding algorithm cannot be null.");
         this.shardingColumns = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
@@ -54,15 +54,15 @@ public final class ComplexShardingStrategy implements ShardingStrategy {
     
     @SuppressWarnings("unchecked")
     @Override
-    public Collection<String> doSharding(final Collection<String> availableTargetNames, final Collection<RouteValue> shardingValues, final ConfigurationProperties props) {
-        Map<String, Collection<Comparable<?>>> columnShardingValues = new HashMap<>(shardingValues.size(), 1);
-        Map<String, Range<Comparable<?>>> columnRangeValues = new HashMap<>(shardingValues.size(), 1);
+    public Collection<String> doSharding(final Collection<String> availableTargetNames, final Collection<ShardingConditionValue> shardingConditionValues, final ConfigurationProperties props) {
+        Map<String, Collection<Comparable<?>>> columnShardingValues = new HashMap<>(shardingConditionValues.size(), 1);
+        Map<String, Range<Comparable<?>>> columnRangeValues = new HashMap<>(shardingConditionValues.size(), 1);
         String logicTableName = "";
-        for (RouteValue each : shardingValues) {
-            if (each instanceof ListRouteValue) {
-                columnShardingValues.put(each.getColumnName(), ((ListRouteValue) each).getValues());
-            } else if (each instanceof RangeRouteValue) {
-                columnRangeValues.put(each.getColumnName(), ((RangeRouteValue) each).getValueRange());
+        for (ShardingConditionValue each : shardingConditionValues) {
+            if (each instanceof ListShardingConditionValue) {
+                columnShardingValues.put(each.getColumnName(), ((ListShardingConditionValue) each).getValues());
+            } else if (each instanceof RangeShardingConditionValue) {
+                columnRangeValues.put(each.getColumnName(), ((RangeShardingConditionValue) each).getValueRange());
             }
             logicTableName = each.getTableName();
         }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/hint/HintShardingStrategy.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/type/hint/HintShardingStrategy.java
similarity index 73%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/hint/HintShardingStrategy.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/type/hint/HintShardingStrategy.java
index 93cef9c..bd2bfbb 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/hint/HintShardingStrategy.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/type/hint/HintShardingStrategy.java
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.hint;
+package org.apache.shardingsphere.sharding.route.strategy.type.hint;
 
 import com.google.common.base.Preconditions;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.sharding.api.sharding.hint.HintShardingAlgorithm;
 import org.apache.shardingsphere.sharding.api.sharding.hint.HintShardingValue;
-import org.apache.shardingsphere.sharding.strategy.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.strategy.ShardingStrategy;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 
 import java.util.Collection;
 import java.util.TreeSet;
@@ -37,9 +37,9 @@ public final class HintShardingStrategy implements ShardingStrategy {
     
     private final Collection<String> shardingColumns;
     
-    private final HintShardingAlgorithm shardingAlgorithm;
+    private final HintShardingAlgorithm<?> shardingAlgorithm;
     
-    public HintShardingStrategy(final HintShardingAlgorithm shardingAlgorithm) {
+    public HintShardingStrategy(final HintShardingAlgorithm<?> shardingAlgorithm) {
         Preconditions.checkNotNull(shardingAlgorithm, "Sharding algorithm cannot be null.");
         shardingColumns = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
         this.shardingAlgorithm = shardingAlgorithm;
@@ -47,8 +47,8 @@ public final class HintShardingStrategy implements ShardingStrategy {
     
     @SuppressWarnings("unchecked")
     @Override
-    public Collection<String> doSharding(final Collection<String> availableTargetNames, final Collection<RouteValue> shardingValues, final ConfigurationProperties props) {
-        ListRouteValue shardingValue = (ListRouteValue) shardingValues.iterator().next();
+    public Collection<String> doSharding(final Collection<String> availableTargetNames, final Collection<ShardingConditionValue> shardingConditionValues, final ConfigurationProperties props) {
+        ListShardingConditionValue<?> shardingValue = (ListShardingConditionValue) shardingConditionValues.iterator().next();
         Collection<String> shardingResult = shardingAlgorithm.doSharding(availableTargetNames, 
                 new HintShardingValue(shardingValue.getTableName(), shardingValue.getColumnName(), shardingValue.getValues()));
         Collection<String> result = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/none/NoneShardingStrategy.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/type/none/NoneShardingStrategy.java
similarity index 79%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/none/NoneShardingStrategy.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/type/none/NoneShardingStrategy.java
index 3a5fe9a..28f1679 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/none/NoneShardingStrategy.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/type/none/NoneShardingStrategy.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.none;
+package org.apache.shardingsphere.sharding.route.strategy.type.none;
 
 import lombok.Getter;
 import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.strategy.ShardingStrategy;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 
 import java.util.Collection;
@@ -33,14 +33,14 @@ import java.util.Collections;
 public final class NoneShardingStrategy implements ShardingStrategy {
     
     private final Collection<String> shardingColumns = Collections.emptyList();
-
+    
     @Override
     public ShardingAlgorithm getShardingAlgorithm() {
         return null;
     }
-
+    
     @Override
-    public Collection<String> doSharding(final Collection<String> availableTargetNames, final Collection<RouteValue> shardingValues, final ConfigurationProperties props) {
+    public Collection<String> doSharding(final Collection<String> availableTargetNames, final Collection<ShardingConditionValue> shardingConditionValues, final ConfigurationProperties props) {
         return availableTargetNames;
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/standard/StandardShardingStrategy.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/type/standard/StandardShardingStrategy.java
similarity index 71%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/standard/StandardShardingStrategy.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/type/standard/StandardShardingStrategy.java
index 52f9b0f..0d0ad05 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/standard/StandardShardingStrategy.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/strategy/type/standard/StandardShardingStrategy.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.standard;
+package org.apache.shardingsphere.sharding.route.strategy.type.standard;
 
 import com.google.common.base.Preconditions;
 import lombok.Getter;
@@ -23,10 +23,10 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.sharding.api.sharding.standard.PreciseShardingValue;
 import org.apache.shardingsphere.sharding.api.sharding.standard.RangeShardingValue;
 import org.apache.shardingsphere.sharding.api.sharding.standard.StandardShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.strategy.ShardingStrategy;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.RangeShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 
 import java.util.Collection;
 import java.util.LinkedList;
@@ -38,11 +38,11 @@ import java.util.TreeSet;
 public final class StandardShardingStrategy implements ShardingStrategy {
     
     private final String shardingColumn;
-
+    
     @Getter
-    private final StandardShardingAlgorithm shardingAlgorithm;
+    private final StandardShardingAlgorithm<?> shardingAlgorithm;
     
-    public StandardShardingStrategy(final String shardingColumn, final StandardShardingAlgorithm shardingAlgorithm) {
+    public StandardShardingStrategy(final String shardingColumn, final StandardShardingAlgorithm<?> shardingAlgorithm) {
         Preconditions.checkNotNull(shardingColumn, "Sharding column cannot be null.");
         Preconditions.checkNotNull(shardingAlgorithm, "sharding algorithm cannot be null.");
         this.shardingColumn = shardingColumn;
@@ -50,23 +50,17 @@ public final class StandardShardingStrategy implements ShardingStrategy {
     }
     
     @Override
-    public Collection<String> doSharding(final Collection<String> availableTargetNames, final Collection<RouteValue> shardingValues, final ConfigurationProperties props) {
-        RouteValue shardingValue = shardingValues.iterator().next();
-        Collection<String> shardingResult = shardingValue instanceof ListRouteValue
-                ? doSharding(availableTargetNames, (ListRouteValue) shardingValue) : doSharding(availableTargetNames, (RangeRouteValue) shardingValue);
+    public Collection<String> doSharding(final Collection<String> availableTargetNames, final Collection<ShardingConditionValue> shardingConditionValues, final ConfigurationProperties props) {
+        ShardingConditionValue shardingConditionValue = shardingConditionValues.iterator().next();
+        Collection<String> shardingResult = shardingConditionValue instanceof ListShardingConditionValue
+                ? doSharding(availableTargetNames, (ListShardingConditionValue) shardingConditionValue) : doSharding(availableTargetNames, (RangeShardingConditionValue) shardingConditionValue);
         Collection<String> result = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
         result.addAll(shardingResult);
         return result;
     }
     
     @SuppressWarnings("unchecked")
-    private Collection<String> doSharding(final Collection<String> availableTargetNames, final RangeRouteValue<?> shardingValue) {
-        return shardingAlgorithm.doSharding(availableTargetNames,
-                new RangeShardingValue(shardingValue.getTableName(), shardingValue.getColumnName(), shardingValue.getValueRange()));
-    }
-    
-    @SuppressWarnings("unchecked")
-    private Collection<String> doSharding(final Collection<String> availableTargetNames, final ListRouteValue<?> shardingValue) {
+    private Collection<String> doSharding(final Collection<String> availableTargetNames, final ListShardingConditionValue<?> shardingValue) {
         Collection<String> result = new LinkedList<>();
         for (Comparable<?> each : shardingValue.getValues()) {
             String target;
@@ -78,6 +72,12 @@ public final class StandardShardingStrategy implements ShardingStrategy {
         return result;
     }
     
+    @SuppressWarnings("unchecked")
+    private Collection<String> doSharding(final Collection<String> availableTargetNames, final RangeShardingConditionValue<?> shardingValue) {
+        return shardingAlgorithm.doSharding(availableTargetNames,
+                new RangeShardingValue(shardingValue.getTableName(), shardingValue.getColumnName(), shardingValue.getValueRange()));
+    }
+    
     @Override
     public Collection<String> getShardingColumns() {
         Collection<String> result = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueBetweenOperatorGeneratorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueBetweenOperatorGeneratorTest.java
index 0094a78..99e7041 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueBetweenOperatorGeneratorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueBetweenOperatorGeneratorTest.java
@@ -18,8 +18,8 @@
 package org.apache.shardingsphere.sharding.route.engine.condition.generator.impl;
 
 import org.apache.shardingsphere.sharding.route.engine.condition.Column;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.RangeShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.complex.CommonExpressionSegment;
@@ -50,13 +50,13 @@ public final class ConditionValueBetweenOperatorGeneratorTest {
         ExpressionSegment betweenSegment = new LiteralExpressionSegment(0, 0, between);
         ExpressionSegment andSegment = new LiteralExpressionSegment(0, 0, and);
         BetweenExpression value = new BetweenExpression(0, 0, null, betweenSegment, andSegment, false);
-        Optional<RouteValue> routeValue = generator.generate(value, column, new LinkedList<>());
-        assertTrue(routeValue.isPresent());
-        RangeRouteValue<Integer> rangeRouteValue = (RangeRouteValue<Integer>) routeValue.get();
-        assertThat(rangeRouteValue.getColumnName(), is(column.getName()));
-        assertThat(rangeRouteValue.getTableName(), is(column.getTableName()));
-        assertTrue(rangeRouteValue.getValueRange().contains(between));
-        assertTrue(rangeRouteValue.getValueRange().contains(and));
+        Optional<ShardingConditionValue> shardingConditionValue = generator.generate(value, column, new LinkedList<>());
+        assertTrue(shardingConditionValue.isPresent());
+        RangeShardingConditionValue<Integer> rangeShardingConditionValue = (RangeShardingConditionValue<Integer>) shardingConditionValue.get();
+        assertThat(rangeShardingConditionValue.getColumnName(), is(column.getName()));
+        assertThat(rangeShardingConditionValue.getTableName(), is(column.getTableName()));
+        assertTrue(rangeShardingConditionValue.getValueRange().contains(between));
+        assertTrue(rangeShardingConditionValue.getValueRange().contains(and));
     }
     
     @SuppressWarnings("unchecked")
@@ -67,13 +67,13 @@ public final class ConditionValueBetweenOperatorGeneratorTest {
         ExpressionSegment betweenSegment = new LiteralExpressionSegment(0, 0, between);
         ExpressionSegment andSegment = new LiteralExpressionSegment(0, 0, and);
         BetweenExpression value = new BetweenExpression(0, 0, null, betweenSegment, andSegment, false);
-        Optional<RouteValue> routeValue = generator.generate(value, column, new LinkedList<>());
-        assertTrue(routeValue.isPresent());
-        RangeRouteValue<Comparable<?>> rangeRouteValue = (RangeRouteValue<Comparable<?>>) routeValue.get();
-        assertThat(rangeRouteValue.getColumnName(), is(column.getName()));
-        assertThat(rangeRouteValue.getTableName(), is(column.getTableName()));
-        assertTrue(SafeNumberOperationUtils.safeContains(rangeRouteValue.getValueRange(), between));
-        assertTrue(SafeNumberOperationUtils.safeContains(rangeRouteValue.getValueRange(), and));
+        Optional<ShardingConditionValue> shardingConditionValue = generator.generate(value, column, new LinkedList<>());
+        assertTrue(shardingConditionValue.isPresent());
+        RangeShardingConditionValue<Comparable<?>> rangeShardingConditionValue = (RangeShardingConditionValue<Comparable<?>>) shardingConditionValue.get();
+        assertThat(rangeShardingConditionValue.getColumnName(), is(column.getName()));
+        assertThat(rangeShardingConditionValue.getTableName(), is(column.getTableName()));
+        assertTrue(SafeNumberOperationUtils.safeContains(rangeShardingConditionValue.getValueRange(), between));
+        assertTrue(SafeNumberOperationUtils.safeContains(rangeShardingConditionValue.getValueRange(), and));
     }
     
     @Test(expected = ClassCastException.class)
@@ -92,12 +92,12 @@ public final class ConditionValueBetweenOperatorGeneratorTest {
         ExpressionSegment betweenSegment = new LiteralExpressionSegment(0, 0, date);
         ExpressionSegment andSegment = new CommonExpressionSegment(0, 0, "now()");
         BetweenExpression value = new BetweenExpression(0, 0, null, betweenSegment, andSegment, false);
-        Optional<RouteValue> routeValue = generator.generate(value, column, new LinkedList<>());
-        assertTrue(routeValue.isPresent());
-        RangeRouteValue<Date> rangeRouteValue = (RangeRouteValue<Date>) routeValue.get();
-        assertThat(rangeRouteValue.getColumnName(), is(column.getName()));
-        assertThat(rangeRouteValue.getTableName(), is(column.getTableName()));
-        assertThat(rangeRouteValue.getValueRange().lowerEndpoint(), is(date));
+        Optional<ShardingConditionValue> shardingConditionValue = generator.generate(value, column, new LinkedList<>());
+        assertTrue(shardingConditionValue.isPresent());
+        RangeShardingConditionValue<Date> rangeShardingConditionValue = (RangeShardingConditionValue<Date>) shardingConditionValue.get();
+        assertThat(rangeShardingConditionValue.getColumnName(), is(column.getName()));
+        assertThat(rangeShardingConditionValue.getTableName(), is(column.getTableName()));
+        assertThat(rangeShardingConditionValue.getValueRange().lowerEndpoint(), is(date));
     }
     
     @SuppressWarnings("unchecked")
@@ -109,11 +109,11 @@ public final class ConditionValueBetweenOperatorGeneratorTest {
         ExpressionSegment betweenSegment = new CommonExpressionSegment(0, 0, "now()");
         ExpressionSegment andSegment = new CommonExpressionSegment(0, 0, "now()");
         BetweenExpression value = new BetweenExpression(0, 0, null, betweenSegment, andSegment, false);
-        Optional<RouteValue> routeValue = generator.generate(value, column, new LinkedList<>());
-        assertTrue(routeValue.isPresent());
-        RangeRouteValue<Date> rangeRouteValue = (RangeRouteValue<Date>) routeValue.get();
-        assertThat(rangeRouteValue.getColumnName(), is(column.getName()));
-        assertThat(rangeRouteValue.getTableName(), is(column.getTableName()));
-        assertTrue(rangeRouteValue.getValueRange().upperEndpoint().before(after));
+        Optional<ShardingConditionValue> shardingConditionValue = generator.generate(value, column, new LinkedList<>());
+        assertTrue(shardingConditionValue.isPresent());
+        RangeShardingConditionValue<Date> rangeShardingConditionValue = (RangeShardingConditionValue<Date>) shardingConditionValue.get();
+        assertThat(rangeShardingConditionValue.getColumnName(), is(column.getName()));
+        assertThat(rangeShardingConditionValue.getTableName(), is(column.getTableName()));
+        assertTrue(rangeShardingConditionValue.getValueRange().upperEndpoint().before(after));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueCompareOperatorGeneratorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueCompareOperatorGeneratorTest.java
index 712b700..c051ecb 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueCompareOperatorGeneratorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueCompareOperatorGeneratorTest.java
@@ -19,9 +19,9 @@ package org.apache.shardingsphere.sharding.route.engine.condition.generator.impl
 
 import com.google.common.collect.Range;
 import org.apache.shardingsphere.sharding.route.engine.condition.Column;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.RangeShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.complex.CommonExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
@@ -44,45 +44,45 @@ public final class ConditionValueCompareOperatorGeneratorTest {
     public void assertGenerateConditionValue() {
         int value = 1;
         BinaryOperationExpression rightValue = new BinaryOperationExpression(0, 0, null, new LiteralExpressionSegment(0, 0, value), "=", null);
-        Optional<RouteValue> routeValue = generator.generate(rightValue, column, new LinkedList<>());
-        assertTrue(routeValue.isPresent());
-        assertTrue(((ListRouteValue<Integer>) routeValue.get()).getValues().contains(value));
+        Optional<ShardingConditionValue> shardingConditionValue = generator.generate(rightValue, column, new LinkedList<>());
+        assertTrue(shardingConditionValue.isPresent());
+        assertTrue(((ListShardingConditionValue<Integer>) shardingConditionValue.get()).getValues().contains(value));
     }
     
     @SuppressWarnings("unchecked")
     @Test
     public void assertGenerateConditionValueWithLessThanOperator() {
         BinaryOperationExpression rightValue = new BinaryOperationExpression(0, 0, null, new LiteralExpressionSegment(0, 0, 1), "<", null);
-        Optional<RouteValue> routeValue = generator.generate(rightValue, column, new LinkedList<>());
-        assertTrue(routeValue.isPresent());
-        assertTrue(Range.lessThan(1).encloses(((RangeRouteValue<Integer>) routeValue.get()).getValueRange()));
+        Optional<ShardingConditionValue> shardingConditionValue = generator.generate(rightValue, column, new LinkedList<>());
+        assertTrue(shardingConditionValue.isPresent());
+        assertTrue(Range.lessThan(1).encloses(((RangeShardingConditionValue<Integer>) shardingConditionValue.get()).getValueRange()));
     }
     
     @SuppressWarnings("unchecked")
     @Test
     public void assertGenerateConditionValueWithGreaterThanOperator() {
         BinaryOperationExpression rightValue = new BinaryOperationExpression(0, 0, null, new LiteralExpressionSegment(0, 0, 1), ">", null);
-        Optional<RouteValue> routeValue = generator.generate(rightValue, column, new LinkedList<>());
-        assertTrue(routeValue.isPresent());
-        assertTrue(Range.greaterThan(1).encloses(((RangeRouteValue<Integer>) routeValue.get()).getValueRange()));
+        Optional<ShardingConditionValue> shardingConditionValue = generator.generate(rightValue, column, new LinkedList<>());
+        assertTrue(shardingConditionValue.isPresent());
+        assertTrue(Range.greaterThan(1).encloses(((RangeShardingConditionValue<Integer>) shardingConditionValue.get()).getValueRange()));
     }
     
     @SuppressWarnings("unchecked")
     @Test
     public void assertGenerateConditionValueWithAtMostOperator() {
         BinaryOperationExpression rightValue = new BinaryOperationExpression(0, 0, null, new LiteralExpressionSegment(0, 0, 1), "<=", null);
-        Optional<RouteValue> routeValue = generator.generate(rightValue, column, new LinkedList<>());
-        assertTrue(routeValue.isPresent());
-        assertTrue(Range.atMost(1).encloses(((RangeRouteValue<Integer>) routeValue.get()).getValueRange()));
+        Optional<ShardingConditionValue> shardingConditionValue = generator.generate(rightValue, column, new LinkedList<>());
+        assertTrue(shardingConditionValue.isPresent());
+        assertTrue(Range.atMost(1).encloses(((RangeShardingConditionValue<Integer>) shardingConditionValue.get()).getValueRange()));
     }
     
     @SuppressWarnings("unchecked")
     @Test
     public void assertGenerateConditionValueWithAtLeastOperator() {
         BinaryOperationExpression rightValue = new BinaryOperationExpression(0, 0, null, new LiteralExpressionSegment(0, 0, 1), ">=", null);
-        Optional<RouteValue> routeValue = generator.generate(rightValue, column, new LinkedList<>());
-        assertTrue(routeValue.isPresent());
-        assertTrue(Range.atLeast(1).encloses(((RangeRouteValue<Integer>) routeValue.get()).getValueRange()));
+        Optional<ShardingConditionValue> shardingConditionValue = generator.generate(rightValue, column, new LinkedList<>());
+        assertTrue(shardingConditionValue.isPresent());
+        assertTrue(Range.atLeast(1).encloses(((RangeShardingConditionValue<Integer>) shardingConditionValue.get()).getValueRange()));
     }
     
     @Test
@@ -101,8 +101,8 @@ public final class ConditionValueCompareOperatorGeneratorTest {
     @Test
     public void assertGenerateConditionValueWithNowExpression() {
         BinaryOperationExpression rightValue = new BinaryOperationExpression(0, 0, null, new LiteralExpressionSegment(0, 0, "now()"), "=", null);
-        Optional<RouteValue> routeValue = generator.generate(rightValue, column, new LinkedList<>());
-        assertTrue(routeValue.isPresent());
-        assertFalse(((ListRouteValue<Integer>) routeValue.get()).getValues().isEmpty());
+        Optional<ShardingConditionValue> shardingConditionValue = generator.generate(rightValue, column, new LinkedList<>());
+        assertTrue(shardingConditionValue.isPresent());
+        assertFalse(((ListShardingConditionValue<Integer>) shardingConditionValue.get()).getValues().isEmpty());
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueInOperatorGeneratorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueInOperatorGeneratorTest.java
index edb4db1..f514dd3 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueInOperatorGeneratorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/generator/impl/ConditionValueInOperatorGeneratorTest.java
@@ -18,8 +18,8 @@
 package org.apache.shardingsphere.sharding.route.engine.condition.generator.impl;
 
 import org.apache.shardingsphere.sharding.route.engine.condition.Column;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ListExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.complex.CommonExpressionSegment;
@@ -44,8 +44,8 @@ public final class ConditionValueInOperatorGeneratorTest {
         ListExpression listExpression = new ListExpression(0, 0);
         listExpression.getItems().add(new CommonExpressionSegment(0, 0, "now()"));
         InExpression inExpression = new InExpression(0, 0, null, listExpression, false);
-        Optional<RouteValue> routeValue = generator.generate(inExpression, column, new LinkedList<>());
-        assertTrue(routeValue.isPresent());
-        assertThat(((ListRouteValue) routeValue.get()).getValues().iterator().next(), instanceOf(Date.class));
+        Optional<ShardingConditionValue> shardingConditionValue = generator.generate(inExpression, column, new LinkedList<>());
+        assertTrue(shardingConditionValue.isPresent());
+        assertThat(((ListShardingConditionValue) shardingConditionValue.get()).getValues().iterator().next(), instanceOf(Date.class));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/value/ListRouteValueTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/value/ListShardingConditionValueTest.java
similarity index 72%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/value/ListRouteValueTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/value/ListShardingConditionValueTest.java
index 2717ae8..c0c526f 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/value/ListRouteValueTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/value/ListShardingConditionValueTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.value;
+package org.apache.shardingsphere.sharding.route.engine.condition.value;
 
 import org.junit.Test;
 
@@ -25,15 +25,15 @@ import java.util.Collections;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 
-public final class ListRouteValueTest {
+public final class ListShardingConditionValueTest {
     
     @Test
     public void assertToStringWithEqual() {
-        assertThat(new ListRouteValue<>("order_id", "t_order", Collections.singleton(10)).toString(), is("t_order.order_id = 10"));
+        assertThat(new ListShardingConditionValue<>("order_id", "t_order", Collections.singleton(10)).toString(), is("t_order.order_id = 10"));
     }
     
     @Test
     public void assertToStringWithIn() {
-        assertThat(new ListRouteValue<>("order_id", "t_order", Arrays.asList(10, 20)).toString(), is("t_order.order_id in (10,20)"));
+        assertThat(new ListShardingConditionValue<>("order_id", "t_order", Arrays.asList(10, 20)).toString(), is("t_order.order_id in (10,20)"));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/fixture/AbstractRoutingEngineTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/fixture/AbstractRoutingEngineTest.java
index 041e1fd..5e57b6c 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/fixture/AbstractRoutingEngineTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/fixture/AbstractRoutingEngineTest.java
@@ -26,8 +26,8 @@ import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardS
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingCondition;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingConditions;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -160,11 +160,11 @@ public abstract class AbstractRoutingEngineTest {
     
     protected final ShardingConditions createShardingConditions(final String tableName) {
         List<ShardingCondition> result = new ArrayList<>(1);
-        RouteValue shardingValue1 = new ListRouteValue<>("user_id", tableName, Collections.singleton(1L));
-        RouteValue shardingValue2 = new ListRouteValue<>("order_id", tableName, Collections.singleton(1L));
+        ShardingConditionValue shardingConditionValue1 = new ListShardingConditionValue<>("user_id", tableName, Collections.singleton(1L));
+        ShardingConditionValue shardingConditionValue2 = new ListShardingConditionValue<>("order_id", tableName, Collections.singleton(1L));
         ShardingCondition shardingCondition = new ShardingCondition();
-        shardingCondition.getRouteValues().add(shardingValue1);
-        shardingCondition.getRouteValues().add(shardingValue2);
+        shardingCondition.getValues().add(shardingConditionValue1);
+        shardingCondition.getValues().add(shardingConditionValue2);
         result.add(shardingCondition);
         return new ShardingConditions(result);
     }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/fixture/HintShardingAlgorithmFixture.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/fixture/HintShardingAlgorithmFixture.java
deleted file mode 100644
index b8bdb46..0000000
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/fixture/HintShardingAlgorithmFixture.java
+++ /dev/null
@@ -1,53 +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.sharding.route.fixture;
-
-import lombok.Getter;
-import lombok.Setter;
-import org.apache.shardingsphere.sharding.api.sharding.hint.HintShardingAlgorithm;
-import org.apache.shardingsphere.sharding.api.sharding.hint.HintShardingValue;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Properties;
-
-@Getter
-@Setter
-public final class HintShardingAlgorithmFixture implements HintShardingAlgorithm<Integer> {
-    
-    private Properties props = new Properties();
-    
-    @Override
-    public void init() {
-    }
-    
-    @Override
-    public Collection<String> doSharding(final Collection<String> availableTargetNames, final HintShardingValue<Integer> shardingValue) {
-        for (String each : availableTargetNames) {
-            if (each.endsWith(String.valueOf(shardingValue.getValues().iterator().next() % 2))) {
-                return Collections.singletonList(each);
-            }
-        }
-        return null;
-    }
-    
-    @Override
-    public String getType() {
-        return "HINT_TEST";
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategyTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/strategy/ShardingStrategyTest.java
similarity index 60%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategyTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/strategy/ShardingStrategyTest.java
index 926ccc7..b6ed159 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategyTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/strategy/ShardingStrategyTest.java
@@ -15,20 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy;
+package org.apache.shardingsphere.sharding.route.strategy;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Range;
 import com.google.common.collect.Sets;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.sharding.strategy.fixture.StandardShardingAlgorithmFixture;
-import org.apache.shardingsphere.sharding.strategy.complex.ComplexShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.fixture.ComplexKeysShardingAlgorithmFixture;
-import org.apache.shardingsphere.sharding.strategy.none.NoneShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
+import org.apache.shardingsphere.sharding.route.strategy.type.complex.ComplexShardingStrategy;
+import org.apache.shardingsphere.sharding.route.strategy.fixture.ComplexKeysShardingAlgorithmFixture;
+import org.apache.shardingsphere.sharding.route.strategy.fixture.StandardShardingAlgorithmFixture;
+import org.apache.shardingsphere.sharding.route.strategy.type.none.NoneShardingStrategy;
+import org.apache.shardingsphere.sharding.route.strategy.type.standard.StandardShardingStrategy;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.RangeShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.junit.Test;
 
 import java.util.Collection;
@@ -52,15 +52,15 @@ public final class ShardingStrategyTest {
     @Test
     public void assertDoShardingForBetweenSingleKey() {
         StandardShardingStrategy strategy = new StandardShardingStrategy("column", new StandardShardingAlgorithmFixture());
-        assertThat(strategy.doSharding(targets, Collections.singletonList(new RangeRouteValue<>("column", "logicTable", Range.open(1, 3))), new ConfigurationProperties(new Properties())),
+        assertThat(strategy.doSharding(targets, Collections.singletonList(new RangeShardingConditionValue<>("column", "logicTable", Range.open(1, 3))), new ConfigurationProperties(new Properties())),
                 is(Sets.newHashSet("1")));
     }
     
     @Test
     public void assertDoShardingForMultipleKeys() {
         ComplexShardingStrategy strategy = new ComplexShardingStrategy("column1, column2", new ComplexKeysShardingAlgorithmFixture());
-        List<RouteValue> routeValues = Lists.newArrayList(
-                new ListRouteValue<>("column1", "logicTable", Collections.singletonList(1)), new RangeRouteValue<>("column2", "logicTable", Range.open(1, 3)));
-        assertThat(strategy.doSharding(targets, routeValues, new ConfigurationProperties(new Properties())), is(Sets.newHashSet("1", "2", "3")));
+        List<ShardingConditionValue> shardingConditionValues = Lists.newArrayList(
+                new ListShardingConditionValue<>("column1", "logicTable", Collections.singletonList(1)), new RangeShardingConditionValue<>("column2", "logicTable", Range.open(1, 3)));
+        assertThat(strategy.doSharding(targets, shardingConditionValues, new ConfigurationProperties(new Properties())), is(Sets.newHashSet("1", "2", "3")));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/ComplexKeysShardingAlgorithmFixture.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/strategy/fixture/ComplexKeysShardingAlgorithmFixture.java
similarity index 96%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/ComplexKeysShardingAlgorithmFixture.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/strategy/fixture/ComplexKeysShardingAlgorithmFixture.java
index c537aaf..09c590a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/ComplexKeysShardingAlgorithmFixture.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/strategy/fixture/ComplexKeysShardingAlgorithmFixture.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.fixture;
+package org.apache.shardingsphere.sharding.route.strategy.fixture;
 
 import lombok.Getter;
 import lombok.Setter;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/HintShardingAlgorithmFixture.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/strategy/fixture/HintShardingAlgorithmFixture.java
similarity index 96%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/HintShardingAlgorithmFixture.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/strategy/fixture/HintShardingAlgorithmFixture.java
index c48edcc..dc03573 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/HintShardingAlgorithmFixture.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/strategy/fixture/HintShardingAlgorithmFixture.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.fixture;
+package org.apache.shardingsphere.sharding.route.strategy.fixture;
 
 import lombok.Getter;
 import lombok.Setter;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/StandardShardingAlgorithmFixture.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/strategy/fixture/StandardShardingAlgorithmFixture.java
similarity index 97%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/StandardShardingAlgorithmFixture.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/strategy/fixture/StandardShardingAlgorithmFixture.java
index 2f7d834..d68e905 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/StandardShardingAlgorithmFixture.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/strategy/fixture/StandardShardingAlgorithmFixture.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.fixture;
+package org.apache.shardingsphere.sharding.route.strategy.fixture;
 
 import lombok.Getter;
 import lombok.Setter;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.ShardingAlgorithm b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.ShardingAlgorithm
index 8f820b9..9fb1b1c 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.ShardingAlgorithm
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.ShardingAlgorithm
@@ -15,5 +15,6 @@
 # limitations under the License.
 #
  
-org.apache.shardingsphere.sharding.route.fixture.HintShardingAlgorithmFixture
-
+org.apache.shardingsphere.sharding.route.strategy.fixture.ComplexKeysShardingAlgorithmFixture
+org.apache.shardingsphere.sharding.route.strategy.fixture.HintShardingAlgorithmFixture
+org.apache.shardingsphere.sharding.route.strategy.fixture.StandardShardingAlgorithmFixture
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/SpringBootStarterTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/SpringBootStarterTest.java
index b5e9566..b056e13 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/SpringBootStarterTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/SpringBootStarterTest.java
@@ -33,7 +33,6 @@ import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineShardi
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
 import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.ShardingStrategy;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.springframework.boot.autoconfigure.SpringBootApplication;
@@ -105,14 +104,6 @@ public class SpringBootStarterTest {
         assertThat(tableRule.getActualDatasourceNames(), is(Sets.newHashSet("ds_0", "ds_1")));
         assertThat(tableRule.getDataNodeGroups(), is(DataNodeUtil.getDataNodeGroups(dataNodes)));
         assertThat(tableRule.getDatasourceToTablesMap(), is(ImmutableMap.of("ds_1", Sets.newHashSet("t_order_0", "t_order_1"), "ds_0", Sets.newHashSet("t_order_0", "t_order_1"))));
-        ShardingStrategy databaseShardingStrategy = tableRule.getDatabaseShardingStrategy();
-        assertNotNull(databaseShardingStrategy);
-        assertThat(databaseShardingStrategy.getShardingColumns(), is(Sets.newTreeSet(Collections.singleton("user_id"))));
-        assertThat(databaseShardingStrategy.getShardingAlgorithm().getProps().getProperty("algorithm-expression"), is("ds_$->{user_id % 2}"));
-        ShardingStrategy tableShardingStrategy = tableRule.getTableShardingStrategy();
-        assertNotNull(tableShardingStrategy);
-        assertThat(tableShardingStrategy.getShardingColumns(), is(Sets.newTreeSet(Collections.singleton("order_id"))));
-        assertThat(tableShardingStrategy.getShardingAlgorithm().getProps().getProperty("algorithm-expression"), is("t_order_$->{order_id % 2}"));
     }
     
     private void assertPrimaryReplicaReplicationRule(final PrimaryReplicaReplicationRule rule) {
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/SpringNamespaceTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/SpringNamespaceTest.java
index f38b04c..7ebe918 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/SpringNamespaceTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/SpringNamespaceTest.java
@@ -74,8 +74,6 @@ public final class SpringNamespaceTest extends AbstractJUnit4SpringContextTests
         assertThat(rule.getTableRule("t_order").getActualDataNodes(), is(Arrays.asList(
                 new DataNode("ds_0.t_order_0"), new DataNode("ds_0.t_order_1"), new DataNode("ds_0.t_order_2"), new DataNode("ds_0.t_order_3"),
                 new DataNode("ds_1.t_order_0"), new DataNode("ds_1.t_order_1"), new DataNode("ds_1.t_order_2"), new DataNode("ds_1.t_order_3"))));
-        assertThat(rule.getTableRule("t_order").getDatabaseShardingStrategy().getShardingAlgorithm().getProps().getProperty("algorithm-expression"), is("ds_$->{user_id % 2}"));
-        assertThat(rule.getTableRule("t_order").getTableShardingStrategy().getShardingAlgorithm().getProps().getProperty("algorithm-expression"), is("t_order_$->{order_id % 4}"));
     }
     
     private void assertPrimaryReplicaReplicationRule(final PrimaryReplicaReplicationRule rule) {
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/governance/type/GovernanceSpringBootRegistryShardingTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/governance/type/GovernanceSpringBootRegi [...]
index aa06b42..16a6cd9 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/governance/type/GovernanceSpringBootRegistryShardingTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/governance/type/GovernanceSpringBootRegistryShardingTest.java
@@ -25,7 +25,6 @@ import org.apache.shardingsphere.infra.context.schema.SchemaContexts;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
-import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
 import org.apache.shardingsphere.spring.boot.governance.registry.TestGovernanceRepository;
 import org.apache.shardingsphere.spring.boot.governance.util.EmbedTestingServer;
 import org.junit.BeforeClass;
@@ -45,7 +44,6 @@ import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.stream.Collectors;
 
-import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
@@ -116,12 +114,8 @@ public class GovernanceSpringBootRegistryShardingTest {
         assertTrue(itemRule.getActualDataNodes().contains(new DataNode("ds_0", "t_order_item_1")));
         assertTrue(itemRule.getActualDataNodes().contains(new DataNode("ds_1", "t_order_item_0")));
         assertTrue(itemRule.getActualDataNodes().contains(new DataNode("ds_1", "t_order_item_1")));
-        assertThat(itemRule.getTableShardingStrategy(), instanceOf(StandardShardingStrategy.class));
-        assertThat(itemRule.getTableShardingStrategy().getShardingColumns().iterator().next(), is("order_id"));
         assertTrue(itemRule.getGenerateKeyColumn().isPresent());
         assertThat(itemRule.getGenerateKeyColumn().get(), is("order_item_id"));
-        assertThat(itemRule.getTableShardingStrategy(), instanceOf(StandardShardingStrategy.class));
-        assertThat(itemRule.getTableShardingStrategy().getShardingColumns().iterator().next(), is("order_id"));
     }
     
     @Test
@@ -143,12 +137,8 @@ public class GovernanceSpringBootRegistryShardingTest {
         assertTrue(itemRule.getActualDataNodes().contains(new DataNode("ds_0", "t_order_item_1")));
         assertTrue(itemRule.getActualDataNodes().contains(new DataNode("ds_1", "t_order_item_0")));
         assertTrue(itemRule.getActualDataNodes().contains(new DataNode("ds_1", "t_order_item_1")));
-        assertThat(itemRule.getTableShardingStrategy(), instanceOf(StandardShardingStrategy.class));
-        assertThat(itemRule.getTableShardingStrategy().getShardingColumns().iterator().next(), is("order_id"));
         assertTrue(itemRule.getGenerateKeyColumn().isPresent());
         assertThat(itemRule.getGenerateKeyColumn().get(), is("order_item_id"));
-        assertThat(itemRule.getTableShardingStrategy(), instanceOf(StandardShardingStrategy.class));
-        assertThat(itemRule.getTableShardingStrategy().getShardingColumns().iterator().next(), is("order_id"));
         assertTrue(orderRule.getGenerateKeyColumn().isPresent());
         assertThat(orderRule.getGenerateKeyColumn().get(), is("order_id"));
         
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/governance/type/GovernanceSpringBootShardingTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/governance/type/GovernanceSpringBootShardingTest.java
index 6f7c9c9..217ee5a 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/governance/type/GovernanceSpringBootShardingTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/governance/type/GovernanceSpringBootShardingTest.java
@@ -25,7 +25,6 @@ import org.apache.shardingsphere.infra.context.schema.SchemaContexts;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
-import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
 import org.apache.shardingsphere.spring.boot.governance.util.EmbedTestingServer;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -39,7 +38,6 @@ import javax.annotation.Resource;
 import javax.sql.DataSource;
 import java.lang.reflect.Field;
 
-import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
@@ -100,12 +98,8 @@ public class GovernanceSpringBootShardingTest {
         assertTrue(itemRule.getActualDataNodes().contains(new DataNode("ds_0", "t_order_item_1")));
         assertTrue(itemRule.getActualDataNodes().contains(new DataNode("ds_1", "t_order_item_0")));
         assertTrue(itemRule.getActualDataNodes().contains(new DataNode("ds_1", "t_order_item_1")));
-        assertThat(itemRule.getTableShardingStrategy(), instanceOf(StandardShardingStrategy.class));
-        assertThat(itemRule.getTableShardingStrategy().getShardingColumns().iterator().next(), is("order_id"));
         assertTrue(itemRule.getGenerateKeyColumn().isPresent());
         assertThat(itemRule.getGenerateKeyColumn().get(), is("order_item_id"));
-        assertThat(itemRule.getTableShardingStrategy(), instanceOf(StandardShardingStrategy.class));
-        assertThat(itemRule.getTableShardingStrategy().getShardingColumns().iterator().next(), is("order_id"));
         
     }
     
@@ -128,12 +122,8 @@ public class GovernanceSpringBootShardingTest {
         assertTrue(itemRule.getActualDataNodes().contains(new DataNode("ds_0", "t_order_item_1")));
         assertTrue(itemRule.getActualDataNodes().contains(new DataNode("ds_1", "t_order_item_0")));
         assertTrue(itemRule.getActualDataNodes().contains(new DataNode("ds_1", "t_order_item_1")));
-        assertThat(itemRule.getTableShardingStrategy(), instanceOf(StandardShardingStrategy.class));
-        assertThat(itemRule.getTableShardingStrategy().getShardingColumns().iterator().next(), is("order_id"));
         assertTrue(itemRule.getGenerateKeyColumn().isPresent());
         assertThat(itemRule.getGenerateKeyColumn().get(), is("order_item_id"));
-        assertThat(itemRule.getTableShardingStrategy(), instanceOf(StandardShardingStrategy.class));
-        assertThat(itemRule.getTableShardingStrategy().getShardingColumns().iterator().next(), is("order_id"));
         assertTrue(orderRule.getGenerateKeyColumn().isPresent());
         assertThat(orderRule.getGenerateKeyColumn().get(), is("order_id"));
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/governance/GovernanceShardingNamespaceTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/governance/GovernanceShardingNamespaceTest.java
index 0d99dd9..4dfb169 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/governance/GovernanceShardingNamespaceTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/governance/GovernanceShardingNamespaceTest.java
@@ -21,7 +21,6 @@ import org.apache.shardingsphere.driver.governance.internal.datasource.Governanc
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.context.schema.SchemaContexts;
 import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.rule.BindingTableRule;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
@@ -66,10 +65,6 @@ public class GovernanceShardingNamespaceTest extends AbstractJUnit4SpringContext
         ShardingRule shardingRule = getShardingRule("shardingRuleWithAttributesDataSourceGovernance");
         assertNotNull(dataSourceMap.get("dbtbl_0"));
         assertNotNull(dataSourceMap.get("dbtbl_1"));
-        assertThat(shardingRule.getDefaultDatabaseShardingStrategy().getShardingColumns().toArray(new String[]{}), 
-                is(new String[]{applicationContext.getBean("standardStrategy", StandardShardingStrategyConfiguration.class).getShardingColumn()}));
-        assertThat(shardingRule.getDefaultTableShardingStrategy().getShardingColumns().toArray(new String[]{}),
-                is(new String[]{applicationContext.getBean("inlineStrategy", StandardShardingStrategyConfiguration.class).getShardingColumn()}));
         assertThat(shardingRule.getDefaultKeyGenerateAlgorithm().getClass().getName(), is(IncrementKeyGenerateAlgorithm.class.getCanonicalName()));
     }
     
@@ -88,10 +83,6 @@ public class GovernanceShardingNamespaceTest extends AbstractJUnit4SpringContext
         assertTrue(tableRule.getActualDataNodes().contains(new DataNode("dbtbl_1", "t_order_1")));
         assertTrue(tableRule.getActualDataNodes().contains(new DataNode("dbtbl_1", "t_order_2")));
         assertTrue(tableRule.getActualDataNodes().contains(new DataNode("dbtbl_1", "t_order_3")));
-        assertThat(tableRule.getDatabaseShardingStrategy().getShardingColumns().toArray(new String[]{}),
-                is(new String[]{applicationContext.getBean("standardStrategy", StandardShardingStrategyConfiguration.class).getShardingColumn()}));
-        assertThat(tableRule.getTableShardingStrategy().getShardingColumns().toArray(new String[]{}),
-                is(new String[]{applicationContext.getBean("inlineStrategy", StandardShardingStrategyConfiguration.class).getShardingColumn()}));
         assertTrue(tableRule.getGenerateKeyColumn().isPresent());
         assertThat(tableRule.getGenerateKeyColumn().get(), is("order_id"));
         assertThat(tableRule.getKeyGeneratorName(), is("incrementAlgorithm"));
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/ColumnExtractFromExpression.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/ColumnExtractor.java
similarity index 82%
rename from shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/ColumnExtractFromExpression.java
rename to shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/ColumnExtractor.java
index b61f301..9a60c1a 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/ColumnExtractFromExpression.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/ColumnExtractor.java
@@ -27,23 +27,28 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpres
 
 import java.util.Optional;
 
+/**
+ * Column extractor.
+ */
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class ColumnExtractFromExpression {
+public final class ColumnExtractor {
     
     /**
-     * Get left value if left value of expression is ColumnSegment.
+     * Get left value if left value of expression is column segment.
      *
-     * @param expression ExpressionSegment.
-     * @return ColumnSegment.
+     * @param expression expression segment
+     * @return column segment
      */
     public static Optional<ColumnSegment> extract(final ExpressionSegment expression) {
         if (expression instanceof BinaryOperationExpression && ((BinaryOperationExpression) expression).getLeft() instanceof ColumnSegment) {
             ColumnSegment column = (ColumnSegment) ((BinaryOperationExpression) expression).getLeft();
             return Optional.of(column);
-        } else if (expression instanceof InExpression && ((InExpression) expression).getLeft() instanceof ColumnSegment) {
+        }
+        if (expression instanceof InExpression && ((InExpression) expression).getLeft() instanceof ColumnSegment) {
             ColumnSegment column = (ColumnSegment) ((InExpression) expression).getLeft();
             return Optional.of(column);
-        } else if (expression instanceof BetweenExpression && ((BetweenExpression) expression).getLeft() instanceof ColumnSegment) {
+        }
+        if (expression instanceof BetweenExpression && ((BetweenExpression) expression).getLeft() instanceof ColumnSegment) {
             ColumnSegment column = (ColumnSegment) ((BetweenExpression) expression).getLeft();
             return Optional.of(column);
         }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/ExpressionBuilder.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/ExpressionBuilder.java
index 513a54a..b187f61 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/ExpressionBuilder.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/ExpressionBuilder.java
@@ -26,15 +26,18 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.OrP
 
 import java.util.Optional;
 
+/**
+ * Expression builder.
+ */
 @RequiredArgsConstructor
 public final class ExpressionBuilder {
     
     private final ExpressionSegment expression;
     
     /**
-     * Extract andPredicates.
+     * Extract and predicates.
      *
-     * @return OrPredicateSegment.
+     * @return Or predicate segment.
      */
     public OrPredicateSegment extractAndPredicates() {
         OrPredicateSegment result = new OrPredicateSegment();