You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ki...@apache.org on 2020/06/16 12:18:33 UTC

[shardingsphere] branch master updated: Use init method to process props with ShardingSphereAlgorithm to avoid getProperties with in sync codes (#6080)

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

kimmking 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 010567d   Use init method to process props with ShardingSphereAlgorithm to avoid getProperties with in sync codes (#6080)
010567d is described below

commit 010567d3c7de89c6edad8c9b530647cce42cbfe2
Author: Liang Zhang <te...@163.com>
AuthorDate: Tue Jun 16 20:18:17 2020 +0800

     Use init method to process props with ShardingSphereAlgorithm to avoid getProperties with in sync codes (#6080)
    
    * move org.apache.shardingsphere.sharding.algorithm
    
    * move org.apache.shardingsphere.sharding.algorithm.config
    
    * Add ShardingSphereAlgorithmPostProcessor for KeyGenerateAlgorithm
    
    * Use init to process props with InlineShardingAlgorithm
    
    * Use init to process props with HashModShardingAlgorithm
    
    * Use init to process props with ModShardingAlgorithm
    
    * Use init to process props with RangeShardingAlgorithm
    
    * Use init to process props with FixedIntervalShardingAlgorithm
    
    * Use init to process props with MutableIntervalShardingAlgorithm
    
    * Use init to process props with AESEncryptAlgorithm
    
    * Use init to process props with RC4EncryptAlgorithm
    
    * fix test cases
---
 .../ShardingShadowDatabasesConfiguration.java      |  2 +-
 .../encrypt/algorithm/AESEncryptAlgorithm.java     | 16 ++--
 .../encrypt/algorithm/RC4EncryptAlgorithm.java     |  2 +-
 .../encrypt/algorithm/AESEncryptAlgorithmTest.java |  3 +
 .../sharding/spi/KeyGenerateAlgorithm.java         |  3 +-
 ...AlgorithmProvidedShardingRuleConfiguration.java |  2 +-
 .../keygen/SnowflakeKeyGenerateAlgorithm.java      | 82 +++++++++---------
 .../algorithm/keygen/TimeService.java              |  2 +-
 .../algorithm/keygen/UUIDKeyGenerateAlgorithm.java |  6 +-
 .../sharding/ShardingAlgorithmException.java}      | 18 ++--
 .../datetime/FixedIntervalShardingAlgorithm.java   | 54 ++++++++----
 .../datetime/MutableIntervalShardingAlgorithm.java | 97 +++++++++++++---------
 .../sharding/inline/InlineExpressionParser.java    |  2 +-
 .../sharding/inline/InlineShardingAlgorithm.java   | 34 +++++---
 .../sharding/mod/HashModShardingAlgorithm.java     | 22 +++--
 .../sharding/mod/ModShardingAlgorithm.java         | 35 ++++----
 .../range/AbstractRangeShardingAlgorithm.java      | 21 +++--
 .../range/BoundaryBasedRangeShardingAlgorithm.java |  8 +-
 .../range/VolumeBasedRangeShardingAlgorithm.java   | 10 +--
 .../shardingsphere/sharding/rule/ShardingRule.java | 10 +--
 .../shardingsphere/sharding/rule/TableRule.java    | 12 +--
 .../AlgorithmProvidedShardingRuleBuilder.java      |  2 +-
 .../strategy/{route => }/ShardingStrategy.java     |  4 +-
 .../{route => }/ShardingStrategyFactory.java       | 10 +--
 .../complex/ComplexShardingStrategy.java           | 10 +--
 .../{route => }/hint/HintShardingStrategy.java     |  8 +-
 .../{route => }/none/NoneShardingStrategy.java     |  6 +-
 .../standard/StandardShardingStrategy.java         | 10 +--
 .../strategy/{route => }/value/ListRouteValue.java |  2 +-
 .../{route => }/value/RangeRouteValue.java         |  2 +-
 .../strategy/{route => }/value/RouteValue.java     |  2 +-
 ...hardingsphere.sharding.spi.KeyGenerateAlgorithm |  4 +-
 ...e.shardingsphere.sharding.spi.ShardingAlgorithm | 14 ++--
 .../keygen/SnowflakeKeyGenerateAlgorithmTest.java  | 22 +++--
 .../keygen/UUIDKeyGenerateAlgorithmTest.java       |  3 +-
 .../algorithm/keygen/fixture/FixedTimeService.java |  6 +-
 .../fixture/IncrementKeyGenerateAlgorithm.java     |  6 +-
 .../FixedIntervalShardingAlgorithmTest.java        | 11 ++-
 .../MutableIntervalShardingAlgorithmTest.java      | 96 +++++++++++----------
 .../inline/InlineExpressionParserTest.java         |  2 +-
 .../inline/InlineShardingAlgorithmTest.java        |  8 +-
 .../mod}/HashModShardingAlgorithmTest.java         | 11 ++-
 .../sharding/mod}/ModShardingAlgorithmTest.java    | 11 ++-
 .../BoundaryBasedRangeShardingAlgorithmTest.java   | 10 +--
 .../VolumeBasedRangeShardingAlgorithmTest.java     | 10 +--
 .../log/ShardingConfigurationLoggerTest.java       |  7 +-
 .../sharding/rule/ShardingRuleTest.java            | 12 +--
 .../sharding/rule/TableRuleTest.java               |  9 +-
 .../strategy/{route => }/ShardingStrategyTest.java | 18 ++--
 .../ComplexKeysShardingAlgorithmFixture.java       |  2 +-
 .../fixture/HintShardingAlgorithmFixture.java      |  2 +-
 .../fixture/StandardShardingAlgorithmFixture.java  |  2 +-
 .../{route => }/value/ListRouteValueTest.java      |  2 +-
 ...hardingsphere.sharding.spi.KeyGenerateAlgorithm |  6 +-
 ...e.shardingsphere.sharding.spi.ShardingAlgorithm |  6 +-
 .../fixture/KeyGenerateAlgorithmFixture.java       |  4 +
 .../route/engine/ShardingRouteDecorator.java       |  6 +-
 .../engine/condition/AlwaysFalseRouteValue.java    |  2 +-
 .../route/engine/condition/ShardingCondition.java  |  2 +-
 .../InsertClauseShardingConditionEngine.java       |  2 +-
 .../engine/WhereClauseShardingConditionEngine.java |  6 +-
 .../generator/ConditionValueGenerator.java         |  2 +-
 .../generator/ConditionValueGeneratorFactory.java  |  2 +-
 .../ConditionValueBetweenOperatorGenerator.java    |  4 +-
 .../ConditionValueCompareOperatorGenerator.java    |  6 +-
 .../impl/ConditionValueInOperatorGenerator.java    |  4 +-
 .../standard/ShardingStandardRoutingEngine.java    |  8 +-
 ...ConditionValueBetweenOperatorGeneratorTest.java |  4 +-
 ...ConditionValueCompareOperatorGeneratorTest.java |  6 +-
 .../ConditionValueInOperatorGeneratorTest.java     |  4 +-
 .../route/fixture/AbstractRoutingEngineTest.java   |  4 +-
 .../rule/ShardingRuleBeanDefinitionParser.java     |  2 +-
 .../infra/config/DataSourceConfiguration.java      |  2 +-
 .../fixture/ConstantKeyGenerateAlgorithm.java      |  4 +
 .../dbtest/cases/dataset/DataSet.java              |  2 +-
 .../shardingsphere/dbtest/engine/BatchIT.java      |  2 +-
 .../dbtest/engine/dml/BaseDMLIT.java               |  2 +-
 .../env/dataset/DataSetEnvironmentManager.java     |  2 +-
 .../fixture/IncrementKeyGenerateAlgorithm.java     |  4 +
 .../ResetIncrementKeyGenerateAlgorithm.java        |  4 +
 .../driver/jdbc/adapter/DataSourceAdapterTest.java |  2 +-
 ...hardingsphere.sharding.spi.KeyGenerateAlgorithm |  4 +-
 .../fixture/DecrementKeyGenerateAlgorithm.java     |  4 +
 .../fixture/IncrementKeyGenerateAlgorithm.java     |  4 +
 ...hardingsphere.sharding.spi.KeyGenerateAlgorithm |  4 +-
 .../spring/boot/type/SpringBootShardingTest.java   |  2 +-
 .../spring/ShardingNamespaceTest.java              |  2 +-
 .../fixture/DecrementKeyGenerateAlgorithm.java     |  4 +
 .../fixture/IncrementKeyGenerateAlgorithm.java     |  4 +
 .../resources/META-INF/rdb/shardingNamespace.xml   |  2 +-
 ...rchestrationSpringBootRegistryShardingTest.java |  2 +-
 .../type/OrchestrationSpringBootShardingTest.java  |  2 +-
 .../fixture/DecrementKeyGenerateAlgorithm.java     |  4 +
 .../fixture/IncrementKeyGenerateAlgorithm.java     |  4 +
 .../boot/datasource/DataSourceMapSetter.java       |  2 +-
 95 files changed, 513 insertions(+), 400 deletions(-)

diff --git a/examples/shardingsphere-jdbc-example/other-feature-example/shadow-example/shadow-raw-jdbc-example/src/main/java/org/apache/shardingsphere/example/shadow/table/raw/jdbc/config/ShardingShadowDatabasesConfiguration.java b/examples/shardingsphere-jdbc-example/other-feature-example/shadow-example/shadow-raw-jdbc-example/src/main/java/org/apache/shardingsphere/example/shadow/table/raw/jdbc/config/ShardingShadowDatabasesConfiguration.java
index ac19cb4..d9a5fae 100644
--- a/examples/shardingsphere-jdbc-example/other-feature-example/shadow-example/shadow-raw-jdbc-example/src/main/java/org/apache/shardingsphere/example/shadow/table/raw/jdbc/config/ShardingShadowDatabasesConfiguration.java
+++ b/examples/shardingsphere-jdbc-example/other-feature-example/shadow-example/shadow-raw-jdbc-example/src/main/java/org/apache/shardingsphere/example/shadow/table/raw/jdbc/config/ShardingShadowDatabasesConfiguration.java
@@ -25,7 +25,7 @@ import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
 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.sharding.StandardShardingStrategyConfiguration;
-import org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline.InlineShardingAlgorithm;
+import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineShardingAlgorithm;
 
 import javax.sql.DataSource;
 import java.sql.SQLException;
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/algorithm/AESEncryptAlgorithm.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/algorithm/AESEncryptAlgorithm.java
index 841250c..003e08e 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/algorithm/AESEncryptAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/algorithm/AESEncryptAlgorithm.java
@@ -46,8 +46,16 @@ public final class AESEncryptAlgorithm implements EncryptAlgorithm {
     
     private Properties props = new Properties();
     
+    private byte[] secretKey;
+    
     @Override
     public void init() {
+        secretKey = createSecretKey();
+    }
+    
+    private byte[] createSecretKey() {
+        Preconditions.checkArgument(props.containsKey(AES_KEY), String.format("%s can not be null.", AES_KEY));
+        return Arrays.copyOf(DigestUtils.sha1(props.getProperty(AES_KEY)), 16);
     }
     
     @SneakyThrows
@@ -71,17 +79,11 @@ public final class AESEncryptAlgorithm implements EncryptAlgorithm {
     }
     
     private Cipher getCipher(final int decryptMode) throws NoSuchPaddingException, NoSuchAlgorithmException, InvalidKeyException {
-        Preconditions.checkArgument(props.containsKey(AES_KEY), "No available secret key for `%s`.", AESEncryptAlgorithm.class.getName());
         Cipher result = Cipher.getInstance(getType());
-        result.init(decryptMode, new SecretKeySpec(createSecretKey(), getType()));
+        result.init(decryptMode, new SecretKeySpec(secretKey, getType()));
         return result;
     }
     
-    private byte[] createSecretKey() {
-        Preconditions.checkArgument(null != props.get(AES_KEY), String.format("%s can not be null.", AES_KEY));
-        return Arrays.copyOf(DigestUtils.sha1(props.get(AES_KEY).toString()), 16);
-    }
-    
     @Override
     public String getType() {
         return "AES";
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/algorithm/RC4EncryptAlgorithm.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/algorithm/RC4EncryptAlgorithm.java
index 606906b..215ba38 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/algorithm/RC4EncryptAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/algorithm/RC4EncryptAlgorithm.java
@@ -52,7 +52,7 @@ public final class RC4EncryptAlgorithm implements EncryptAlgorithm {
     @SneakyThrows
     public void init() {
         reset();
-        setKey(StringUtils.getBytesUtf8(props.get(RC4_KEY).toString()));
+        setKey(StringUtils.getBytesUtf8(props.getProperty(RC4_KEY)));
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/algorithm/AESEncryptAlgorithmTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/algorithm/AESEncryptAlgorithmTest.java
index 049cba3..1027dac 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/algorithm/AESEncryptAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/algorithm/AESEncryptAlgorithmTest.java
@@ -35,6 +35,7 @@ public final class AESEncryptAlgorithmTest {
         Properties props = new Properties();
         props.setProperty("aes.key.value", "test");
         encryptAlgorithm.setProps(props);
+        encryptAlgorithm.init();
     }
     
     @Test
@@ -51,6 +52,7 @@ public final class AESEncryptAlgorithmTest {
     public void assertEncodeWithoutKey() {
         Properties props = new Properties();
         encryptAlgorithm.setProps(props);
+        encryptAlgorithm.init();
         assertThat(encryptAlgorithm.encrypt("test"), is("dSpPiyENQGDUXMKFMJPGWA=="));
     }
     
@@ -68,6 +70,7 @@ public final class AESEncryptAlgorithmTest {
     public void assertDecodeWithoutKey() {
         Properties props = new Properties();
         encryptAlgorithm.setProps(props);
+        encryptAlgorithm.init();
         assertThat(encryptAlgorithm.decrypt("dSpPiyENQGDUXMKFMJPGWA==").toString(), is("test"));
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/spi/KeyGenerateAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/spi/KeyGenerateAlgorithm.java
index f35e0df..4865f7a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/spi/KeyGenerateAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/spi/KeyGenerateAlgorithm.java
@@ -18,11 +18,12 @@
 package org.apache.shardingsphere.sharding.spi;
 
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithm;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmPostProcessor;
 
 /**
  * Key generate algorithm.
  */
-public interface KeyGenerateAlgorithm extends ShardingSphereAlgorithm {
+public interface KeyGenerateAlgorithm extends ShardingSphereAlgorithm, ShardingSphereAlgorithmPostProcessor {
     
     /**
      * Generate key.
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/config/AlgorithmProvidedShardingRuleConfiguration.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/config/AlgorithmProvidedShardingRuleConfiguration.java
similarity index 96%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/config/AlgorithmProvidedShardingRuleConfiguration.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/config/AlgorithmProvidedShardingRuleConfiguration.java
index c2a94e1..0214639 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/config/AlgorithmProvidedShardingRuleConfiguration.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/config/AlgorithmProvidedShardingRuleConfiguration.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.keygen.config;
+package org.apache.shardingsphere.sharding.algorithm.config;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/SnowflakeKeyGenerateAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithm.java
similarity index 73%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/SnowflakeKeyGenerateAlgorithm.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithm.java
index ee6c4d0..f90029c 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/SnowflakeKeyGenerateAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithm.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.keygen;
+package org.apache.shardingsphere.sharding.algorithm.keygen;
 
 import com.google.common.base.Preconditions;
 import lombok.Getter;
@@ -27,36 +27,25 @@ import java.util.Calendar;
 import java.util.Properties;
 
 /**
- * Snowflake distributed primary key generate algorithm.
- * 
- * <p>
- * Use snowflake algorithm. Length is 64 bit.
- * </p>
+ * Snowflake key generate algorithm.
  * 
  * <pre>
- * 1bit sign bit.
- * 41bits timestamp offset from 2016.11.01(ShardingSphere distributed primary key published data) to now.
- * 10bits worker process id.
- * 12bits auto increment offset in one mills
+ *     Length of key is 64 bit.
+ *     1 bit sign bit.
+ *     41 bits timestamp offset from 2016.11.01(ShardingSphere distributed primary key published data) to now.
+ *     10 bits worker process id.
+ *     12 bits auto increment offset in one mills
  * </pre>
- * 
- * <p>
- * Call @{@code SnowflakeKeyGenerateAlgorithm.setWorkerId} to set worker id, default value is 0.
- * </p>
- * 
- * <p>
- * Call @{@code SnowflakeKeyGenerateAlgorithm.setMaxTolerateTimeDifferenceMilliseconds} to set max tolerate time difference milliseconds, default value is 0.
- * </p>
  */
 public final class SnowflakeKeyGenerateAlgorithm implements KeyGenerateAlgorithm {
     
     public static final long EPOCH;
     
-    private static final String WORKER_ID_PROPERTY_KEY = "worker.id";
+    private static final String WORKER_ID_KEY = "worker.id";
     
-    private static final String MAX_VIBRATION_OFFSET_PROPERTY_KEY = "max.vibration.offset";
+    private static final String MAX_VIBRATION_OFFSET_KEY = "max.vibration.offset";
     
-    private static final String MAX_TOLERATE_TIME_DIFFERENCE_MILLISECONDS_PROPERTY_KEY = "max.tolerate.time.difference.milliseconds";
+    private static final String MAX_TOLERATE_TIME_DIFFERENCE_MILLISECONDS_KEY = "max.tolerate.time.difference.milliseconds";
     
     private static final long SEQUENCE_BITS = 12L;
     
@@ -83,6 +72,12 @@ public final class SnowflakeKeyGenerateAlgorithm implements KeyGenerateAlgorithm
     @Setter
     private Properties props = new Properties();
     
+    private long workerId;
+    
+    private int maxVibrationOffset;
+    
+    private int maxTolerateTimeDifferenceMilliseconds;
+    
     private int sequenceOffset = -1;
     
     private long sequence;
@@ -100,6 +95,29 @@ public final class SnowflakeKeyGenerateAlgorithm implements KeyGenerateAlgorithm
     }
     
     @Override
+    public void init() {
+        workerId = getWorkerId();
+        maxVibrationOffset = getMaxVibrationOffset();
+        maxTolerateTimeDifferenceMilliseconds = getMaxTolerateTimeDifferenceMilliseconds();
+    }
+    
+    private long getWorkerId() {
+        long result = Long.valueOf(props.getOrDefault(WORKER_ID_KEY, WORKER_ID).toString());
+        Preconditions.checkArgument(result >= 0L && result < WORKER_ID_MAX_VALUE, "Illegal worker id.");
+        return result;
+    }
+    
+    private int getMaxVibrationOffset() {
+        int result = Integer.parseInt(props.getOrDefault(MAX_VIBRATION_OFFSET_KEY, DEFAULT_VIBRATION_VALUE).toString());
+        Preconditions.checkArgument(result >= 0 && result <= SEQUENCE_MASK, "Illegal max vibration offset.");
+        return result;
+    }
+    
+    private int getMaxTolerateTimeDifferenceMilliseconds() {
+        return Integer.valueOf(props.getOrDefault(MAX_TOLERATE_TIME_DIFFERENCE_MILLISECONDS_KEY, MAX_TOLERATE_TIME_DIFFERENCE_MILLISECONDS).toString());
+    }
+    
+    @Override
     public synchronized Comparable<?> generateKey() {
         long currentMilliseconds = timeService.getCurrentMillis();
         if (waitTolerateTimeDifferenceIfNeed(currentMilliseconds)) {
@@ -114,7 +132,7 @@ public final class SnowflakeKeyGenerateAlgorithm implements KeyGenerateAlgorithm
             sequence = sequenceOffset;
         }
         lastMilliseconds = currentMilliseconds;
-        return ((currentMilliseconds - EPOCH) << TIMESTAMP_LEFT_SHIFT_BITS) | (getWorkerId() << WORKER_ID_LEFT_SHIFT_BITS) | sequence;
+        return ((currentMilliseconds - EPOCH) << TIMESTAMP_LEFT_SHIFT_BITS) | (workerId << WORKER_ID_LEFT_SHIFT_BITS) | sequence;
     }
     
     @SneakyThrows
@@ -123,28 +141,12 @@ public final class SnowflakeKeyGenerateAlgorithm implements KeyGenerateAlgorithm
             return false;
         }
         long timeDifferenceMilliseconds = lastMilliseconds - currentMilliseconds;
-        Preconditions.checkState(timeDifferenceMilliseconds < getMaxTolerateTimeDifferenceMilliseconds(), 
+        Preconditions.checkState(timeDifferenceMilliseconds < maxTolerateTimeDifferenceMilliseconds, 
                 "Clock is moving backwards, last time is %d milliseconds, current time is %d milliseconds", lastMilliseconds, currentMilliseconds);
         Thread.sleep(timeDifferenceMilliseconds);
         return true;
     }
     
-    private long getWorkerId() {
-        long result = Long.valueOf(props.getProperty(WORKER_ID_PROPERTY_KEY, String.valueOf(WORKER_ID)));
-        Preconditions.checkArgument(result >= 0L && result < WORKER_ID_MAX_VALUE);
-        return result;
-    }
-    
-    private int getMaxVibrationOffset() {
-        int result = Integer.parseInt(props.getProperty(MAX_VIBRATION_OFFSET_PROPERTY_KEY, String.valueOf(DEFAULT_VIBRATION_VALUE)));
-        Preconditions.checkArgument(result >= 0 && result <= SEQUENCE_MASK, "Illegal max vibration offset");
-        return result;
-    }
-    
-    private int getMaxTolerateTimeDifferenceMilliseconds() {
-        return Integer.valueOf(props.getProperty(MAX_TOLERATE_TIME_DIFFERENCE_MILLISECONDS_PROPERTY_KEY, String.valueOf(MAX_TOLERATE_TIME_DIFFERENCE_MILLISECONDS)));
-    }
-    
     private long waitUntilNextTime(final long lastTime) {
         long result = timeService.getCurrentMillis();
         while (result <= lastTime) {
@@ -154,7 +156,7 @@ public final class SnowflakeKeyGenerateAlgorithm implements KeyGenerateAlgorithm
     }
     
     private void vibrateSequenceOffset() {
-        sequenceOffset = sequenceOffset >= getMaxVibrationOffset() ? 0 : sequenceOffset + 1;
+        sequenceOffset = sequenceOffset >= maxVibrationOffset ? 0 : sequenceOffset + 1;
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/TimeService.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/keygen/TimeService.java
similarity index 93%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/TimeService.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/keygen/TimeService.java
index 067fd34..a2dcbf4 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/TimeService.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/keygen/TimeService.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.keygen;
+package org.apache.shardingsphere.sharding.algorithm.keygen;
 
 /**
  * Time service.
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/UUIDKeyGenerateAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/keygen/UUIDKeyGenerateAlgorithm.java
similarity index 92%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/UUIDKeyGenerateAlgorithm.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/keygen/UUIDKeyGenerateAlgorithm.java
index 79821ab..e142162 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/UUIDKeyGenerateAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/keygen/UUIDKeyGenerateAlgorithm.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.keygen;
+package org.apache.shardingsphere.sharding.algorithm.keygen;
 
 import lombok.Getter;
 import lombok.Setter;
@@ -34,6 +34,10 @@ public final class UUIDKeyGenerateAlgorithm implements KeyGenerateAlgorithm {
     private Properties props = new Properties();
     
     @Override
+    public void init() {
+    }
+    
+    @Override
     public synchronized Comparable<?> generateKey() {
         return UUID.randomUUID().toString().replaceAll("-", "");
     }
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-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/ShardingAlgorithmException.java
similarity index 65%
copy from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/AlwaysFalseRouteValue.java
copy to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/ShardingAlgorithmException.java
index 047b7f3..100d83b 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-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/ShardingAlgorithmException.java
@@ -15,22 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.condition;
+package org.apache.shardingsphere.sharding.algorithm.sharding;
 
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 
 /**
- * Always false route value.
+ * Sharding algorithm exception.
  */
-public final class AlwaysFalseRouteValue implements RouteValue {
+public final class ShardingAlgorithmException extends ShardingSphereException {
     
-    @Override
-    public String getColumnName() {
-        return "";
-    }
+    private static final long serialVersionUID = -8513765890834900694L;
     
-    @Override
-    public String getTableName() {
-        return "";
+    public ShardingAlgorithmException(final String errorMessage, final Object... args) {
+        super(errorMessage, args);
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/datetime/FixedIntervalShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/FixedIntervalShardingAlgorithm.java
similarity index 73%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/datetime/FixedIntervalShardingAlgorithm.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/FixedIntervalShardingAlgorithm.java
index 71b04f4..e2d6c9a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/datetime/FixedIntervalShardingAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/FixedIntervalShardingAlgorithm.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding.datetime;
+package org.apache.shardingsphere.sharding.algorithm.sharding.datetime;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Range;
@@ -48,27 +48,48 @@ import java.util.Properties;
 @Getter
 public final class FixedIntervalShardingAlgorithm implements StandardShardingAlgorithm<Comparable<?>>, ShardingAutoTableAlgorithm {
     
-    private static final String DATETIME_LOWER = "datetime.lower";
+    private static final String DATE_TIME_LOWER_KEY = "datetime.lower";
     
-    private static final String DATETIME_UPPER = "datetime.upper";
+    private static final String DATE_TIME_UPPER_KEY = "datetime.upper";
     
     private static final String SHARDING_SECONDS_KEY = "sharding.seconds";
     
-    private static final String DATETIME_PATTERN = "yyyy-MM-dd HH:mm:ss";
+    private static final String DATE_TIME_PATTERN = "yyyy-MM-dd HH:mm:ss";
     
-    private static final DateTimeFormatter DATE_FORMAT = DateTimeFormatter.ofPattern(DATETIME_PATTERN);
-    
-    private int autoTablesAmount;
+    private static final DateTimeFormatter DATE_FORMAT = DateTimeFormatter.ofPattern(DATE_TIME_PATTERN);
     
     @Setter
     private Properties props = new Properties();
     
+    private LocalDateTime dateTimeLower;
+    
+    private LocalDateTime dateTimeUpper;
+    
+    private long shardingSeconds;
+    
+    private int autoTablesAmount;
+    
     @Override
     public void init() {
-        Preconditions.checkNotNull(props.get(SHARDING_SECONDS_KEY), "Sharding partition volume cannot be null.");
-        Preconditions.checkState(null != props.get(DATETIME_LOWER) && checkDatetimePattern(props.get(DATETIME_LOWER).toString()), "%s pattern is required.", DATETIME_PATTERN);
-        Preconditions.checkState(null != props.get(DATETIME_UPPER) && checkDatetimePattern(props.get(DATETIME_UPPER).toString()), "%s pattern is required.", DATETIME_PATTERN);
-        autoTablesAmount = (int) (Math.ceil(parseDate(props.get(DATETIME_UPPER).toString()) / getPartitionValue()) + 2);
+        dateTimeLower = getDateTimeLower();
+        dateTimeUpper = getDateTimeUpper();
+        shardingSeconds = getShardingSeconds();
+        autoTablesAmount = (int) (Math.ceil(parseDate(props.getProperty(DATE_TIME_UPPER_KEY)) / shardingSeconds) + 2);
+    }
+    
+    private LocalDateTime getDateTimeLower() {
+        Preconditions.checkState(props.containsKey(DATE_TIME_LOWER_KEY) && checkDatetimePattern(props.getProperty(DATE_TIME_LOWER_KEY)), "%s pattern is required.", DATE_TIME_PATTERN);
+        return LocalDateTime.parse(props.getProperty(DATE_TIME_LOWER_KEY), DATE_FORMAT);
+    }
+    
+    private LocalDateTime getDateTimeUpper() {
+        Preconditions.checkState(props.containsKey(DATE_TIME_UPPER_KEY) && checkDatetimePattern(props.getProperty(DATE_TIME_UPPER_KEY)), "%s pattern is required.", DATE_TIME_PATTERN);
+        return LocalDateTime.parse(props.getProperty(DATE_TIME_UPPER_KEY), DATE_FORMAT);
+    }
+    
+    private long getShardingSeconds() {
+        Preconditions.checkArgument(props.containsKey(SHARDING_SECONDS_KEY), "Sharding seconds cannot be null.");
+        return Long.parseLong(props.get(SHARDING_SECONDS_KEY).toString());
     }
     
     @Override
@@ -78,7 +99,9 @@ public final class FixedIntervalShardingAlgorithm implements StandardShardingAlg
                 return each;
             }
         }
+        // TODO check if return null
         return null;
+        //        throw new ShardingAlgorithmException("Sharding failure, cannot find target name via `%s`", shardingValue);
     }
     
     @Override
@@ -101,7 +124,7 @@ public final class FixedIntervalShardingAlgorithm implements StandardShardingAlg
     
     private int doSharding(final long shardingValue) {
         DecimalFormat decimalFormat = new DecimalFormat("0.00");
-        String position = decimalFormat.format((float) shardingValue / getPartitionValue());
+        String position = decimalFormat.format((float) shardingValue / shardingSeconds);
         return Math.min(Math.max(0, (int) Math.ceil(Float.parseFloat(position))), autoTablesAmount - 1);
     }
     
@@ -124,12 +147,7 @@ public final class FixedIntervalShardingAlgorithm implements StandardShardingAlg
     
     private long parseDate(final Comparable<?> shardingValue) {
         LocalDateTime dateValue = LocalDateTime.parse(shardingValue.toString(), DATE_FORMAT);
-        LocalDateTime sinceDate = LocalDateTime.parse(props.get(DATETIME_LOWER).toString(), DATE_FORMAT);
-        return Duration.between(sinceDate, dateValue).toMillis() / 1000;
-    }
-    
-    private long getPartitionValue() {
-        return Long.parseLong(props.get(SHARDING_SECONDS_KEY).toString());
+        return Duration.between(dateTimeLower, dateValue).toMillis() / 1000;
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/datetime/MutableIntervalShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/MutableIntervalShardingAlgorithm.java
similarity index 71%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/datetime/MutableIntervalShardingAlgorithm.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/MutableIntervalShardingAlgorithm.java
index 0499e93..ebe9e37 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/datetime/MutableIntervalShardingAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/MutableIntervalShardingAlgorithm.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding.datetime;
+package org.apache.shardingsphere.sharding.algorithm.sharding.datetime;
 
 import com.google.common.base.Preconditions;
 import lombok.Getter;
@@ -67,46 +67,83 @@ public final class MutableIntervalShardingAlgorithm implements StandardShardingA
     
     private static final String TABLE_SUFFIX_FORMAT_KEY = "table.suffix.format";
     
-    private static final String DEFAULT_LOWER_KEY = "datetime.lower";
+    private static final String DATE_TIME_LOWER_KEY = "datetime.lower";
     
-    private static final String DEFAULT_UPPER_KEY = "datetime.upper";
+    private static final String DATE_TIME_UPPER_KEY = "datetime.upper";
     
     private static final String STEP_UNIT_KEY = "datetime.step.unit";
     
     private static final String STEP_AMOUNT_KEY = "datetime.step.amount";
     
-    private DateTimeFormatter datetimeFormatter;
+    @Getter
+    @Setter
+    private Properties props = new Properties();
+    
+    private String datetimeFormat;
+    
+    private DateTimeFormatter tableSuffixFormat;
+    
+    private String dateTimeLower;
+    
+    private String dateTimeUpper;
     
     private ChronoUnit stepUnit;
     
     private int stepAmount;
     
-    @Getter
-    @Setter
-    private Properties props = new Properties();
+    private DateTimeFormatter datetimeFormatter;
     
     @Override
     public void init() {
-        Preconditions.checkNotNull(props.getProperty(DATE_TIME_FORMAT_KEY));
-        Preconditions.checkNotNull(props.getProperty(TABLE_SUFFIX_FORMAT_KEY));
-        Preconditions.checkNotNull(props.getProperty(DEFAULT_LOWER_KEY));
-        stepUnit = null == props.getProperty(STEP_UNIT_KEY) ? ChronoUnit.DAYS : generateStepUnit();
+        datetimeFormat = getDatetimeFormat();
+        tableSuffixFormat = getTableSuffixFormat();
+        dateTimeLower = getDateTimeLower();
+        dateTimeUpper = getDateTimeUpper();
+        stepUnit = null == props.getProperty(STEP_UNIT_KEY) ? ChronoUnit.DAYS : generateStepUnit(props.getProperty(STEP_UNIT_KEY));
         stepAmount = Integer.parseInt(props.getProperty(STEP_AMOUNT_KEY, "1"));
-        datetimeFormatter = DateTimeFormatter.ofPattern(props.getProperty(DATE_TIME_FORMAT_KEY));
+        datetimeFormatter = DateTimeFormatter.ofPattern(datetimeFormat);
         try {
-            parseDateTimeForValue(props.getProperty(DEFAULT_LOWER_KEY));
-            if (props.getProperty(DEFAULT_UPPER_KEY) != null) {
-                parseDateTimeForValue(props.getProperty(DEFAULT_UPPER_KEY));
+            parseDateTimeForValue(dateTimeLower);
+            if (null != dateTimeUpper) {
+                parseDateTimeForValue(dateTimeUpper);
+            }
+        } catch (final DateTimeParseException ex) {
+            throw new UnsupportedOperationException("Cannot apply shard value for default lower/upper values", ex);
+        }
+    }
+    
+    private String getDatetimeFormat() {
+        Preconditions.checkArgument(props.containsKey(DATE_TIME_FORMAT_KEY));
+        return props.getProperty(DATE_TIME_FORMAT_KEY);
+    }
+    
+    private DateTimeFormatter getTableSuffixFormat() {
+        Preconditions.checkArgument(props.containsKey(TABLE_SUFFIX_FORMAT_KEY));
+        return DateTimeFormatter.ofPattern(props.getProperty(TABLE_SUFFIX_FORMAT_KEY));
+    }
+    
+    private String getDateTimeLower() {
+        Preconditions.checkArgument(props.containsKey(DATE_TIME_LOWER_KEY));
+        return props.getProperty(DATE_TIME_LOWER_KEY);
+    }
+    
+    private String getDateTimeUpper() {
+        return props.getProperty(DATE_TIME_UPPER_KEY);
+    }
+    
+    private ChronoUnit generateStepUnit(final String stepUnit) {
+        for (ChronoUnit unit : ChronoUnit.values()) {
+            if (unit.toString().equalsIgnoreCase(stepUnit)) {
+                return unit;
             }
-        } catch (DateTimeParseException e) {
-            throw new UnsupportedOperationException("can't apply shard value for default lower/upper values", e);
         }
+        throw new UnsupportedOperationException(String.format("Cannot find step unit for specified datetime.step.unit prop: `%s`", stepUnit));
     }
     
     @Override
     public String doSharding(final Collection<String> availableTargetNames, final PreciseShardingValue<Comparable<?>> shardingValue) {
         return availableTargetNames.stream()
-                .filter(tableName -> tableName.endsWith(formatForDateTime(parseDateTimeForValue(shardingValue.getValue().toString()))))
+                .filter(tableName -> tableName.endsWith(parseDateTimeForValue(shardingValue.getValue().toString()).format(tableSuffixFormat)))
                 .findFirst().orElseThrow(() -> new UnsupportedOperationException(
                         String.format("failed to shard value %s, and availableTables %s", shardingValue, availableTargetNames)));
     }
@@ -119,12 +156,12 @@ public final class MutableIntervalShardingAlgorithm implements StandardShardingA
         if (!hasStart && !hasEnd) {
             return availableTargetNames;
         }
-        LocalDateTime start = hasStart ? parseDateTimeForValue(shardingValue.getValueRange().lowerEndpoint().toString()) : parseDateTimeForValue(props.getProperty(DEFAULT_LOWER_KEY));
+        LocalDateTime start = hasStart ? parseDateTimeForValue(shardingValue.getValueRange().lowerEndpoint().toString()) : parseDateTimeForValue(dateTimeLower);
         LocalDateTime end = hasEnd
                 ? parseDateTimeForValue(shardingValue.getValueRange().upperEndpoint().toString())
-                : props.getProperty(DEFAULT_UPPER_KEY) == null
+                : dateTimeUpper == null
                 ? LocalDateTime.now()
-                : parseDateTimeForValue(props.getProperty(DEFAULT_UPPER_KEY));
+                : parseDateTimeForValue(dateTimeUpper);
         LocalDateTime tmp = start;
         while (!tmp.isAfter(end)) {
             mergeTableIfMatch(tmp, tables, availableTargetNames);
@@ -135,28 +172,14 @@ public final class MutableIntervalShardingAlgorithm implements StandardShardingA
     }
     
     private LocalDateTime parseDateTimeForValue(final String value) {
-        return LocalDateTime.parse(value.substring(0, props.getProperty(DATE_TIME_FORMAT_KEY).length()), datetimeFormatter);
-    }
-    
-    private String formatForDateTime(final LocalDateTime localDateTime) {
-        return localDateTime.format(DateTimeFormatter.ofPattern(props.get(TABLE_SUFFIX_FORMAT_KEY).toString()));
+        return LocalDateTime.parse(value.substring(0, datetimeFormat.length()), datetimeFormatter);
     }
     
     private void mergeTableIfMatch(final LocalDateTime dateTime, final Collection<String> tables, final Collection<String> availableTargetNames) {
-        String suffix = formatForDateTime(dateTime);
+        String suffix = dateTime.format(tableSuffixFormat);
         availableTargetNames.parallelStream().filter(tableName -> tableName.endsWith(suffix)).findAny().map(tables::add);
     }
     
-    private ChronoUnit generateStepUnit() {
-        for (ChronoUnit unit : ChronoUnit.values()) {
-            if (unit.toString().equalsIgnoreCase(props.getProperty(STEP_UNIT_KEY))) {
-                return unit;
-            }
-        }
-        throw new UnsupportedOperationException(
-                String.format("can't find step unit for specified datetime.step.unit prop: %s", props.getProperty(STEP_UNIT_KEY)));
-    }
-    
     @Override
     public String getType() {
         return "MUTABLE_INTERVAL";
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/inline/InlineExpressionParser.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineExpressionParser.java
similarity index 98%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/inline/InlineExpressionParser.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineExpressionParser.java
index 5f525ab..0360130 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/inline/InlineExpressionParser.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineExpressionParser.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline;
+package org.apache.shardingsphere.sharding.algorithm.sharding.inline;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Strings;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/inline/InlineShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineShardingAlgorithm.java
similarity index 71%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/inline/InlineShardingAlgorithm.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineShardingAlgorithm.java
index 703f992..e1a99e8 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/inline/InlineShardingAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineShardingAlgorithm.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline;
+package org.apache.shardingsphere.sharding.algorithm.sharding.inline;
 
 import com.google.common.base.Preconditions;
 import groovy.lang.Closure;
@@ -34,23 +34,35 @@ import java.util.Properties;
  */
 public final class InlineShardingAlgorithm implements StandardShardingAlgorithm<Comparable<?>> {
     
-    private static final String ALGORITHM_EXPRESSION = "algorithm.expression";
+    private static final String ALGORITHM_EXPRESSION_KEY = "algorithm.expression";
     
-    private static final String ALLOW_RANGE_QUERY = "allow.range.query.with.inline.sharding";
+    private static final String ALLOW_RANGE_QUERY_KEY = "allow.range.query.with.inline.sharding";
     
     private Closure<?> closure;
     
+    private boolean allowRangeQuery;
+    
     @Getter
     @Setter
     private Properties props = new Properties();
     
     @Override
     public void init() {
-        Preconditions.checkNotNull(props.get(ALGORITHM_EXPRESSION), "Inline sharding algorithm expression cannot be null.");
-        String algorithmExpression = InlineExpressionParser.handlePlaceHolder(props.get(ALGORITHM_EXPRESSION).toString().trim());
-        Closure<?> closure = new InlineExpressionParser(algorithmExpression).evaluateClosure();
-        this.closure = closure.rehydrate(new Expando(), null, null);
-        this.closure.setResolveStrategy(Closure.DELEGATE_ONLY);
+        closure = createClosure();
+        allowRangeQuery = isAllowRangeQuery();
+    }
+    
+    private Closure<?> createClosure() {
+        String expression = props.getProperty(ALGORITHM_EXPRESSION_KEY);
+        Preconditions.checkNotNull(expression, "Inline sharding algorithm expression cannot be null.");
+        String algorithmExpression = InlineExpressionParser.handlePlaceHolder(expression.trim());
+        Closure<?> result = new InlineExpressionParser(algorithmExpression).evaluateClosure().rehydrate(new Expando(), null, null);
+        result.setResolveStrategy(Closure.DELEGATE_ONLY);
+        return result;
+    }
+    
+    private boolean isAllowRangeQuery() {
+        return Boolean.valueOf(props.getOrDefault(ALLOW_RANGE_QUERY_KEY, Boolean.FALSE.toString()).toString());
     }
     
     @Override
@@ -61,16 +73,12 @@ public final class InlineShardingAlgorithm implements StandardShardingAlgorithm<
     
     @Override
     public Collection<String> doSharding(final Collection<String> availableTargetNames, final RangeShardingValue<Comparable<?>> shardingValue) {
-        if (isAllowRangeQuery()) {
+        if (allowRangeQuery) {
             return availableTargetNames;
         }
         throw new UnsupportedOperationException("Since the property of `allow.range.query.with.inline.sharding` is false, inline sharding algorithm can not tackle with range query.");
     }
     
-    private boolean isAllowRangeQuery() {
-        return null != props.get(ALLOW_RANGE_QUERY) && Boolean.parseBoolean(props.get(ALLOW_RANGE_QUERY).toString());
-    }
-    
     @Override
     public String getType() {
         return "INLINE";
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/mod/HashModShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/HashModShardingAlgorithm.java
similarity index 82%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/mod/HashModShardingAlgorithm.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/HashModShardingAlgorithm.java
index 79ca380..51bfbdd 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/mod/HashModShardingAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/HashModShardingAlgorithm.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding.mod;
+package org.apache.shardingsphere.sharding.algorithm.sharding.mod;
 
 import com.google.common.base.Preconditions;
 import lombok.Getter;
@@ -44,19 +44,28 @@ public final class HashModShardingAlgorithm implements StandardShardingAlgorithm
     
     private Properties props = new Properties();
     
+    private int shardingCount;
+    
     @Override
     public void init() {
-        Preconditions.checkNotNull(props.get(SHARDING_COUNT_KEY), "Modulo value cannot be null.");
+        shardingCount = getShardingCount();
+    }
+    
+    private int getShardingCount() {
+        Preconditions.checkArgument(props.containsKey(SHARDING_COUNT_KEY), "Sharding count cannot be null.");
+        return Integer.parseInt(props.getProperty(SHARDING_COUNT_KEY));
     }
     
     @Override
     public String doSharding(final Collection<String> availableTargetNames, final PreciseShardingValue<Comparable<?>> shardingValue) {
         for (String each : availableTargetNames) {
-            if (each.endsWith(hashShardingValue(shardingValue.getValue()) % getModuloValue() + "")) {
+            if (each.endsWith(hashShardingValue(shardingValue.getValue()) % shardingCount + "")) {
                 return each;
             }
         }
+        // TODO check if return null
         return null;
+        //        throw new ShardingAlgorithmException("Sharding failure, cannot find target name via `%s`", shardingValue);
     }
     
     @Override
@@ -68,14 +77,9 @@ public final class HashModShardingAlgorithm implements StandardShardingAlgorithm
         return Math.abs((long) shardingValue.hashCode());
     }
     
-    private long getModuloValue() {
-        return Long.parseLong(props.get(SHARDING_COUNT_KEY).toString());
-    }
-    
     @Override
     public int getAutoTablesAmount() {
-        Preconditions.checkNotNull(props.get(SHARDING_COUNT_KEY), "Modulo value cannot be null.");
-        return Integer.parseInt(props.get(SHARDING_COUNT_KEY).toString());
+        return shardingCount;
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/mod/ModShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/ModShardingAlgorithm.java
similarity index 82%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/mod/ModShardingAlgorithm.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/ModShardingAlgorithm.java
index 3f8d4e7..81ad315 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/mod/ModShardingAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/ModShardingAlgorithm.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding.mod;
+package org.apache.shardingsphere.sharding.algorithm.sharding.mod;
 
 import com.google.common.base.Preconditions;
 import lombok.Getter;
@@ -44,39 +44,45 @@ public final class ModShardingAlgorithm implements StandardShardingAlgorithm<Com
     
     private Properties props = new Properties();
     
+    private int shardingCount;
+    
     @Override
     public void init() {
-        Preconditions.checkNotNull(props.get(SHARDING_COUNT_KEY), "Modulo value cannot be null.");
+        shardingCount = getShardingCount();
+    }
+    
+    private int getShardingCount() {
+        Preconditions.checkArgument(props.containsKey(SHARDING_COUNT_KEY), "Sharding count cannot be null.");
+        return Integer.parseInt(props.get(SHARDING_COUNT_KEY).toString());
     }
     
     @Override
     public String doSharding(final Collection<String> availableTargetNames, final PreciseShardingValue<Comparable<?>> shardingValue) {
         for (String each : availableTargetNames) {
-            if (each.endsWith(getLongValue(shardingValue.getValue()) % getModuloValue() + "")) {
+            if (each.endsWith(getLongValue(shardingValue.getValue()) % shardingCount + "")) {
                 return each;
             }
         }
+        // TODO check if return null
         return null;
+//        throw new ShardingAlgorithmException("Sharding failure, cannot find target name via `%s`", shardingValue);
     }
     
     @Override
     public Collection<String> doSharding(final Collection<String> availableTargetNames, final RangeShardingValue<Comparable<?>> shardingValue) {
-        if (isContainAllTargets(shardingValue)) {
-            return availableTargetNames;
-        }
-        return getAvailableTargetNames(availableTargetNames, shardingValue);
+        return isContainAllTargets(shardingValue) ? availableTargetNames : getAvailableTargetNames(availableTargetNames, shardingValue);
     }
     
     private boolean isContainAllTargets(final RangeShardingValue<Comparable<?>> shardingValue) {
         return !shardingValue.getValueRange().hasUpperBound() || shardingValue.getValueRange().hasLowerBound()
-                && getLongValue(shardingValue.getValueRange().upperEndpoint()) - getLongValue(shardingValue.getValueRange().lowerEndpoint()) >= getModuloValue() - 1;
+                && getLongValue(shardingValue.getValueRange().upperEndpoint()) - getLongValue(shardingValue.getValueRange().lowerEndpoint()) >= shardingCount - 1;
     }
     
     private Collection<String> getAvailableTargetNames(final Collection<String> availableTargetNames, final RangeShardingValue<Comparable<?>> shardingValue) {
         Collection<String> result = new LinkedHashSet<>(availableTargetNames.size());
         for (long i = getLongValue(shardingValue.getValueRange().lowerEndpoint()); i <= getLongValue(shardingValue.getValueRange().upperEndpoint()); i++) {
             for (String each : availableTargetNames) {
-                if (each.endsWith(i % getModuloValue() + "")) {
+                if (each.endsWith(i % shardingCount + "")) {
                     result.add(each);
                 }
             }
@@ -84,18 +90,13 @@ public final class ModShardingAlgorithm implements StandardShardingAlgorithm<Com
         return result;
     }
     
-    private long getModuloValue() {
-        return Long.parseLong(props.get(SHARDING_COUNT_KEY).toString());
+    private long getLongValue(final Comparable<?> value) {
+        return Long.parseLong(value.toString());
     }
     
     @Override
     public int getAutoTablesAmount() {
-        Preconditions.checkNotNull(props.get(SHARDING_COUNT_KEY), "Modulo value cannot be null.");
-        return Integer.parseInt(props.get(SHARDING_COUNT_KEY).toString());
-    }
-    
-    private long getLongValue(final Comparable<?> value) {
-        return Long.parseLong(value.toString());
+        return shardingCount;
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/range/AbstractRangeShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/AbstractRangeShardingAlgorithm.java
similarity index 77%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/range/AbstractRangeShardingAlgorithm.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/AbstractRangeShardingAlgorithm.java
index e8b9a9d..c2a05fc 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/range/AbstractRangeShardingAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/AbstractRangeShardingAlgorithm.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding.range;
+package org.apache.shardingsphere.sharding.algorithm.sharding.range;
 
 import com.google.common.collect.Range;
 import lombok.Getter;
@@ -36,7 +36,7 @@ import java.util.Properties;
  */
 public abstract class AbstractRangeShardingAlgorithm implements StandardShardingAlgorithm<Long>, ShardingAutoTableAlgorithm {
     
-    private Map<Integer, Range<Long>> partitionRangeMap;
+    private volatile Map<Integer, Range<Long>> partitionRange;
     
     @Getter
     @Setter
@@ -44,15 +44,14 @@ public abstract class AbstractRangeShardingAlgorithm implements StandardSharding
     
     @Override
     public final void init() {
-        partitionRangeMap = createPartitionRangeMap(props);
+        partitionRange = calculatePartitionRange(props);
     }
     
-    abstract Map<Integer, Range<Long>> createPartitionRangeMap(Properties props);
+    protected abstract Map<Integer, Range<Long>> calculatePartitionRange(Properties props);
     
     @Override
     public final String doSharding(final Collection<String> availableTargetNames, final PreciseShardingValue<Long> shardingValue) {
-        return availableTargetNames.stream().filter(each -> each.endsWith(getPartition(partitionRangeMap, shardingValue.getValue()) + ""))
-                .findFirst().orElse(null);
+        return availableTargetNames.stream().filter(each -> each.endsWith(getPartition(shardingValue.getValue()) + "")).findFirst().orElse(null);
     }
     
     @Override
@@ -71,15 +70,15 @@ public abstract class AbstractRangeShardingAlgorithm implements StandardSharding
     }
     
     private int getFirstPartition(final Range<Long> valueRange) {
-        return valueRange.hasLowerBound() ? getPartition(partitionRangeMap, valueRange.lowerEndpoint()) : 0;
+        return valueRange.hasLowerBound() ? getPartition(valueRange.lowerEndpoint()) : 0;
     }
     
     private int getLastPartition(final Range<Long> valueRange) {
-        return valueRange.hasUpperBound() ? getPartition(partitionRangeMap, valueRange.upperEndpoint()) : partitionRangeMap.size() - 1;
+        return valueRange.hasUpperBound() ? getPartition(valueRange.upperEndpoint()) : partitionRange.size() - 1;
     }
     
-    private Integer getPartition(final Map<Integer, Range<Long>> partitionRangeMap, final Long value) {
-        for (Entry<Integer, Range<Long>> entry : partitionRangeMap.entrySet()) {
+    private Integer getPartition(final Long value) {
+        for (Entry<Integer, Range<Long>> entry : partitionRange.entrySet()) {
             if (entry.getValue().contains(value)) {
                 return entry.getKey();
             }
@@ -89,6 +88,6 @@ public abstract class AbstractRangeShardingAlgorithm implements StandardSharding
     
     @Override
     public final int getAutoTablesAmount() {
-        return partitionRangeMap.size();
+        return partitionRange.size();
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/range/BoundaryBasedRangeShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/BoundaryBasedRangeShardingAlgorithm.java
similarity index 90%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/range/BoundaryBasedRangeShardingAlgorithm.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/BoundaryBasedRangeShardingAlgorithm.java
index c7b9d02..1db1f3e 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/range/BoundaryBasedRangeShardingAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/BoundaryBasedRangeShardingAlgorithm.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding.range;
+package org.apache.shardingsphere.sharding.algorithm.sharding.range;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Splitter;
@@ -45,9 +45,9 @@ public final class BoundaryBasedRangeShardingAlgorithm extends AbstractRangeShar
     private static final String SHARDING_RANGES_KEY = "sharding.ranges";
     
     @Override
-    public Map<Integer, Range<Long>> createPartitionRangeMap(final Properties props) {
-        Preconditions.checkNotNull(props.get(SHARDING_RANGES_KEY), "Sharding ranges cannot be null.");
-        List<Long> partitionRanges = Splitter.on(",").trimResults().splitToList(props.get(SHARDING_RANGES_KEY).toString())
+    public Map<Integer, Range<Long>> calculatePartitionRange(final Properties props) {
+        Preconditions.checkState(props.containsKey(SHARDING_RANGES_KEY), "Sharding ranges cannot be null.");
+        List<Long> partitionRanges = Splitter.on(",").trimResults().splitToList(props.getProperty(SHARDING_RANGES_KEY))
                 .stream().map(Longs::tryParse).filter(Objects::nonNull).sorted().collect(Collectors.toList());
         Preconditions.checkArgument(CollectionUtils.isNotEmpty(partitionRanges), "Sharding ranges is not valid.");
         Map<Integer, Range<Long>> result = Maps.newHashMapWithExpectedSize(partitionRanges.size() + 1);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/range/VolumeBasedRangeShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/VolumeBasedRangeShardingAlgorithm.java
similarity index 86%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/range/VolumeBasedRangeShardingAlgorithm.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/VolumeBasedRangeShardingAlgorithm.java
index e998500..8e115cd 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/range/VolumeBasedRangeShardingAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/VolumeBasedRangeShardingAlgorithm.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding.range;
+package org.apache.shardingsphere.sharding.algorithm.sharding.range;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
@@ -45,10 +45,10 @@ public final class VolumeBasedRangeShardingAlgorithm extends AbstractRangeShardi
     private static final String SHARDING_VOLUME_KEY = "sharding.volume";
     
     @Override
-    public Map<Integer, Range<Long>> createPartitionRangeMap(final Properties props) {
-        Preconditions.checkNotNull(props.get(RANGE_LOWER_KEY), "Lower range cannot be null.");
-        Preconditions.checkNotNull(props.get(RANGE_UPPER_KEY), "Upper range cannot be null.");
-        Preconditions.checkNotNull(props.get(SHARDING_VOLUME_KEY), "Sharding volume cannot be null.");
+    public Map<Integer, Range<Long>> calculatePartitionRange(final Properties props) {
+        Preconditions.checkState(props.containsKey(RANGE_LOWER_KEY), "Lower range cannot be null.");
+        Preconditions.checkState(props.containsKey(RANGE_UPPER_KEY), "Upper range cannot be null.");
+        Preconditions.checkState(props.containsKey(SHARDING_VOLUME_KEY), "Sharding volume cannot be null.");
         long lower = Long.parseLong(props.get(RANGE_LOWER_KEY).toString());
         long upper = Long.parseLong(props.get(RANGE_UPPER_KEY).toString());
         long volume = Long.parseLong(props.get(SHARDING_VOLUME_KEY).toString());
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 d3aab14..5131a9c 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
@@ -33,11 +33,11 @@ import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerate
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
 import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.algorithm.keygen.config.AlgorithmProvidedShardingRuleConfiguration;
-import org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline.InlineExpressionParser;
-import org.apache.shardingsphere.sharding.strategy.route.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.ShardingStrategyFactory;
-import org.apache.shardingsphere.sharding.strategy.route.none.NoneShardingStrategy;
+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;
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 c97e989..4e10f4b 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
@@ -32,10 +32,10 @@ import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerate
 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.strategy.algorithm.sharding.inline.InlineExpressionParser;
-import org.apache.shardingsphere.sharding.strategy.route.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.ShardingStrategyFactory;
-import org.apache.shardingsphere.sharding.strategy.route.none.NoneShardingStrategy;
+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;
@@ -105,8 +105,8 @@ public final class TableRule {
         checkRule(dataNodes);
     }
     
-    public TableRule(final ShardingAutoTableRuleConfiguration tableRuleConfig, final Collection<String> dataSourceNames, 
-                     final ShardingAlgorithm shardingAlgorithm, final String defaultGenerateKeyColumn) {
+    public TableRule(final ShardingAutoTableRuleConfiguration tableRuleConfig, 
+                     final Collection<String> dataSourceNames, final ShardingAlgorithm shardingAlgorithm, final String defaultGenerateKeyColumn) {
         logicTable = tableRuleConfig.getLogicTable().toLowerCase();
         databaseShardingStrategy = new NoneShardingStrategy();
         tableShardingStrategy = createShardingStrategy(tableRuleConfig.getShardingStrategy(), shardingAlgorithm);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilder.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilder.java
index 31459c3..96f5e32 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilder.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.sharding.rule.builder;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRuleBuilder;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.sharding.strategy.algorithm.keygen.config.AlgorithmProvidedShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.algorithm.config.AlgorithmProvidedShardingRuleConfiguration;
 
 import java.util.Collection;
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/ShardingStrategy.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategy.java
similarity index 92%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/ShardingStrategy.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategy.java
index c047514..5c5d92d 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/ShardingStrategy.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategy.java
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route;
+package org.apache.shardingsphere.sharding.strategy;
 
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 
 import java.util.Collection;
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/ShardingStrategyFactory.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategyFactory.java
similarity index 87%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/ShardingStrategyFactory.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategyFactory.java
index 1110c4d..62b38f0 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/ShardingStrategyFactory.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategyFactory.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route;
+package org.apache.shardingsphere.sharding.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.route.complex.ComplexShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.hint.HintShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.none.NoneShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.standard.StandardShardingStrategy;
+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;
 
 /**
  * Sharding strategy factory.
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/complex/ComplexShardingStrategy.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/complex/ComplexShardingStrategy.java
similarity index 89%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/complex/ComplexShardingStrategy.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/complex/ComplexShardingStrategy.java
index 5d418c2..05a9d1d 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/complex/ComplexShardingStrategy.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/complex/ComplexShardingStrategy.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route.complex;
+package org.apache.shardingsphere.sharding.strategy.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.route.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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 java.util.Collection;
 import java.util.HashMap;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/hint/HintShardingStrategy.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/hint/HintShardingStrategy.java
similarity index 89%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/hint/HintShardingStrategy.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/hint/HintShardingStrategy.java
index 6a5c638..93cef9c 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/hint/HintShardingStrategy.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/hint/HintShardingStrategy.java
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route.hint;
+package org.apache.shardingsphere.sharding.strategy.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.route.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.sharding.strategy.ShardingStrategy;
+import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 
 import java.util.Collection;
 import java.util.TreeSet;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/none/NoneShardingStrategy.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/none/NoneShardingStrategy.java
similarity index 87%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/none/NoneShardingStrategy.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/none/NoneShardingStrategy.java
index c6336ae..3a5fe9a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/none/NoneShardingStrategy.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/none/NoneShardingStrategy.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route.none;
+package org.apache.shardingsphere.sharding.strategy.none;
 
 import lombok.Getter;
 import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.route.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.sharding.strategy.ShardingStrategy;
+import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 
 import java.util.Collection;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/standard/StandardShardingStrategy.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/standard/StandardShardingStrategy.java
similarity index 91%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/standard/StandardShardingStrategy.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/standard/StandardShardingStrategy.java
index 8cda60b..52f9b0f 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/standard/StandardShardingStrategy.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/standard/StandardShardingStrategy.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route.standard;
+package org.apache.shardingsphere.sharding.strategy.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.route.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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 java.util.Collection;
 import java.util.LinkedList;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/value/ListRouteValue.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/ListRouteValue.java
similarity index 95%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/value/ListRouteValue.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/ListRouteValue.java
index 33317fd..ec548ae 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/value/ListRouteValue.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/ListRouteValue.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route.value;
+package org.apache.shardingsphere.sharding.strategy.value;
 
 import com.google.common.base.Joiner;
 import lombok.Getter;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/value/RangeRouteValue.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/RangeRouteValue.java
similarity index 94%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/value/RangeRouteValue.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/RangeRouteValue.java
index 710f0b8..464a423 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/value/RangeRouteValue.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/RangeRouteValue.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route.value;
+package org.apache.shardingsphere.sharding.strategy.value;
 
 import com.google.common.collect.Range;
 import lombok.Getter;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/value/RouteValue.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/RouteValue.java
similarity index 94%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/value/RouteValue.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/RouteValue.java
index e708ff8..066e957 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/route/value/RouteValue.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/strategy/value/RouteValue.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route.value;
+package org.apache.shardingsphere.sharding.strategy.value;
 
 /**
  * Route value.
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm
index 76397f8..3ce4e58 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm
@@ -15,5 +15,5 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.sharding.strategy.algorithm.keygen.SnowflakeKeyGenerateAlgorithm
-org.apache.shardingsphere.sharding.strategy.algorithm.keygen.UUIDKeyGenerateAlgorithm
+org.apache.shardingsphere.sharding.algorithm.keygen.SnowflakeKeyGenerateAlgorithm
+org.apache.shardingsphere.sharding.algorithm.keygen.UUIDKeyGenerateAlgorithm
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.ShardingAlgorithm b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.ShardingAlgorithm
index fa0ca39..8c1a706 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.ShardingAlgorithm
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.ShardingAlgorithm
@@ -15,10 +15,10 @@
 # limitations under the License.
 #
  
-org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline.InlineShardingAlgorithm
-org.apache.shardingsphere.sharding.strategy.algorithm.sharding.mod.ModShardingAlgorithm
-org.apache.shardingsphere.sharding.strategy.algorithm.sharding.mod.HashModShardingAlgorithm
-org.apache.shardingsphere.sharding.strategy.algorithm.sharding.range.VolumeBasedRangeShardingAlgorithm
-org.apache.shardingsphere.sharding.strategy.algorithm.sharding.range.BoundaryBasedRangeShardingAlgorithm
-org.apache.shardingsphere.sharding.strategy.algorithm.sharding.datetime.FixedIntervalShardingAlgorithm
-org.apache.shardingsphere.sharding.strategy.algorithm.sharding.datetime.MutableIntervalShardingAlgorithm
+org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineShardingAlgorithm
+org.apache.shardingsphere.sharding.algorithm.sharding.mod.ModShardingAlgorithm
+org.apache.shardingsphere.sharding.algorithm.sharding.mod.HashModShardingAlgorithm
+org.apache.shardingsphere.sharding.algorithm.sharding.range.VolumeBasedRangeShardingAlgorithm
+org.apache.shardingsphere.sharding.algorithm.sharding.range.BoundaryBasedRangeShardingAlgorithm
+org.apache.shardingsphere.sharding.algorithm.sharding.datetime.FixedIntervalShardingAlgorithm
+org.apache.shardingsphere.sharding.algorithm.sharding.datetime.MutableIntervalShardingAlgorithm
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
similarity index 94%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
index 5c59375..9b9a716 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.keygen;
+package org.apache.shardingsphere.sharding.algorithm.keygen;
 
 import lombok.SneakyThrows;
-import org.apache.shardingsphere.sharding.strategy.algorithm.keygen.fixture.FixedTimeService;
+import org.apache.shardingsphere.sharding.algorithm.keygen.fixture.FixedTimeService;
 import org.junit.Test;
 
 import java.lang.reflect.Field;
@@ -50,6 +50,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
         int taskNumber = threadNumber << 2;
         SnowflakeKeyGenerateAlgorithm keyGenerateAlgorithm = new SnowflakeKeyGenerateAlgorithm();
         keyGenerateAlgorithm.setProps(new Properties());
+        keyGenerateAlgorithm.init();
         Set<Comparable<?>> actual = new HashSet<>();
         for (int i = 0; i < taskNumber; i++) {
             actual.add(executor.submit((Callable<Comparable<?>>) keyGenerateAlgorithm::generateKey).get());
@@ -60,8 +61,9 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
     @Test
     public void assertGenerateKeyWithSingleThread() {
         SnowflakeKeyGenerateAlgorithm keyGenerateAlgorithm = new SnowflakeKeyGenerateAlgorithm();
-        keyGenerateAlgorithm.setProps(new Properties());
         SnowflakeKeyGenerateAlgorithm.setTimeService(new FixedTimeService(1));
+        keyGenerateAlgorithm.setProps(new Properties());
+        keyGenerateAlgorithm.init();
         List<Comparable<?>> expected = Arrays.asList(0L, 4194305L, 4194306L, 8388608L, 8388609L, 12582913L, 12582914L, 16777216L, 16777217L, 20971521L);
         List<Comparable<?>> actual = new ArrayList<>();
         for (int i = 0; i < DEFAULT_KEY_AMOUNT; i++) {
@@ -73,10 +75,11 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
     @Test
     public void assertLastDigitalOfGenerateKeySameMillisecond() {
         SnowflakeKeyGenerateAlgorithm keyGenerateAlgorithm = new SnowflakeKeyGenerateAlgorithm();
-        Properties props = new Properties();
         SnowflakeKeyGenerateAlgorithm.setTimeService(new FixedTimeService(5));
+        Properties props = new Properties();
         props.setProperty("max.vibration.offset", String.valueOf(3));
         keyGenerateAlgorithm.setProps(props);
+        keyGenerateAlgorithm.init();
         assertThat(keyGenerateAlgorithm.generateKey(), is((Comparable) 0L));
         assertThat(keyGenerateAlgorithm.generateKey(), is((Comparable) 1L));
         assertThat(keyGenerateAlgorithm.generateKey(), is((Comparable) 2L));
@@ -91,6 +94,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
         SnowflakeKeyGenerateAlgorithm.setTimeService(new TimeService());
         props.setProperty("max.vibration.offset", String.valueOf(3));
         keyGenerateAlgorithm.setProps(props);
+        keyGenerateAlgorithm.init();
         String actualGenerateKeyBinaryString0 = Long.toBinaryString(Long.parseLong(keyGenerateAlgorithm.generateKey().toString()));
         assertThat(Integer.parseInt(actualGenerateKeyBinaryString0.substring(actualGenerateKeyBinaryString0.length() - 3), 2), is(0));
         Thread.sleep(2L);
@@ -113,6 +117,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
         TimeService timeService = new FixedTimeService(1);
         SnowflakeKeyGenerateAlgorithm.setTimeService(timeService);
         keyGenerateAlgorithm.setProps(new Properties());
+        keyGenerateAlgorithm.init();
         setLastMilliseconds(keyGenerateAlgorithm, timeService.getCurrentMillis() + 2);
         List<Comparable<?>> expected = Arrays.asList(4194304L, 8388609L, 8388610L, 12582912L, 12582913L, 16777217L, 16777218L, 20971520L, 20971521L, 25165825L);
         List<Comparable<?>> actual = new ArrayList<>();
@@ -127,10 +132,10 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
         SnowflakeKeyGenerateAlgorithm keyGenerateAlgorithm = new SnowflakeKeyGenerateAlgorithm();
         TimeService timeService = new FixedTimeService(1);
         SnowflakeKeyGenerateAlgorithm.setTimeService(timeService);
-        keyGenerateAlgorithm.setProps(new Properties());
         Properties props = new Properties();
         props.setProperty("max.tolerate.time.difference.milliseconds", String.valueOf(0));
         keyGenerateAlgorithm.setProps(props);
+        keyGenerateAlgorithm.init();
         setLastMilliseconds(keyGenerateAlgorithm, timeService.getCurrentMillis() + 2);
         List<Comparable<?>> actual = new ArrayList<>();
         for (int i = 0; i < DEFAULT_KEY_AMOUNT; i++) {
@@ -145,6 +150,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
         TimeService timeService = new FixedTimeService(2);
         SnowflakeKeyGenerateAlgorithm.setTimeService(timeService);
         keyGenerateAlgorithm.setProps(new Properties());
+        keyGenerateAlgorithm.init();
         setLastMilliseconds(keyGenerateAlgorithm, timeService.getCurrentMillis());
         setSequence(keyGenerateAlgorithm, (1 << DEFAULT_SEQUENCE_BITS) - 1);
         List<Comparable<?>> expected = Arrays.asList(4194304L, 4194305L, 4194306L, 8388608L, 8388609L, 8388610L, 12582913L, 12582914L, 12582915L, 16777216L);
@@ -175,6 +181,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
         Properties props = new Properties();
         props.setProperty("worker.id", String.valueOf(-1L));
         keyGenerateAlgorithm.setProps(props);
+        keyGenerateAlgorithm.init();
         keyGenerateAlgorithm.generateKey();
     }
     
@@ -184,6 +191,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
         Properties props = new Properties();
         props.setProperty("max.vibration.offset", String.valueOf(-1));
         keyGenerateAlgorithm.setProps(props);
+        keyGenerateAlgorithm.init();
         keyGenerateAlgorithm.generateKey();
     }
     
@@ -193,6 +201,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
         Properties props = new Properties();
         props.setProperty("worker.id", String.valueOf(Long.MIN_VALUE));
         keyGenerateAlgorithm.setProps(props);
+        keyGenerateAlgorithm.init();
         keyGenerateAlgorithm.generateKey();
     }
     
@@ -202,6 +211,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
         Properties props = new Properties();
         props.setProperty("max.vibration.offset", String.valueOf(4096));
         keyGenerateAlgorithm.setProps(props);
+        keyGenerateAlgorithm.init();
         keyGenerateAlgorithm.generateKey();
     }
     
@@ -212,6 +222,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
         Properties props = new Properties();
         props.setProperty("worker.id", String.valueOf(1L));
         keyGenerateAlgorithm.setProps(props);
+        keyGenerateAlgorithm.init();
         Field field = keyGenerateAlgorithm.getClass().getDeclaredField("props");
         field.setAccessible(true);
         assertThat(((Properties) field.get(keyGenerateAlgorithm)).get("worker.id"), is("1"));
@@ -224,6 +235,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
         Properties props = new Properties();
         props.setProperty("max.tolerate.time.difference.milliseconds", String.valueOf(1));
         keyGenerateAlgorithm.setProps(props);
+        keyGenerateAlgorithm.init();
         Field field = keyGenerateAlgorithm.getClass().getDeclaredField("props");
         field.setAccessible(true);
         assertThat(((Properties) field.get(keyGenerateAlgorithm)).get("max.tolerate.time.difference.milliseconds"), is("1"));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/UUIDKeyGenerateAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/UUIDKeyGenerateAlgorithmTest.java
similarity index 95%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/UUIDKeyGenerateAlgorithmTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/UUIDKeyGenerateAlgorithmTest.java
index cbbde10..7f13f6e 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/UUIDKeyGenerateAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/UUIDKeyGenerateAlgorithmTest.java
@@ -15,9 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.keygen;
+package org.apache.shardingsphere.sharding.algorithm.keygen;
 
 import org.junit.Test;
+
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/fixture/FixedTimeService.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/fixture/FixedTimeService.java
similarity index 84%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/fixture/FixedTimeService.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/fixture/FixedTimeService.java
index 1fe1373..0546c9c 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/fixture/FixedTimeService.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/fixture/FixedTimeService.java
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.keygen.fixture;
+package org.apache.shardingsphere.sharding.algorithm.keygen.fixture;
 
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.sharding.strategy.algorithm.keygen.SnowflakeKeyGenerateAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.algorithm.keygen.TimeService;
+import org.apache.shardingsphere.sharding.algorithm.keygen.SnowflakeKeyGenerateAlgorithm;
+import org.apache.shardingsphere.sharding.algorithm.keygen.TimeService;
 
 import java.util.concurrent.atomic.AtomicInteger;
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/fixture/IncrementKeyGenerateAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/fixture/IncrementKeyGenerateAlgorithm.java
similarity index 92%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/fixture/IncrementKeyGenerateAlgorithm.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/fixture/IncrementKeyGenerateAlgorithm.java
index 96c9fc0..f0fd927 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/keygen/fixture/IncrementKeyGenerateAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/fixture/IncrementKeyGenerateAlgorithm.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.keygen.fixture;
+package org.apache.shardingsphere.sharding.algorithm.keygen.fixture;
 
 import lombok.Getter;
 import lombok.Setter;
@@ -33,6 +33,10 @@ public final class IncrementKeyGenerateAlgorithm implements KeyGenerateAlgorithm
     private Properties props = new Properties();
     
     @Override
+    public void init() {
+    }
+    
+    @Override
     public Comparable<?> generateKey() {
         return count.incrementAndGet();
     }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/FixedIntervalShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/FixedIntervalShardingAlgorithmTest.java
similarity index 93%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/FixedIntervalShardingAlgorithmTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/FixedIntervalShardingAlgorithmTest.java
index 2cb6ae7..cd555f2 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/FixedIntervalShardingAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/FixedIntervalShardingAlgorithmTest.java
@@ -15,15 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding;
+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.algorithm.sharding.datetime.FixedIntervalShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.route.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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.junit.Before;
 import org.junit.Test;
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/MutableIntervalShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/MutableIntervalShardingAlgorithmTest.java
similarity index 82%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/MutableIntervalShardingAlgorithmTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/MutableIntervalShardingAlgorithmTest.java
index ba7f653..8ae939c 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/MutableIntervalShardingAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/MutableIntervalShardingAlgorithmTest.java
@@ -15,23 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding;
+package org.apache.shardingsphere.sharding.algorithm.sharding.datetime;
 
-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.algorithm.sharding.datetime.MutableIntervalShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.route.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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.junit.Before;
 import org.junit.Test;
 
 import java.time.LocalDateTime;
 import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Properties;
 
@@ -56,10 +56,43 @@ public final class MutableIntervalShardingAlgorithmTest {
         initShardStrategyByQuarter();
     }
     
+    private void initShardStrategyByQuarter() {
+        MutableIntervalShardingAlgorithm shardingAlgorithm = new MutableIntervalShardingAlgorithm();
+        shardingAlgorithm.getProps().setProperty("datetime.format", "yyyy-MM-dd HH:mm:ss");
+        shardingAlgorithm.getProps().setProperty("table.suffix.format", "yyyyQQ");
+        shardingAlgorithm.getProps().setProperty("datetime.lower", "2016-01-01 00:00:00.000");
+        shardingAlgorithm.getProps().setProperty("datetime.upper", "2021-12-31 00:00:00.000");
+        shardingAlgorithm.getProps().setProperty("datetime.step.unit", "Months");
+        shardingAlgorithm.getProps().setProperty("datetime.step.amount", "3");
+        shardingAlgorithm.init();
+        this.shardingStrategyByQuarter = new StandardShardingStrategy("create_time", shardingAlgorithm);
+        for (int i = 2016; i <= 2020; i++) {
+            for (int j = 1; j <= 4; j++) {
+                availableTablesForQuarterStrategy.add(String.format("t_order_%04d%02d", i, j));
+            }
+        }
+    }
+    
+    private void initShardStrategyByMonth() {
+        MutableIntervalShardingAlgorithm shardingAlgorithm = new MutableIntervalShardingAlgorithm();
+        shardingAlgorithm.getProps().setProperty("datetime.format", "yyyy-MM-dd HH:mm:ss");
+        shardingAlgorithm.getProps().setProperty("table.suffix.format", "yyyyMM");
+        shardingAlgorithm.getProps().setProperty("datetime.lower", "2016-01-01 00:00:00.000");
+        shardingAlgorithm.getProps().setProperty("datetime.upper", "2021-12-31 00:00:00.000");
+        shardingAlgorithm.getProps().setProperty("datetime.step.unit", "Months");
+        shardingAlgorithm.getProps().setProperty("datetime.step.amount", "1");
+        shardingAlgorithm.init();
+        this.shardingStrategyByMonth = new StandardShardingStrategy("create_time", shardingAlgorithm);
+        for (int i = 2016; i <= 2020; i++) {
+            for (int j = 1; j <= 12; j++) {
+                availableTablesForMonthStrategy.add(String.format("t_order_%04d%02d", i, j));
+            }
+        }
+    }
+    
     @Test
     public void assertPreciseDoShardingByQuarter() {
-        List<RouteValue> shardingValues = Lists.newArrayList(new ListRouteValue<>("create_time", "t_order",
-                Lists.newArrayList("2020-01-01 00:00:01", "2020-01-01 00:00:02", "2020-04-15 10:59:08")));
+        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"));
@@ -69,15 +102,14 @@ public final class MutableIntervalShardingAlgorithmTest {
     @Test
     public void assertRangeDoShardingByQuarter() {
         Range<String> rangeValue = Range.closed("2019-10-15 10:59:08", "2020-04-08 10:59:08");
-        List<RouteValue> shardingValues = Lists.newArrayList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
+        List<RouteValue> shardingValues = Collections.singletonList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
         Collection<String> actual = shardingStrategyByQuarter.doSharding(availableTablesForQuarterStrategy, shardingValues, new ConfigurationProperties(new Properties()));
         assertThat(actual.size(), is(3));
     }
     
     @Test
     public void assertPreciseDoShardingByMonth() {
-        List<RouteValue> shardingValues = Lists.newArrayList(new ListRouteValue<>("create_time", "t_order",
-                Lists.newArrayList("2020-01-01 00:00:01", "2020-01-01 00:00:02", "2020-04-15 10:59:08")));
+        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"));
@@ -87,7 +119,7 @@ public final class MutableIntervalShardingAlgorithmTest {
     @Test
     public void assertRangeDoShardingByMonth() {
         Range<String> rangeValue = Range.closed("2019-10-15 10:59:08", "2020-04-08 10:59:08");
-        List<RouteValue> shardingValues = Lists.newArrayList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
+        List<RouteValue> shardingValues = Collections.singletonList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
         Collection<String> actual = shardingStrategyByMonth.doSharding(availableTablesForMonthStrategy, shardingValues, new ConfigurationProperties(new Properties()));
         assertThat(actual.size(), is(7));
     }
@@ -95,7 +127,7 @@ public final class MutableIntervalShardingAlgorithmTest {
     @Test
     public void assertLowerHalfRangeDoSharding() {
         Range<String> rangeValue = Range.atLeast("2018-10-15 10:59:08");
-        List<RouteValue> shardingValues = Lists.newArrayList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
+        List<RouteValue> shardingValues = Collections.singletonList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
         Collection<String> actual = shardingStrategyByQuarter.doSharding(availableTablesForQuarterStrategy, shardingValues, new ConfigurationProperties(new Properties()));
         assertThat(actual.size(), is(9));
     }
@@ -103,7 +135,7 @@ public final class MutableIntervalShardingAlgorithmTest {
     @Test
     public void assertUpperHalfRangeDoSharding() {
         Range<String> rangeValue = Range.atMost("2019-09-01 00:00:00");
-        List<RouteValue> shardingValues = Lists.newArrayList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
+        List<RouteValue> shardingValues = Collections.singletonList(new RangeRouteValue<>("create_time", "t_order", rangeValue));
         Collection<String> actual = shardingStrategyByQuarter.doSharding(availableTablesForQuarterStrategy, shardingValues, new ConfigurationProperties(new Properties()));
         assertThat(actual.size(), is(15));
     }
@@ -120,38 +152,4 @@ public final class MutableIntervalShardingAlgorithmTest {
         assertEquals("202004", tableNameShardedByQuarter);
         assertEquals("202010", tableNameShardedByMonth);
     }
-    
-    private void initShardStrategyByQuarter() {
-        MutableIntervalShardingAlgorithm shardingAlgorithm = new MutableIntervalShardingAlgorithm();
-        shardingAlgorithm.getProps().setProperty("datetime.format", "yyyy-MM-dd HH:mm:ss");
-        shardingAlgorithm.getProps().setProperty("table.suffix.format", "yyyyQQ");
-        shardingAlgorithm.getProps().setProperty("datetime.lower", "2016-01-01 00:00:00.000");
-        shardingAlgorithm.getProps().setProperty("datetime.upper", "2021-12-31 00:00:00.000");
-        shardingAlgorithm.getProps().setProperty("datetime.step.unit", "Months");
-        shardingAlgorithm.getProps().setProperty("datetime.step.amount", "3");
-        shardingAlgorithm.init();
-        this.shardingStrategyByQuarter = new StandardShardingStrategy("create_time", shardingAlgorithm);
-        for (int i = 2016; i <= 2020; i++) {
-            for (int j = 1; j <= 4; j++) {
-                availableTablesForQuarterStrategy.add(String.format("t_order_%04d%02d", i, j));
-            }
-        }
-    }
-    
-    private void initShardStrategyByMonth() {
-        MutableIntervalShardingAlgorithm shardingAlgorithm = new MutableIntervalShardingAlgorithm();
-        shardingAlgorithm.getProps().setProperty("datetime.format", "yyyy-MM-dd HH:mm:ss");
-        shardingAlgorithm.getProps().setProperty("table.suffix.format", "yyyyMM");
-        shardingAlgorithm.getProps().setProperty("datetime.lower", "2016-01-01 00:00:00.000");
-        shardingAlgorithm.getProps().setProperty("datetime.upper", "2021-12-31 00:00:00.000");
-        shardingAlgorithm.getProps().setProperty("datetime.step.unit", "Months");
-        shardingAlgorithm.getProps().setProperty("datetime.step.amount", "1");
-        shardingAlgorithm.init();
-        this.shardingStrategyByMonth = new StandardShardingStrategy("create_time", shardingAlgorithm);
-        for (int i = 2016; i <= 2020; i++) {
-            for (int j = 1; j <= 12; j++) {
-                availableTablesForMonthStrategy.add(String.format("t_order_%04d%02d", i, j));
-            }
-        }
-    }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/inline/InlineExpressionParserTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineExpressionParserTest.java
similarity index 98%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/inline/InlineExpressionParserTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineExpressionParserTest.java
index 49d1114..38210d4 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/inline/InlineExpressionParserTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineExpressionParserTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline;
+package org.apache.shardingsphere.sharding.algorithm.sharding.inline;
 
 import org.junit.Test;
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/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
similarity index 92%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/inline/InlineShardingAlgorithmTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/inline/InlineShardingAlgorithmTest.java
index dcbdf6f..7fa388a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/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
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline;
+package org.apache.shardingsphere.sharding.algorithm.sharding.inline;
 
 import com.google.common.collect.Lists;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.sharding.strategy.route.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
+import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 import org.junit.Before;
 import org.junit.Test;
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/HashModShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/HashModShardingAlgorithmTest.java
similarity index 85%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/HashModShardingAlgorithmTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/HashModShardingAlgorithmTest.java
index 6af6fb6..ab1f872 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/HashModShardingAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/HashModShardingAlgorithmTest.java
@@ -15,16 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding;
+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.algorithm.sharding.mod.HashModShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.route.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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.junit.Before;
 import org.junit.Test;
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/ModShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/ModShardingAlgorithmTest.java
similarity index 87%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/ModShardingAlgorithmTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/ModShardingAlgorithmTest.java
index fa9c9d0..6270fa7 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/ModShardingAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/mod/ModShardingAlgorithmTest.java
@@ -15,16 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding;
+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.algorithm.sharding.mod.ModShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.route.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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.junit.Before;
 import org.junit.Test;
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/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
similarity index 89%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/range/BoundaryBasedRangeShardingAlgorithmTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/BoundaryBasedRangeShardingAlgorithmTest.java
index ca6cd0c..2b06759 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/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
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding.range;
+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.route.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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.junit.Before;
 import org.junit.Test;
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/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
similarity index 92%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/algorithm/sharding/range/VolumeBasedRangeShardingAlgorithmTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/range/VolumeBasedRangeShardingAlgorithmTest.java
index dc67e0f..d3ac5bb 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/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
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.algorithm.sharding.range;
+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.route.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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.junit.Before;
 import org.junit.Test;
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/log/ShardingConfigurationLoggerTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/log/ShardingConfigurationLoggerTest.java
index ee689ed..270f292 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/log/ShardingConfigurationLoggerTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/log/ShardingConfigurationLoggerTest.java
@@ -18,12 +18,10 @@
 package org.apache.shardingsphere.sharding.log;
 
 import lombok.SneakyThrows;
+import org.apache.shardingsphere.infra.log.ConfigurationLogger;
 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.sharding.NoneShardingStrategyConfiguration;
-import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.infra.log.ConfigurationLogger;
-import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapper;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -71,9 +69,8 @@ public final class ShardingConfigurationLoggerTest {
                 + "      logicTable: user\n"
                 + "      tableStrategy:\n"
                 + "        none: ''\n";
-//        assertLogInfo(yaml);
+        assertLogInfo(yaml);
         ConfigurationLogger.log(Collections.singletonList(getShardingRuleConfiguration()));
-        System.out.println(ShardingSphereServiceLoader.newServiceInstances(YamlRuleConfigurationSwapper.class).size());
     }
     
     private ShardingRuleConfiguration getShardingRuleConfiguration() {
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 3d1889e..e2a8401 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
@@ -25,12 +25,12 @@ import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfi
 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.strategy.algorithm.keygen.SnowflakeKeyGenerateAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.algorithm.keygen.fixture.IncrementKeyGenerateAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline.InlineShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.route.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.none.NoneShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.standard.StandardShardingStrategy;
+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;
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 e422993..fb3f42b 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
@@ -27,9 +27,9 @@ import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerate
 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.strategy.algorithm.sharding.mod.ModShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline.InlineShardingAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.route.none.NoneShardingStrategy;
+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;
 
@@ -92,6 +92,7 @@ public final class TableRuleTest {
         tableRuleConfig.setShardingStrategy(new StandardShardingStrategyConfiguration("col_1", "MOD"));
         ModShardingAlgorithm shardingAlgorithm = new ModShardingAlgorithm();
         shardingAlgorithm.getProps().setProperty("sharding.count", "4");
+        shardingAlgorithm.init();
         TableRule actual = new TableRule(tableRuleConfig, Arrays.asList("ds0", "ds1", "ds2"), shardingAlgorithm, null);
         assertThat(actual.getLogicTable(), is("logic_table"));
         assertThat(actual.getActualDataNodes().size(), is(4));
@@ -110,6 +111,7 @@ public final class TableRuleTest {
         tableRuleConfig.setShardingStrategy(new StandardShardingStrategyConfiguration("col_1", "MOD"));
         ModShardingAlgorithm shardingAlgorithm = new ModShardingAlgorithm();
         shardingAlgorithm.getProps().setProperty("sharding.count", "4");
+        shardingAlgorithm.init();
         TableRule actual = new TableRule(tableRuleConfig, Arrays.asList("ds0", "ds1", "ds2"), shardingAlgorithm, null);
         assertThat(actual.getLogicTable(), is("logic_table"));
         assertThat(actual.getActualDataNodes().size(), is(4));
@@ -165,6 +167,7 @@ public final class TableRuleTest {
         shardingTableRuleConfiguration.setTableShardingStrategy(new StandardShardingStrategyConfiguration("shardingColumn", "INLINE"));
         InlineShardingAlgorithm inlineShardingAlgorithm = new InlineShardingAlgorithm();
         inlineShardingAlgorithm.getProps().setProperty("algorithm.expression", "xxx");
+        inlineShardingAlgorithm.init();
         new TableRule(shardingTableRuleConfiguration, Arrays.asList("ds0", "ds1"), null, inlineShardingAlgorithm, null);
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/route/ShardingStrategyTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategyTest.java
similarity index 77%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/route/ShardingStrategyTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategyTest.java
index af2037a..926ccc7 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/route/ShardingStrategyTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/ShardingStrategyTest.java
@@ -15,20 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route;
+package org.apache.shardingsphere.sharding.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.route.complex.ComplexShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.fixture.ComplexKeysShardingAlgorithmFixture;
-import org.apache.shardingsphere.sharding.strategy.route.fixture.StandardShardingAlgorithmFixture;
-import org.apache.shardingsphere.sharding.strategy.route.none.NoneShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.standard.StandardShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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.junit.Test;
 
 import java.util.Collection;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/route/fixture/ComplexKeysShardingAlgorithmFixture.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/ComplexKeysShardingAlgorithmFixture.java
similarity index 96%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/route/fixture/ComplexKeysShardingAlgorithmFixture.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/ComplexKeysShardingAlgorithmFixture.java
index cc30eca..c537aaf 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/route/fixture/ComplexKeysShardingAlgorithmFixture.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/ComplexKeysShardingAlgorithmFixture.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route.fixture;
+package org.apache.shardingsphere.sharding.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/route/fixture/HintShardingAlgorithmFixture.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/HintShardingAlgorithmFixture.java
similarity index 96%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/route/fixture/HintShardingAlgorithmFixture.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/HintShardingAlgorithmFixture.java
index 72aae38..c48edcc 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/route/fixture/HintShardingAlgorithmFixture.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/HintShardingAlgorithmFixture.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route.fixture;
+package org.apache.shardingsphere.sharding.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/route/fixture/StandardShardingAlgorithmFixture.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/StandardShardingAlgorithmFixture.java
similarity index 97%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/route/fixture/StandardShardingAlgorithmFixture.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/StandardShardingAlgorithmFixture.java
index d8e33d2..2f7d834 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/route/fixture/StandardShardingAlgorithmFixture.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/fixture/StandardShardingAlgorithmFixture.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route.fixture;
+package org.apache.shardingsphere.sharding.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/route/value/ListRouteValueTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/value/ListRouteValueTest.java
similarity index 95%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/route/value/ListRouteValueTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/value/ListRouteValueTest.java
index f793755..2717ae8 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/route/value/ListRouteValueTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/strategy/value/ListRouteValueTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.strategy.route.value;
+package org.apache.shardingsphere.sharding.strategy.value;
 
 import org.junit.Test;
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm
index 2dafb64..8e6368e 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm
@@ -15,6 +15,6 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.sharding.strategy.algorithm.keygen.SnowflakeKeyGenerateAlgorithm
-org.apache.shardingsphere.sharding.strategy.algorithm.keygen.UUIDKeyGenerateAlgorithm
-org.apache.shardingsphere.sharding.strategy.algorithm.keygen.fixture.IncrementKeyGenerateAlgorithm
+org.apache.shardingsphere.sharding.algorithm.keygen.SnowflakeKeyGenerateAlgorithm
+org.apache.shardingsphere.sharding.algorithm.keygen.UUIDKeyGenerateAlgorithm
+org.apache.shardingsphere.sharding.algorithm.keygen.fixture.IncrementKeyGenerateAlgorithm
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 542a25a..c1e4bc7 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,8 @@
 #
  
 org.apache.shardingsphere.sharding.fixture.ComplexKeysShardingAlgorithmFixture
-org.apache.shardingsphere.sharding.strategy.route.fixture.ComplexKeysShardingAlgorithmFixture
+org.apache.shardingsphere.sharding.strategy.fixture.ComplexKeysShardingAlgorithmFixture
 org.apache.shardingsphere.sharding.fixture.HintShardingAlgorithmFixture
-org.apache.shardingsphere.sharding.strategy.route.fixture.HintShardingAlgorithmFixture
+org.apache.shardingsphere.sharding.strategy.fixture.HintShardingAlgorithmFixture
 org.apache.shardingsphere.sharding.fixture.StandardShardingAlgorithmFixture
-org.apache.shardingsphere.sharding.strategy.route.fixture.StandardShardingAlgorithmFixture
+org.apache.shardingsphere.sharding.strategy.fixture.StandardShardingAlgorithmFixture
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/fixture/KeyGenerateAlgorithmFixture.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/fixture/KeyGenerateAlgorithmFixture.java
index 245afb8..35187a4 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/fixture/KeyGenerateAlgorithmFixture.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/fixture/KeyGenerateAlgorithmFixture.java
@@ -30,6 +30,10 @@ public final class KeyGenerateAlgorithmFixture implements KeyGenerateAlgorithm {
     private Properties props = new Properties();
     
     @Override
+    public void init() {
+    }
+    
+    @Override
     public Comparable<?> generateKey() {
         return 1;
     }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingRouteDecorator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingRouteDecorator.java
index 16ac868..2749cc7 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingRouteDecorator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingRouteDecorator.java
@@ -35,9 +35,9 @@ 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.route.hint.HintShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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.sql.parser.binder.metadata.schema.SchemaMetaData;
 import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.binder.statement.dml.InsertStatementContext;
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/AlwaysFalseRouteValue.java
index 047b7f3..63de889 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/AlwaysFalseRouteValue.java
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.sharding.route.engine.condition;
 
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 
 /**
  * Always false route value.
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 0557bcd..d2a8524 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.route.value.RouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 
 import java.util.LinkedList;
 import java.util.List;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/InsertClauseShardingConditionEngine.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/InsertClauseShardingConditionEngine.java
index 281c4f0..579747d 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/InsertClauseShardingConditionEngine.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/InsertClauseShardingConditionEngine.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.sharding.route.engine.condition.engine;
 import com.google.common.base.Preconditions;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
 import org.apache.shardingsphere.sharding.route.engine.condition.ExpressionConditionUtils;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingCondition;
 import org.apache.shardingsphere.sharding.route.spi.SPITimeService;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/WhereClauseShardingConditionEngine.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/WhereClauseShardingConditionEngine.java
index 3a19bf5..5f21a1b 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/WhereClauseShardingConditionEngine.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/WhereClauseShardingConditionEngine.java
@@ -26,9 +26,9 @@ 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.generator.ConditionValueGeneratorFactory;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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.sql.parser.binder.metadata.schema.SchemaMetaData;
 import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.binder.type.WhereAvailable;
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 57c930a..fc052f4 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
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.sharding.route.engine.condition.generator;
 
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 import org.apache.shardingsphere.sharding.route.engine.condition.Column;
 import org.apache.shardingsphere.sql.parser.sql.segment.dml.predicate.value.PredicateRightValue;
 
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 cf3b6b3..7a9603e 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
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.sharding.route.engine.condition.generator;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 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;
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 21321e6..424f6a1 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
@@ -18,8 +18,8 @@
 package org.apache.shardingsphere.sharding.route.engine.condition.generator.impl;
 
 import com.google.common.collect.Range;
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 import org.apache.shardingsphere.sharding.route.engine.condition.Column;
 import org.apache.shardingsphere.sharding.route.engine.condition.ExpressionConditionUtils;
 import org.apache.shardingsphere.sharding.route.engine.condition.generator.ConditionValue;
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 af1fd8d..b963556 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
@@ -19,9 +19,9 @@ package org.apache.shardingsphere.sharding.route.engine.condition.generator.impl
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Range;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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.Column;
 import org.apache.shardingsphere.sharding.route.engine.condition.ExpressionConditionUtils;
 import org.apache.shardingsphere.sharding.route.engine.condition.generator.ConditionValue;
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 5438cfb..7b568ce 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
@@ -17,8 +17,8 @@
 
 package org.apache.shardingsphere.sharding.route.engine.condition.generator.impl;
 
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 import org.apache.shardingsphere.sharding.route.engine.condition.Column;
 import org.apache.shardingsphere.sharding.route.engine.condition.ExpressionConditionUtils;
 import org.apache.shardingsphere.sharding.route.engine.condition.generator.ConditionValue;
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 2772fc3..fbf7bb9 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,10 +25,10 @@ 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.route.ShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.hint.HintShardingStrategy;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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.sql.parser.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.binder.statement.dml.DeleteStatementContext;
 import org.apache.shardingsphere.sql.parser.binder.statement.dml.InsertStatementContext;
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 dd4ecc3..3f4cf30 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
@@ -17,8 +17,8 @@
 
 package org.apache.shardingsphere.sharding.route.engine.condition.generator.impl;
 
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RangeRouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 import org.apache.shardingsphere.sharding.route.engine.condition.Column;
 import org.apache.shardingsphere.sql.parser.sql.segment.dml.expr.ExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.segment.dml.expr.complex.CommonExpressionSegment;
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 2611d23..0b9baac 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
@@ -18,9 +18,9 @@
 package org.apache.shardingsphere.sharding.route.engine.condition.generator.impl;
 
 import com.google.common.collect.Range;
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RangeRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+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.Column;
 import org.apache.shardingsphere.sql.parser.sql.segment.dml.expr.complex.CommonExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.segment.dml.expr.simple.LiteralExpressionSegment;
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 e166171..aee3f49 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
@@ -17,8 +17,8 @@
 
 package org.apache.shardingsphere.sharding.route.engine.condition.generator.impl;
 
-import org.apache.shardingsphere.sharding.strategy.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 import org.apache.shardingsphere.sharding.route.engine.condition.Column;
 import org.apache.shardingsphere.sql.parser.sql.segment.dml.expr.complex.CommonExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.segment.dml.predicate.value.PredicateBracketValue;
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 d8e4845..8e7b1da 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.route.value.ListRouteValue;
-import org.apache.shardingsphere.sharding.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.ListRouteValue;
+import org.apache.shardingsphere.sharding.strategy.value.RouteValue;
 
 import java.util.ArrayList;
 import java.util.Arrays;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-spring/shardingsphere-sharding-spring-namespace/src/main/java/org/apache/shardingsphere/sharding/spring/namespace/parser/rule/ShardingRuleBeanDefinitionParser.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-spring/shardingsphere-sharding-spring-namespace/src/main/java/org/apache/shardingsphere/sharding/spring/namespace/parser/rule/ShardingRuleBeanDefinitionParser.java
index d9a698a..9a3c586 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-spring/shardingsphere-sharding-spring-namespace/src/main/java/org/apache/shardingsphere/sharding/spring/namespace/parser/rule/ShardingRuleBeanDefinitionParser.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-spring/shardingsphere-sharding-spring-namespace/src/main/java/org/apache/shardingsphere/sharding/spring/namespace/parser/rule/ShardingRuleBeanDefinitionParser.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfi
 import org.apache.shardingsphere.sharding.spring.namespace.factorybean.KeyGenerateAlgorithmFactoryBean;
 import org.apache.shardingsphere.sharding.spring.namespace.factorybean.ShardingAlgorithmFactoryBean;
 import org.apache.shardingsphere.sharding.spring.namespace.tag.rule.ShardingRuleBeanDefinitionTag;
-import org.apache.shardingsphere.sharding.strategy.algorithm.keygen.config.AlgorithmProvidedShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.algorithm.config.AlgorithmProvidedShardingRuleConfiguration;
 import org.apache.shardingsphere.spring.namespace.parser.ShardingSphereAlgorithmBeanRegistry;
 import org.springframework.beans.factory.config.BeanDefinition;
 import org.springframework.beans.factory.support.AbstractBeanDefinition;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/config/DataSourceConfiguration.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/config/DataSourceConfiguration.java
index 5e5751b..9b81619 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/config/DataSourceConfiguration.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/config/DataSourceConfiguration.java
@@ -158,7 +158,7 @@ public final class DataSourceConfiguration {
     public void addAlias(final String... alias) {
         Object value = null;
         for (String each : alias) {
-            if (null != props.get(each)) {
+            if (props.containsKey(each)) {
                 value = props.get(each);
             }
         }
diff --git a/shardingsphere-integration-test/shardingsphere-test-fixture/src/test/java/org/apache/shardingsphere/dbtest/fixture/ConstantKeyGenerateAlgorithm.java b/shardingsphere-integration-test/shardingsphere-test-fixture/src/test/java/org/apache/shardingsphere/dbtest/fixture/ConstantKeyGenerateAlgorithm.java
index 6bff0b4..221ade3 100644
--- a/shardingsphere-integration-test/shardingsphere-test-fixture/src/test/java/org/apache/shardingsphere/dbtest/fixture/ConstantKeyGenerateAlgorithm.java
+++ b/shardingsphere-integration-test/shardingsphere-test-fixture/src/test/java/org/apache/shardingsphere/dbtest/fixture/ConstantKeyGenerateAlgorithm.java
@@ -30,6 +30,10 @@ public final class ConstantKeyGenerateAlgorithm implements KeyGenerateAlgorithm
     private Properties props = new Properties();
     
     @Override
+    public void init() {
+    }
+    
+    @Override
     public Comparable<?> generateKey() {
         return 1;
     }
diff --git a/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/cases/dataset/DataSet.java b/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/cases/dataset/DataSet.java
index aa56e10..f923b7f 100644
--- a/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/cases/dataset/DataSet.java
+++ b/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/cases/dataset/DataSet.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.dbtest.cases.dataset;
 import lombok.Getter;
 import org.apache.shardingsphere.dbtest.cases.dataset.metadata.DataSetMetadata;
 import org.apache.shardingsphere.dbtest.cases.dataset.row.DataSetRow;
-import org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 
 import javax.xml.bind.annotation.XmlAttribute;
diff --git a/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/engine/BatchIT.java b/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/engine/BatchIT.java
index 3052e84..e7f11f6 100644
--- a/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/engine/BatchIT.java
+++ b/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/engine/BatchIT.java
@@ -27,7 +27,7 @@ import org.apache.shardingsphere.dbtest.cases.dataset.metadata.DataSetMetadata;
 import org.apache.shardingsphere.dbtest.cases.dataset.row.DataSetRow;
 import org.apache.shardingsphere.dbtest.env.EnvironmentPath;
 import org.apache.shardingsphere.dbtest.env.dataset.DataSetEnvironmentManager;
-import org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.junit.After;
diff --git a/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/engine/dml/BaseDMLIT.java b/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/engine/dml/BaseDMLIT.java
index b5e67df..b68465b 100644
--- a/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/engine/dml/BaseDMLIT.java
+++ b/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/engine/dml/BaseDMLIT.java
@@ -26,7 +26,7 @@ import org.apache.shardingsphere.dbtest.cases.dataset.row.DataSetRow;
 import org.apache.shardingsphere.dbtest.engine.SingleIT;
 import org.apache.shardingsphere.dbtest.env.EnvironmentPath;
 import org.apache.shardingsphere.dbtest.env.dataset.DataSetEnvironmentManager;
-import org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.junit.After;
diff --git a/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/env/dataset/DataSetEnvironmentManager.java b/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/env/dataset/DataSetEnvironmentManager.java
index 2853e43..fb39db7 100644
--- a/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/env/dataset/DataSetEnvironmentManager.java
+++ b/shardingsphere-integration-test/shardingsphere-test-suite/src/test/java/org/apache/shardingsphere/dbtest/env/dataset/DataSetEnvironmentManager.java
@@ -24,7 +24,7 @@ import org.apache.shardingsphere.dbtest.cases.dataset.DataSet;
 import org.apache.shardingsphere.dbtest.cases.dataset.metadata.DataSetColumn;
 import org.apache.shardingsphere.dbtest.cases.dataset.metadata.DataSetMetadata;
 import org.apache.shardingsphere.dbtest.cases.dataset.row.DataSetRow;
-import org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 import org.apache.shardingsphere.infra.datanode.DataNode;
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/fixture/IncrementKeyGenerateAlgorithm.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/fixture/IncrementKeyGenerateAlgorithm.java
index 0e6c8b7..d21cd29 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/fixture/IncrementKeyGenerateAlgorithm.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/fixture/IncrementKeyGenerateAlgorithm.java
@@ -33,6 +33,10 @@ public final class IncrementKeyGenerateAlgorithm implements KeyGenerateAlgorithm
     private Properties props = new Properties();
     
     @Override
+    public void init() {
+    }
+    
+    @Override
     public Comparable<?> generateKey() {
         return count.incrementAndGet();
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/fixture/ResetIncrementKeyGenerateAlgorithm.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/fixture/ResetIncrementKeyGenerateAlgorithm.java
index ddc14bc..1577da8 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/fixture/ResetIncrementKeyGenerateAlgorithm.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/fixture/ResetIncrementKeyGenerateAlgorithm.java
@@ -34,6 +34,10 @@ public final class ResetIncrementKeyGenerateAlgorithm implements KeyGenerateAlgo
     private Properties props = new Properties();
     
     @Override
+    public void init() {
+    }
+    
+    @Override
     public Comparable<?> generateKey() {
         return COUNT.incrementAndGet();
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/DataSourceAdapterTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/DataSourceAdapterTest.java
index c7729cf..6762557 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/DataSourceAdapterTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/DataSourceAdapterTest.java
@@ -83,7 +83,7 @@ public final class DataSourceAdapterTest extends AbstractShardingSphereDataSourc
     }
     
     @Test
-    public void assertGetConnectionWithUsername() throws SQLException {
+    public void assertGetConnectionWithUsername() {
         assertThat(getShardingSphereDataSource().getConnection("username", "password"), instanceOf(ShardingSphereConnection.class));
     }
 }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm
index 10cc0f4..1cd605b 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.sharding.strategy.algorithm.keygen.SnowflakeKeyGenerateAlgorithm
-org.apache.shardingsphere.sharding.strategy.algorithm.keygen.UUIDKeyGenerateAlgorithm
+org.apache.shardingsphere.sharding.algorithm.keygen.SnowflakeKeyGenerateAlgorithm
+org.apache.shardingsphere.sharding.algorithm.keygen.UUIDKeyGenerateAlgorithm
 org.apache.shardingsphere.driver.fixture.IncrementKeyGenerateAlgorithm
 org.apache.shardingsphere.driver.fixture.ResetIncrementKeyGenerateAlgorithm
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/java/org/apache/shardingsphere/driver/orchestration/api/yaml/fixture/DecrementKeyGenerateAlgorithm.java b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/java/org/apache/shardingsphere/driver/orchestration/api/yaml/fixture/DecrementKeyGenerateAlgorithm.java
index d209eb2..cb597ad 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/java/org/apache/shardingsphere/driver/orchestration/api/yaml/fixture/DecrementKeyGenerateAlgorithm.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/java/org/apache/shardingsphere/driver/orchestration/api/yaml/fixture/DecrementKeyGenerateAlgorithm.java
@@ -33,6 +33,10 @@ public final class DecrementKeyGenerateAlgorithm implements KeyGenerateAlgorithm
     private Properties props = new Properties();
     
     @Override
+    public void init() {
+    }
+    
+    @Override
     public Comparable<?> generateKey() {
         return sequence.decrementAndGet();
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/java/org/apache/shardingsphere/driver/orchestration/api/yaml/fixture/IncrementKeyGenerateAlgorithm.java b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/java/org/apache/shardingsphere/driver/orchestration/api/yaml/fixture/IncrementKeyGenerateAlgorithm.java
index e81ab46..0fff2db 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/java/org/apache/shardingsphere/driver/orchestration/api/yaml/fixture/IncrementKeyGenerateAlgorithm.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/java/org/apache/shardingsphere/driver/orchestration/api/yaml/fixture/IncrementKeyGenerateAlgorithm.java
@@ -33,6 +33,10 @@ public final class IncrementKeyGenerateAlgorithm implements KeyGenerateAlgorithm
     private Properties props = new Properties();
     
     @Override
+    public void init() {
+    }
+    
+    @Override
     public Comparable<?> generateKey() {
         return SEQUENCE.incrementAndGet();
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm
index 6709fdd..31006e6 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-orchestration/src/test/resources/META-INF/services/org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.sharding.strategy.algorithm.keygen.SnowflakeKeyGenerateAlgorithm
-org.apache.shardingsphere.sharding.strategy.algorithm.keygen.UUIDKeyGenerateAlgorithm
+org.apache.shardingsphere.sharding.algorithm.keygen.SnowflakeKeyGenerateAlgorithm
+org.apache.shardingsphere.sharding.algorithm.keygen.UUIDKeyGenerateAlgorithm
 org.apache.shardingsphere.driver.orchestration.api.yaml.fixture.IncrementKeyGenerateAlgorithm
 org.apache.shardingsphere.driver.orchestration.api.yaml.fixture.DecrementKeyGenerateAlgorithm
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/type/SpringBootShardingTest.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/type/SpringBootShardingTest.java
index ea82818..3b8deea 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/type/SpringBootShardingTest.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/type/SpringBootShardingTest.java
@@ -24,7 +24,7 @@ import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.kernel.context.SchemaContexts;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
-import org.apache.shardingsphere.sharding.strategy.route.standard.StandardShardingStrategy;
+import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.springframework.boot.autoconfigure.SpringBootApplication;
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/ShardingNamespaceTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/ShardingNamespaceTest.java
index 8031833..ab29c9a 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/ShardingNamespaceTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/ShardingNamespaceTest.java
@@ -31,7 +31,7 @@ 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.spi.KeyGenerateAlgorithm;
-import org.apache.shardingsphere.sharding.strategy.route.none.NoneShardingStrategy;
+import org.apache.shardingsphere.sharding.strategy.none.NoneShardingStrategy;
 import org.apache.shardingsphere.spring.fixture.IncrementKeyGenerateAlgorithm;
 import org.apache.shardingsphere.spring.transaction.ShardingTransactionTypeScanner;
 import org.junit.Test;
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/fixture/DecrementKeyGenerateAlgorithm.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/fixture/DecrementKeyGenerateAlgorithm.java
index 149ef02..37b8008 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/fixture/DecrementKeyGenerateAlgorithm.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/fixture/DecrementKeyGenerateAlgorithm.java
@@ -33,6 +33,10 @@ public final class DecrementKeyGenerateAlgorithm implements KeyGenerateAlgorithm
     private Properties props = new Properties();
     
     @Override
+    public void init() {
+    }
+    
+    @Override
     public Comparable<?> generateKey() {
         return sequence.decrementAndGet();
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/fixture/IncrementKeyGenerateAlgorithm.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/fixture/IncrementKeyGenerateAlgorithm.java
index c4dfcdf..aa70213 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/fixture/IncrementKeyGenerateAlgorithm.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/fixture/IncrementKeyGenerateAlgorithm.java
@@ -33,6 +33,10 @@ public final class IncrementKeyGenerateAlgorithm implements KeyGenerateAlgorithm
     private Properties props = new Properties();
     
     @Override
+    public void init() {
+    }
+    
+    @Override
     public Comparable<?> generateKey() {
         return sequence.incrementAndGet();
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/resources/META-INF/rdb/shardingNamespace.xml b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/resources/META-INF/rdb/shardingNamespace.xml
index d7445d5..57e83dd 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/resources/META-INF/rdb/shardingNamespace.xml
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/resources/META-INF/rdb/shardingNamespace.xml
@@ -79,7 +79,7 @@
     
     <encrypt:encrypt-algorithm id="aes_encryptor" type="AES">
         <props>
-            <prop key="appToken">business</prop>
+            <prop key="aes.key.value">123456</prop>
         </props>
     </encrypt:encrypt-algorithm>
     <encrypt:encrypt-algorithm id="md5_encryptor" type="MD5" />
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/orchestration/type/OrchestrationSpringBootRegistryShardingTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/orchestration/type/Orc [...]
index 9d62db1..34c6c7b 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/orchestration/type/OrchestrationSpringBootRegistryShardingTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/orchestration/type/OrchestrationSpringBootRegistryShardingTest.java
@@ -26,7 +26,7 @@ import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.kernel.context.SchemaContexts;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
-import org.apache.shardingsphere.sharding.strategy.route.standard.StandardShardingStrategy;
+import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
 import org.apache.shardingsphere.spring.boot.orchestration.registry.TestCenterRepository;
 import org.apache.shardingsphere.spring.boot.orchestration.util.EmbedTestingServer;
 import org.junit.BeforeClass;
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/orchestration/type/OrchestrationSpringBootShardingTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/orchestration/type/Orchestrati [...]
index 30b5344..8f8ad9b 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/orchestration/type/OrchestrationSpringBootShardingTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/orchestration/type/OrchestrationSpringBootShardingTest.java
@@ -26,7 +26,7 @@ import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.kernel.context.SchemaContexts;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
-import org.apache.shardingsphere.sharding.strategy.route.standard.StandardShardingStrategy;
+import org.apache.shardingsphere.sharding.strategy.standard.StandardShardingStrategy;
 import org.apache.shardingsphere.spring.boot.orchestration.util.EmbedTestingServer;
 import org.junit.BeforeClass;
 import org.junit.Test;
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/orchestration/fixture/DecrementKeyGenerateAlgorithm.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/orchestration/fixture/Decreme [...]
index 6a8d733..6fc8c97 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/orchestration/fixture/DecrementKeyGenerateAlgorithm.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/orchestration/fixture/DecrementKeyGenerateAlgorithm.java
@@ -33,6 +33,10 @@ public final class DecrementKeyGenerateAlgorithm implements KeyGenerateAlgorithm
     private Properties props = new Properties();
     
     @Override
+    public void init() {
+    }
+    
+    @Override
     public Comparable<?> generateKey() {
         return sequence.decrementAndGet();
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/orchestration/fixture/IncrementKeyGenerateAlgorithm.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/orchestration/fixture/Increme [...]
index c56fa01..6430e3c 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/orchestration/fixture/IncrementKeyGenerateAlgorithm.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-orchestration-spring/shardingsphere-jdbc-orchestration-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/orchestration/fixture/IncrementKeyGenerateAlgorithm.java
@@ -33,6 +33,10 @@ public final class IncrementKeyGenerateAlgorithm implements KeyGenerateAlgorithm
     private Properties props = new Properties();
     
     @Override
+    public void init() {
+    }
+    
+    @Override
     public Comparable<?> generateKey() {
         return sequence.incrementAndGet();
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/DataSourceMapSetter.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/DataSourceMapSetter.java
index 63dfafa..8166b5e 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/DataSourceMapSetter.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/DataSourceMapSetter.java
@@ -21,7 +21,7 @@ import com.google.common.base.Preconditions;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
-import org.apache.shardingsphere.sharding.strategy.algorithm.sharding.inline.InlineExpressionParser;
+import org.apache.shardingsphere.sharding.algorithm.sharding.inline.InlineExpressionParser;
 import org.apache.shardingsphere.spring.boot.datasource.prop.impl.DataSourcePropertiesSetterHolder;
 import org.apache.shardingsphere.spring.boot.util.DataSourceUtil;
 import org.apache.shardingsphere.spring.boot.util.PropertyUtil;