You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2020/12/07 05:00:41 UTC

[shardingsphere] branch master updated: Fixes #8330, add class based sharding algorithm SPI. (#8508)

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

zhangliang 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 48a7deb  Fixes #8330, add class based sharding algorithm SPI. (#8508)
48a7deb is described below

commit 48a7debe59e43470da1dfd7fb32903a9aa18c371
Author: Raigor <ra...@gmail.com>
AuthorDate: Mon Dec 7 13:00:24 2020 +0800

    Fixes #8330, add class based sharding algorithm SPI. (#8508)
    
    * add class based sharding algorithm
    
    * For #8330, add test cases.
    
    * Solve problems & add document.
---
 docs/document/content/dev-manual/sharding.cn.md    |   1 +
 docs/document/content/dev-manual/sharding.en.md    |   1 +
 .../built-in-algorithm/sharding.cn.md              |  14 ++
 .../built-in-algorithm/sharding.en.md              |  13 ++
 .../classbased/ClassBasedShardingAlgorithm.java    | 109 ++++++++++++++++
 .../ClassBasedShardingAlgorithmFactory.java        |  50 ++++++++
 .../ClassBasedShardingAlgorithmStrategyType.java   |  40 ++++++
 ...e.shardingsphere.sharding.spi.ShardingAlgorithm |   1 +
 .../ClassBasedShardingAlgorithmTest.java           | 142 +++++++++++++++++++++
 ...ssBasedComplexKeysShardingAlgorithmFixture.java |  40 ++++++
 .../ClassBasedHintShardingAlgorithmFixture.java    |  40 ++++++
 ...ClassBasedStandardShardingAlgorithmFixture.java |  51 ++++++++
 12 files changed, 502 insertions(+)

diff --git a/docs/document/content/dev-manual/sharding.cn.md b/docs/document/content/dev-manual/sharding.cn.md
index 204f778..74c3cd1 100644
--- a/docs/document/content/dev-manual/sharding.cn.md
+++ b/docs/document/content/dev-manual/sharding.cn.md
@@ -20,6 +20,7 @@ chapter = true
 | MutableIntervalShardingAlgorithm    | 基于可变时间范围的分片算法  |
 | VolumeBasedRangeShardingAlgorithm   | 基于分片容量的范围分片算法  |
 | BoundaryBasedRangeShardingAlgorithm | 基于分片边界的范围分片算法  |
+| ClassBasedShardingAlgorithm         | 基于自定义类的分片算法      |
 
 ## KeyGenerateAlgorithm
 
diff --git a/docs/document/content/dev-manual/sharding.en.md b/docs/document/content/dev-manual/sharding.en.md
index 9cef545..5e2bc8d 100644
--- a/docs/document/content/dev-manual/sharding.en.md
+++ b/docs/document/content/dev-manual/sharding.en.md
@@ -20,6 +20,7 @@ chapter = true
 | MutableIntervalShardingAlgorithm    | Mutable interval sharding algorithm     |
 | VolumeBasedRangeShardingAlgorithm   | Volume based range sharding algorithm   |
 | BoundaryBasedRangeShardingAlgorithm | Boundary based range sharding algorithm |
+| ClassBasedShardingAlgorithm         | Class based sharding algorithm          |
 
 ## KeyGenerateAlgorithm
 
diff --git a/docs/document/content/user-manual/shardingsphere-jdbc/configuration/built-in-algorithm/sharding.cn.md b/docs/document/content/user-manual/shardingsphere-jdbc/configuration/built-in-algorithm/sharding.cn.md
index a6cdb70..053a455 100644
--- a/docs/document/content/user-manual/shardingsphere-jdbc/configuration/built-in-algorithm/sharding.cn.md
+++ b/docs/document/content/user-manual/shardingsphere-jdbc/configuration/built-in-algorithm/sharding.cn.md
@@ -100,3 +100,17 @@ Apache ShardingSphere 暂无内置复合分片算法。
 ## Hint 分片算法
 
 Apache ShardingSphere 暂无内置 Hint 分片算法。
+
+## 自定义类分片算法
+
+通过配置分片策略类型和算法类名,实现自定义扩展。
+
+类型:CLASS_BASED
+
+可配置属性:
+
+| *属性名称*           | *数据类型* | *说明*                                              |
+| ------------------ | --------- | -------------------------------------------------- |
+| strategy           | String    | 分片策略类型,支持STANDARD、COMPLEX或HINT(不区分大小写) |
+| algorithmClassName | String    | 分片算法全限定名                                      |
+
diff --git a/docs/document/content/user-manual/shardingsphere-jdbc/configuration/built-in-algorithm/sharding.en.md b/docs/document/content/user-manual/shardingsphere-jdbc/configuration/built-in-algorithm/sharding.en.md
index 5bb22f4..9fb152f 100644
--- a/docs/document/content/user-manual/shardingsphere-jdbc/configuration/built-in-algorithm/sharding.en.md
+++ b/docs/document/content/user-manual/shardingsphere-jdbc/configuration/built-in-algorithm/sharding.en.md
@@ -101,3 +101,16 @@ There is no built-in complex sharding algorithm in Apache ShardingSphere.
 ## Hint Sharding Algorithm
 
 There is no built-in hint sharding algorithm in Apache ShardingSphere.
+
+## Class Based Sharding Algorithm
+
+Realize custom extension by configuring the sharding strategy type and algorithm class name.
+
+Type:CLASS_BASED
+
+Attributes:
+
+| *Name*           | *DataType* | *Description*                                              |
+| ------------------ | --------- | -------------------------------------------------- |
+| strategy           | String    | Sharding strategy type, support STANDARD, COMPLEX or HINT (case insensitive) |
+| algorithmClassName | String    | Fully qualified name of sharding algorithm                                   |
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/classbased/ClassBasedShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/classbased/ClassBasedShardingAlgorithm.java
new file mode 100644
index 0000000..f2bdeed
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/classbased/ClassBasedShardingAlgorithm.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.algorithm.sharding.classbased;
+
+import com.google.common.base.Preconditions;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.sharding.api.sharding.complex.ComplexKeysShardingAlgorithm;
+import org.apache.shardingsphere.sharding.api.sharding.complex.ComplexKeysShardingValue;
+import org.apache.shardingsphere.sharding.api.sharding.hint.HintShardingAlgorithm;
+import org.apache.shardingsphere.sharding.api.sharding.hint.HintShardingValue;
+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 java.util.Collection;
+import java.util.Properties;
+
+/**
+ * Class based sharding algorithm.
+ */
+public final class ClassBasedShardingAlgorithm implements StandardShardingAlgorithm<Comparable<?>>, ComplexKeysShardingAlgorithm<Comparable<?>>, HintShardingAlgorithm<Comparable<?>> {
+
+    private static final String STRATEGY_KEY = "strategy";
+
+    private static final String ALGORITHM_CLASS_NAME_KEY = "algorithmClassName";
+
+    private StandardShardingAlgorithm standardShardingAlgorithm;
+
+    private ComplexKeysShardingAlgorithm complexKeysShardingAlgorithm;
+
+    private HintShardingAlgorithm hintShardingAlgorithm;
+
+    @Getter
+    private ClassBasedShardingAlgorithmStrategyType strategy;
+
+    @Getter
+    private String algorithmClassName;
+
+    @Getter
+    @Setter
+    private Properties props = new Properties();
+
+    @Override
+    public void init() {
+        String strategyKey = props.getProperty(STRATEGY_KEY);
+        Preconditions.checkNotNull(strategyKey, "The props`%s` cannot be null when uses class based sharding strategy.", STRATEGY_KEY);
+        strategy = ClassBasedShardingAlgorithmStrategyType.valueOf(strategyKey.toUpperCase().trim());
+        algorithmClassName = props.getProperty(ALGORITHM_CLASS_NAME_KEY);
+        Preconditions.checkNotNull(algorithmClassName, "The props `%s` cannot be null when uses class based sharding strategy.", ALGORITHM_CLASS_NAME_KEY);
+        createAlgorithmInstance();
+    }
+
+    private void createAlgorithmInstance() {
+        switch (strategy) {
+            case STANDARD:
+                standardShardingAlgorithm = ClassBasedShardingAlgorithmFactory.newInstance(algorithmClassName, StandardShardingAlgorithm.class);
+                break;
+            case COMPLEX:
+                complexKeysShardingAlgorithm = ClassBasedShardingAlgorithmFactory.newInstance(algorithmClassName, ComplexKeysShardingAlgorithm.class);
+                break;
+            case HINT:
+                hintShardingAlgorithm = ClassBasedShardingAlgorithmFactory.newInstance(algorithmClassName, HintShardingAlgorithm.class);
+                break;
+            default:
+                break;
+        }
+    }
+
+    @Override
+    public String doSharding(final Collection<String> availableTargetNames, final PreciseShardingValue<Comparable<?>> shardingValue) {
+        return standardShardingAlgorithm.doSharding(availableTargetNames, shardingValue);
+    }
+
+    @Override
+    public Collection<String> doSharding(final Collection<String> availableTargetNames, final RangeShardingValue<Comparable<?>> shardingValue) {
+        return standardShardingAlgorithm.doSharding(availableTargetNames, shardingValue);
+    }
+
+    @Override
+    public Collection<String> doSharding(final Collection<String> availableTargetNames, final ComplexKeysShardingValue<Comparable<?>> shardingValue) {
+        return complexKeysShardingAlgorithm.doSharding(availableTargetNames, shardingValue);
+    }
+
+    @Override
+    public Collection<String> doSharding(final Collection<String> availableTargetNames, final HintShardingValue<Comparable<?>> shardingValue) {
+        return hintShardingAlgorithm.doSharding(availableTargetNames, shardingValue);
+    }
+
+    @Override
+    public String getType() {
+        return "CLASS_BASED";
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/classbased/ClassBasedShardingAlgorithmFactory.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/classbased/ClassBasedShardingAlgorithmFactory.java
new file mode 100644
index 0000000..d3a941c
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/classbased/ClassBasedShardingAlgorithmFactory.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.algorithm.sharding.classbased;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import lombok.SneakyThrows;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
+
+/**
+ * ShardingSphere class based algorithm factory.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class ClassBasedShardingAlgorithmFactory {
+
+    /**
+     * Create sharding algorithm.
+     *
+     * @param shardingAlgorithmClassName sharding algorithm class name
+     * @param superShardingAlgorithmClass sharding algorithm super class
+     * @param <T> class generic type
+     * @return sharding algorithm instance
+     */
+    @SneakyThrows
+    @SuppressWarnings("unchecked")
+    public static <T extends ShardingAlgorithm> T newInstance(final String shardingAlgorithmClassName, final Class<T> superShardingAlgorithmClass) {
+        Class<?> result = Class.forName(shardingAlgorithmClassName);
+        if (!superShardingAlgorithmClass.isAssignableFrom(result)) {
+            throw new ShardingSphereException("Class %s should be implement %s", shardingAlgorithmClassName, superShardingAlgorithmClass.getName());
+        }
+        return (T) result.newInstance();
+    }
+    
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/classbased/ClassBasedShardingAlgorithmStrategyType.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/classbased/ClassBasedShardingAlgorithmStrategyType.java
new file mode 100644
index 0000000..83517a3
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/classbased/ClassBasedShardingAlgorithmStrategyType.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.algorithm.sharding.classbased;
+
+/**
+ * Class based sharding strategy.
+ */
+public enum ClassBasedShardingAlgorithmStrategyType {
+
+    /**
+     * The sharding strategy is standard.
+     */
+    STANDARD,
+
+    /**
+     * The sharding strategy is complex.
+     */
+    COMPLEX,
+
+    /**
+     * The sharding strategy is hint.
+     */
+    HINT
+
+}
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 e9bdf6c..7b71c68 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
@@ -22,3 +22,4 @@ org.apache.shardingsphere.sharding.algorithm.sharding.range.VolumeBasedRangeShar
 org.apache.shardingsphere.sharding.algorithm.sharding.range.BoundaryBasedRangeShardingAlgorithm
 org.apache.shardingsphere.sharding.algorithm.sharding.datetime.AutoIntervalShardingAlgorithm
 org.apache.shardingsphere.sharding.algorithm.sharding.datetime.IntervalShardingAlgorithm
+org.apache.shardingsphere.sharding.algorithm.sharding.classbased.ClassBasedShardingAlgorithm
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/classbased/ClassBasedShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/classbased/ClassBasedShardingAlgorithmTest.java
new file mode 100644
index 0000000..6535d9a
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/classbased/ClassBasedShardingAlgorithmTest.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.algorithm.sharding.classbased;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.sharding.api.sharding.complex.ComplexKeysShardingValue;
+import org.apache.shardingsphere.sharding.api.sharding.hint.HintShardingValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.sharding.api.sharding.standard.RangeShardingValue;
+import org.apache.shardingsphere.sharding.fixture.ClassBasedComplexKeysShardingAlgorithmFixture;
+import org.apache.shardingsphere.sharding.fixture.ClassBasedHintShardingAlgorithmFixture;
+import org.apache.shardingsphere.sharding.fixture.ClassBasedStandardShardingAlgorithmFixture;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.List;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public final class ClassBasedShardingAlgorithmTest {
+
+    @Test
+    public void assertStandardStrategyInit() {
+        ClassBasedShardingAlgorithm shardingAlgorithm = getStandardShardingAlgorithm();
+        assertThat(shardingAlgorithm.getType(), is("CLASS_BASED"));
+        assertThat(shardingAlgorithm.getStrategy(), is(ClassBasedShardingAlgorithmStrategyType.STANDARD));
+        assertThat(shardingAlgorithm.getAlgorithmClassName(), is(ClassBasedStandardShardingAlgorithmFixture.class.getName()));
+    }
+
+    @Test(expected = NullPointerException.class)
+    public void assertInitWithNullStrategy() {
+        ClassBasedShardingAlgorithm shardingAlgorithm = new ClassBasedShardingAlgorithm();
+        shardingAlgorithm.getProps().setProperty("strategy", null);
+        shardingAlgorithm.init();
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void assertInitWithWrongStrategy() {
+        ClassBasedShardingAlgorithm shardingAlgorithm = new ClassBasedShardingAlgorithm();
+        shardingAlgorithm.getProps().setProperty("strategy", "wrong");
+        shardingAlgorithm.init();
+    }
+
+    @Test(expected = NullPointerException.class)
+    public void assertInitWithNullClass() {
+        ClassBasedShardingAlgorithm shardingAlgorithm = new ClassBasedShardingAlgorithm();
+        shardingAlgorithm.getProps().setProperty("strategy", "standard");
+        shardingAlgorithm.init();
+    }
+
+    @Test(expected = ClassNotFoundException.class)
+    public void assertInitWithUndefinedClass() {
+        ClassBasedShardingAlgorithm shardingAlgorithm = new ClassBasedShardingAlgorithm();
+        shardingAlgorithm.getProps().setProperty("strategy", "standard");
+        shardingAlgorithm.getProps().setProperty("algorithmClassName", "org.apache.shardingsphere.sharding.UndefinedClass");
+        shardingAlgorithm.init();
+    }
+
+    @Test(expected = ShardingSphereException.class)
+    public void assertInitWithMismatchStrategy() {
+        ClassBasedShardingAlgorithm shardingAlgorithm = new ClassBasedShardingAlgorithm();
+        shardingAlgorithm.getProps().setProperty("strategy", "standard");
+        shardingAlgorithm.getProps().setProperty("algorithmClassName", ClassBasedComplexKeysShardingAlgorithmFixture.class.getName());
+        shardingAlgorithm.init();
+    }
+
+    @Test
+    public void assertPreciseDoSharding() {
+        ClassBasedShardingAlgorithm shardingAlgorithm = getStandardShardingAlgorithm();
+        List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
+        assertThat(shardingAlgorithm.doSharding(availableTargetNames, new PreciseShardingValue<>("t_order", "order_id", 0)), is("t_order_0"));
+    }
+
+    @Test
+    public void assertRangeDoSharding() {
+        ClassBasedShardingAlgorithm shardingAlgorithm = getStandardShardingAlgorithm();
+        List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new RangeShardingValue<>("t_order", "order_id", Range.closed(2, 15)));
+        assertThat(actual.size(), is(4));
+    }
+
+    @Test
+    public void assertComplexKeysDoSharding() {
+        ClassBasedShardingAlgorithm shardingAlgorithm = getComplexKeysShardingAlgorithm();
+        assertThat(shardingAlgorithm.getStrategy(), is(ClassBasedShardingAlgorithmStrategyType.COMPLEX));
+        List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new ComplexKeysShardingValue<>("t_order", null, null));
+        assertThat(actual.size(), is(4));
+    }
+
+    @Test
+    public void assertHintDoSharding() {
+        ClassBasedShardingAlgorithm shardingAlgorithm = getHintShardingAlgorithm();
+        assertThat(shardingAlgorithm.getStrategy(), is(ClassBasedShardingAlgorithmStrategyType.HINT));
+        List<String> availableTargetNames = Lists.newArrayList("t_order_0", "t_order_1", "t_order_2", "t_order_3");
+        Collection<String> actual = shardingAlgorithm.doSharding(availableTargetNames, new HintShardingValue<>("t_order", "order_id", null));
+        assertThat(actual.size(), is(4));
+    }
+
+    private ClassBasedShardingAlgorithm getStandardShardingAlgorithm() {
+        ClassBasedShardingAlgorithm shardingAlgorithm = new ClassBasedShardingAlgorithm();
+        shardingAlgorithm.getProps().setProperty("strategy", "standard");
+        shardingAlgorithm.getProps().setProperty("algorithmClassName", ClassBasedStandardShardingAlgorithmFixture.class.getName());
+        shardingAlgorithm.init();
+        return shardingAlgorithm;
+    }
+
+    private ClassBasedShardingAlgorithm getComplexKeysShardingAlgorithm() {
+        ClassBasedShardingAlgorithm shardingAlgorithm = new ClassBasedShardingAlgorithm();
+        shardingAlgorithm.getProps().setProperty("strategy", "complex");
+        shardingAlgorithm.getProps().setProperty("algorithmClassName", ClassBasedComplexKeysShardingAlgorithmFixture.class.getName());
+        shardingAlgorithm.init();
+        return shardingAlgorithm;
+    }
+
+    private ClassBasedShardingAlgorithm getHintShardingAlgorithm() {
+        ClassBasedShardingAlgorithm shardingAlgorithm = new ClassBasedShardingAlgorithm();
+        shardingAlgorithm.getProps().setProperty("strategy", "hint");
+        shardingAlgorithm.getProps().setProperty("algorithmClassName", ClassBasedHintShardingAlgorithmFixture.class.getName());
+        shardingAlgorithm.init();
+        return shardingAlgorithm;
+    }
+
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/fixture/ClassBasedComplexKeysShardingAlgorithmFixture.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/fixture/ClassBasedComplexKeysShardingAlgorithmFixture.java
new file mode 100644
index 0000000..5dc271a
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/fixture/ClassBasedComplexKeysShardingAlgorithmFixture.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.fixture;
+
+import org.apache.shardingsphere.sharding.api.sharding.complex.ComplexKeysShardingAlgorithm;
+import org.apache.shardingsphere.sharding.api.sharding.complex.ComplexKeysShardingValue;
+
+import java.util.Collection;
+
+public final class ClassBasedComplexKeysShardingAlgorithmFixture implements ComplexKeysShardingAlgorithm<Integer> {
+
+    @Override
+    public void init() {
+    }
+
+    @Override
+    public Collection<String> doSharding(final Collection<String> availableTargetNames, final ComplexKeysShardingValue<Integer> shardingValue) {
+        return availableTargetNames;
+    }
+
+    @Override
+    public String getType() {
+        return null;
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/fixture/ClassBasedHintShardingAlgorithmFixture.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/fixture/ClassBasedHintShardingAlgorithmFixture.java
new file mode 100644
index 0000000..1cc24df
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/fixture/ClassBasedHintShardingAlgorithmFixture.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.fixture;
+
+import org.apache.shardingsphere.sharding.api.sharding.hint.HintShardingAlgorithm;
+import org.apache.shardingsphere.sharding.api.sharding.hint.HintShardingValue;
+
+import java.util.Collection;
+
+public final class ClassBasedHintShardingAlgorithmFixture implements HintShardingAlgorithm<Integer> {
+
+    @Override
+    public void init() {
+    }
+
+    @Override
+    public Collection<String> doSharding(final Collection<String> availableTargetNames, final HintShardingValue<Integer> shardingValue) {
+        return availableTargetNames;
+    }
+
+    @Override
+    public String getType() {
+        return null;
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/fixture/ClassBasedStandardShardingAlgorithmFixture.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/fixture/ClassBasedStandardShardingAlgorithmFixture.java
new file mode 100644
index 0000000..1542206
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/fixture/ClassBasedStandardShardingAlgorithmFixture.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.fixture;
+
+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 java.util.Collection;
+
+public final class ClassBasedStandardShardingAlgorithmFixture implements StandardShardingAlgorithm<Integer> {
+
+    @Override
+    public void init() {
+    }
+
+    @Override
+    public String doSharding(final Collection<String> availableTargetNames, final PreciseShardingValue<Integer> shardingValue) {
+        for (String each : availableTargetNames) {
+            if (each.endsWith(String.valueOf(shardingValue.getValue() % 4))) {
+                return each;
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Collection<String> doSharding(final Collection<String> availableTargetNames, final RangeShardingValue<Integer> shardingValue) {
+        return availableTargetNames;
+    }
+
+    @Override
+    public String getType() {
+        return null;
+    }
+}