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 2022/08/27 03:29:36 UTC

[shardingsphere] branch master updated: Enhance `CosIdIntervalShardingAlgorithm` to Support for more date types (#20571)

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 8d87854038c Enhance `CosIdIntervalShardingAlgorithm` to Support for more date types (#20571)
8d87854038c is described below

commit 8d87854038c141614662dc13b2253136aedb0a4c
Author: Ahoo Wang <ah...@qq.com>
AuthorDate: Sat Aug 27 11:29:28 2022 +0800

    Enhance `CosIdIntervalShardingAlgorithm` to Support for more date types (#20571)
    
    * add CosId keygen documentation
    
    * Update String ID Description
    
    * add CosId keygen English-documentation
    
    * enhance `CosIdIntervalShardingAlgorithm` to Support for more date types
---
 .../shardingsphere-sharding-cosid/pom.xml          |  2 +-
 .../interval/CosIdIntervalShardingAlgorithm.java   | 32 +++++++-
 .../keygen/CosIdKeyGenerateAlgorithmTest.java      |  4 +-
 .../CosIdSnowflakeKeyGenerateAlgorithmTest.java    | 54 +++++++++++++-
 .../InstantCosIdIntervalShardingAlgorithmTest.java | 73 ++++++++++++++++++
 ...ocalDateCosIdIntervalShardingAlgorithmTest.java | 74 +++++++++++++++++++
 ...DateTimeCosIdIntervalShardingAlgorithmTest.java | 73 ++++++++++++++++++
 .../SqlDateCosIdIntervalShardingAlgorithmTest.java | 73 ++++++++++++++++++
 ...imestampCosIdIntervalShardingAlgorithmTest.java | 73 ++++++++++++++++++
 ...earMonthCosIdIntervalShardingAlgorithmTest.java | 86 ++++++++++++++++++++++
 ...DateTimeCosIdIntervalShardingAlgorithmTest.java | 73 ++++++++++++++++++
 11 files changed, 609 insertions(+), 8 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/pom.xml b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/pom.xml
index 2426422492c..e5cfcca9907 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/pom.xml
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/pom.xml
@@ -28,7 +28,7 @@
     <name>${project.artifactId}</name>
     
     <properties>
-        <cosid.version>1.10.0</cosid.version>
+        <cosid.version>1.14.4</cosid.version>
     </properties>
     
     <dependencies>
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/main/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/CosIdIntervalShardingAlgorithm.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/main/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/CosIdIntervalShardingAlgorithm.java
index 828d1dcd83b..e33585e48b0 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/main/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/CosIdIntervalShardingAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/main/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/CosIdIntervalShardingAlgorithm.java
@@ -18,14 +18,22 @@
 package org.apache.shardingsphere.sharding.cosid.algorithm.sharding.interval;
 
 import com.google.common.base.Strings;
-import me.ahoo.cosid.util.LocalDateTimeConvert;
-import org.apache.shardingsphere.sharding.cosid.algorithm.CosIdAlgorithmConstants;
 
+import java.time.Instant;
+import java.time.LocalDate;
 import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.Year;
+import java.time.YearMonth;
+import java.time.ZonedDateTime;
 import java.time.format.DateTimeFormatter;
 import java.util.Date;
 import java.util.Properties;
 
+import me.ahoo.cosid.util.LocalDateTimeConvert;
+import org.apache.shardingsphere.sharding.cosid.algorithm.CosIdAlgorithmConstants;
+
 /**
  * Interval range sharding algorithm with CosId.
  */
@@ -56,14 +64,34 @@ public final class CosIdIntervalShardingAlgorithm extends AbstractCosIdIntervalS
         return DateTimeFormatter.ofPattern(props.getProperty(DATE_TIME_PATTERN_KEY, DEFAULT_DATE_TIME_PATTERN));
     }
     
+    @SuppressWarnings("checkstyle:CyclomaticComplexity")
     @Override
     protected LocalDateTime convertShardingValue(final Comparable<?> shardingValue) {
         if (shardingValue instanceof LocalDateTime) {
             return (LocalDateTime) shardingValue;
         }
+        if (shardingValue instanceof ZonedDateTime) {
+            return ((ZonedDateTime) shardingValue).toLocalDateTime();
+        }
+        if (shardingValue instanceof OffsetDateTime) {
+            return ((OffsetDateTime) shardingValue).toLocalDateTime();
+        }
+        if (shardingValue instanceof Instant) {
+            return LocalDateTimeConvert.fromInstant((Instant) shardingValue, getZoneId());
+        }
+        if (shardingValue instanceof LocalDate) {
+            return LocalDateTime.of((LocalDate) shardingValue, LocalTime.MIN);
+        }
         if (shardingValue instanceof Date) {
             return LocalDateTimeConvert.fromDate((Date) shardingValue, getZoneId());
         }
+        if (shardingValue instanceof YearMonth) {
+            YearMonth yearMonth = (YearMonth) shardingValue;
+            return LocalDateTime.of(yearMonth.getYear(), yearMonth.getMonthValue(), 1, 0, 0);
+        }
+        if (shardingValue instanceof Year) {
+            return LocalDateTime.of(((Year) shardingValue).getValue(), 1, 1, 0, 0);
+        }
         if (shardingValue instanceof Long) {
             return isSecondTs ? LocalDateTimeConvert.fromTimestampSecond((Long) shardingValue, getZoneId()) : LocalDateTimeConvert.fromTimestamp((Long) shardingValue, getZoneId());
         }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdKeyGenerateAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdKeyGenerateAlgorithmTest.java
index c1e0ba28303..1776a011131 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdKeyGenerateAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdKeyGenerateAlgorithmTest.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.sharding.cosid.algorithm.keygen;
 
 import me.ahoo.cosid.IdGenerator;
-import me.ahoo.cosid.StringIdGenerator;
+import me.ahoo.cosid.StringIdGeneratorDecorator;
 import me.ahoo.cosid.converter.PrefixIdConverter;
 import me.ahoo.cosid.converter.Radix62IdConverter;
 import me.ahoo.cosid.provider.DefaultIdGeneratorProvider;
@@ -77,7 +77,7 @@ public final class CosIdKeyGenerateAlgorithmTest {
     public void assertGenerateKeyAsString() {
         String idName = "test-cosid-as-string";
         String prefix = "test_";
-        IdGenerator stringIdGen = new StringIdGenerator(new MillisecondSnowflakeId(1, 0), new PrefixIdConverter(prefix, Radix62IdConverter.INSTANCE));
+        IdGenerator stringIdGen = new StringIdGeneratorDecorator(new MillisecondSnowflakeId(1, 0), new PrefixIdConverter(prefix, Radix62IdConverter.INSTANCE));
         DefaultIdGeneratorProvider.INSTANCE.set(idName, stringIdGen);
         KeyGenerateAlgorithm algorithm = KeyGenerateAlgorithmFactory.newInstance(new AlgorithmConfiguration("COSID", createAsStringProperties(idName)));
         Comparable<?> actual = algorithm.generateKey();
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java
index 351778af8ed..32901c2f804 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java
@@ -17,6 +17,9 @@
 
 package org.apache.shardingsphere.sharding.cosid.algorithm.keygen;
 
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.locks.LockSupport;
+
 import me.ahoo.cosid.converter.Radix62IdConverter;
 import me.ahoo.cosid.snowflake.MillisecondSnowflakeId;
 import me.ahoo.cosid.snowflake.MillisecondSnowflakeIdStateParser;
@@ -31,12 +34,14 @@ import org.apache.shardingsphere.infra.instance.metadata.InstanceMetaData;
 import org.apache.shardingsphere.infra.lock.LockContext;
 import org.apache.shardingsphere.sharding.cosid.algorithm.keygen.fixture.WorkerIdGeneratorFixture;
 import org.apache.shardingsphere.sharding.factory.KeyGenerateAlgorithmFactory;
+import org.hamcrest.MatcherAssert;
 import org.junit.Test;
 
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.closeTo;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.mock;
 
@@ -63,12 +68,55 @@ public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
         SnowflakeIdState firstActualState = snowflakeIdStateParser.parse(firstActualKey);
         SnowflakeIdState secondActualState = snowflakeIdStateParser.parse(secondActualKey);
         assertThat(firstActualState.getMachineId(), is(FIXTURE_WORKER_ID));
-        assertThat(firstActualState.getSequence(), is(0L));
+        assertThat(firstActualState.getSequence(), is(1L));
         assertThat(secondActualState.getMachineId(), is(FIXTURE_WORKER_ID));
-        long expectedSecondSequence = secondActualState.getTimestamp().isAfter(firstActualState.getTimestamp()) ? 0L : 1L;
+        long expectedSecondSequence = 2L;
         assertThat(secondActualState.getSequence(), is(expectedSecondSequence));
     }
     
+    @Test
+    public void assertGenerateKeyModUniformity() {
+        CosIdSnowflakeKeyGenerateAlgorithm algorithm = (CosIdSnowflakeKeyGenerateAlgorithm) KeyGenerateAlgorithmFactory.newInstance(
+                new AlgorithmConfiguration("COSID_SNOWFLAKE", new Properties()));
+        algorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(FIXTURE_WORKER_ID),
+                new ModeConfiguration("Standalone", null, false), mock(LockContext.class), eventBusContext));
+        int divisor = 4;
+        int total = 99999;
+        int avg = total / divisor;
+        double tolerance = avg * .001;
+        
+        int mod0Counter = 0;
+        int mod1Counter = 0;
+        int mod2Counter = 0;
+        int mod3Counter = 0;
+        for (int i = 0; i < total; i++) {
+            long id = (Long) algorithm.generateKey();
+            int mod = (int) (id % divisor);
+            switch (mod) {
+                case 0:
+                    mod0Counter++;
+                    break;
+                case 1:
+                    mod1Counter++;
+                    break;
+                case 2:
+                    mod2Counter++;
+                    break;
+                case 3:
+                    mod3Counter++;
+                    break;
+                default:
+                    throw new IllegalStateException("Unexpected value: " + mod);
+            }
+            int wait = ThreadLocalRandom.current().nextInt(10, 1000);
+            LockSupport.parkNanos(wait);
+        }
+        MatcherAssert.assertThat((double) mod0Counter, closeTo(avg, tolerance));
+        MatcherAssert.assertThat((double) mod1Counter, closeTo(avg, tolerance));
+        MatcherAssert.assertThat((double) mod2Counter, closeTo(avg, tolerance));
+        MatcherAssert.assertThat((double) mod3Counter, closeTo(avg, tolerance));
+    }
+    
     @Test
     public void assertGenerateKeyAsString() {
         Properties props = new Properties();
@@ -84,7 +132,7 @@ public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
         long actualLongKey = Radix62IdConverter.PAD_START.asLong(actualStringKey);
         SnowflakeIdState actualState = snowflakeIdStateParser.parse(actualLongKey);
         assertThat(actualState.getMachineId(), is(FIXTURE_WORKER_ID));
-        assertThat(actualState.getSequence(), is(0L));
+        assertThat(actualState.getSequence(), is(1L));
     }
     
     @Test(expected = NullPointerException.class)
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/InstantCosIdIntervalShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/InstantCosIdIntervalShardingAlgorithmTest.java
new file mode 100644
index 00000000000..19c3bc0e19c
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/InstantCosIdIntervalShardingAlgorithmTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.cosid.algorithm.sharding.interval;
+
+import com.google.common.collect.Range;
+import java.time.Instant;
+import java.util.Collection;
+import org.apache.shardingsphere.sharding.cosid.algorithm.sharding.interval.fixture.IntervalShardingAlgorithmDataFixture;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+public final class InstantCosIdIntervalShardingAlgorithmTest {
+    
+    static Iterable<Object[]> preciseArgsProviderAsInstant() {
+        return IntervalShardingAlgorithmDataFixture.preciseArgsProvider(ldt -> ldt.toInstant(IntervalShardingAlgorithmDataFixture.ZONE_OFFSET_SHANGHAI));
+    }
+    
+    static Iterable<Object[]> rangeArgsProviderAsInstant() {
+        return IntervalShardingAlgorithmDataFixture.rangeArgsProvider(ldt -> ldt.toInstant(IntervalShardingAlgorithmDataFixture.ZONE_OFFSET_SHANGHAI));
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class PreciseShardingValueDoShardingTest extends AbstractPreciseShardingValueParameterizedTest<Instant> {
+        
+        public PreciseShardingValueDoShardingTest(final Instant input, final String expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            return preciseArgsProviderAsInstant();
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class RangeShardingValueDoShardingTest extends AbstractRangeShardingValueParameterizedTest<Instant> {
+        
+        public RangeShardingValueDoShardingTest(final Range<Instant> input, final Collection<String> expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            return rangeArgsProviderAsInstant();
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/LocalDateCosIdIntervalShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/LocalDateCosIdIntervalShardingAlgorithmTest.java
new file mode 100644
index 00000000000..63adc443485
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/LocalDateCosIdIntervalShardingAlgorithmTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.cosid.algorithm.sharding.interval;
+
+import com.google.common.collect.Range;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.util.Collection;
+import org.apache.shardingsphere.sharding.cosid.algorithm.sharding.interval.fixture.IntervalShardingAlgorithmDataFixture;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+public final class LocalDateCosIdIntervalShardingAlgorithmTest {
+    
+    static Iterable<Object[]> preciseArgsProviderAsLocalDate() {
+        return IntervalShardingAlgorithmDataFixture.preciseArgsProvider(LocalDateTime::toLocalDate);
+    }
+    
+    static Iterable<Object[]> rangeArgsProviderAsLocalDate() {
+        return IntervalShardingAlgorithmDataFixture.rangeArgsProvider(LocalDateTime::toLocalDate);
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class PreciseShardingValueDoShardingTest extends AbstractPreciseShardingValueParameterizedTest<LocalDate> {
+        
+        public PreciseShardingValueDoShardingTest(final LocalDate input, final String expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            return preciseArgsProviderAsLocalDate();
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class RangeShardingValueDoShardingTest extends AbstractRangeShardingValueParameterizedTest<LocalDate> {
+        
+        public RangeShardingValueDoShardingTest(final Range<LocalDate> input, final Collection<String> expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            return rangeArgsProviderAsLocalDate();
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/OffsetDateTimeCosIdIntervalShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/OffsetDateTimeCosIdIntervalShardingAlgor [...]
new file mode 100644
index 00000000000..e9b2e300dad
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/OffsetDateTimeCosIdIntervalShardingAlgorithmTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.cosid.algorithm.sharding.interval;
+
+import com.google.common.collect.Range;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import org.apache.shardingsphere.sharding.cosid.algorithm.sharding.interval.fixture.IntervalShardingAlgorithmDataFixture;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+public final class OffsetDateTimeCosIdIntervalShardingAlgorithmTest {
+    
+    static Iterable<Object[]> preciseArgsProviderAsOffsetDateTime() {
+        return IntervalShardingAlgorithmDataFixture.preciseArgsProvider(ldt -> OffsetDateTime.of(ldt, IntervalShardingAlgorithmDataFixture.ZONE_OFFSET_SHANGHAI));
+    }
+    
+    static Iterable<Object[]> rangeArgsProviderAsOffsetDateTime() {
+        return IntervalShardingAlgorithmDataFixture.rangeArgsProvider(ldt -> OffsetDateTime.of(ldt, IntervalShardingAlgorithmDataFixture.ZONE_OFFSET_SHANGHAI));
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class PreciseShardingValueDoShardingTest extends AbstractPreciseShardingValueParameterizedTest<OffsetDateTime> {
+        
+        public PreciseShardingValueDoShardingTest(final OffsetDateTime input, final String expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            return preciseArgsProviderAsOffsetDateTime();
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class RangeShardingValueDoShardingTest extends AbstractRangeShardingValueParameterizedTest<OffsetDateTime> {
+        
+        public RangeShardingValueDoShardingTest(final Range<OffsetDateTime> input, final Collection<String> expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            return rangeArgsProviderAsOffsetDateTime();
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/SqlDateCosIdIntervalShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/SqlDateCosIdIntervalShardingAlgorithmTest.java
new file mode 100644
index 00000000000..1a30dfb52b5
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/SqlDateCosIdIntervalShardingAlgorithmTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.cosid.algorithm.sharding.interval;
+
+import com.google.common.collect.Range;
+import java.util.Collection;
+import java.sql.Date;
+import org.apache.shardingsphere.sharding.cosid.algorithm.sharding.interval.fixture.IntervalShardingAlgorithmDataFixture;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+public final class SqlDateCosIdIntervalShardingAlgorithmTest {
+    
+    static Iterable<Object[]> preciseArgsProviderAsDate() {
+        return IntervalShardingAlgorithmDataFixture.preciseArgsProvider(ldt -> new Date(ldt.toInstant(IntervalShardingAlgorithmDataFixture.ZONE_OFFSET_SHANGHAI).toEpochMilli()));
+    }
+    
+    static Iterable<Object[]> rangeArgsProviderAsDate() {
+        return IntervalShardingAlgorithmDataFixture.rangeArgsProvider(ldt -> new Date(ldt.toInstant(IntervalShardingAlgorithmDataFixture.ZONE_OFFSET_SHANGHAI).toEpochMilli()));
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class PreciseShardingValueDoShardingTest extends AbstractPreciseShardingValueParameterizedTest<Date> {
+        
+        public PreciseShardingValueDoShardingTest(final Date input, final String expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            return preciseArgsProviderAsDate();
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class RangeShardingValueDoShardingTest extends AbstractRangeShardingValueParameterizedTest<Date> {
+        
+        public RangeShardingValueDoShardingTest(final Range<Date> input, final Collection<String> expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            return rangeArgsProviderAsDate();
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/SqlTimestampCosIdIntervalShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/SqlTimestampCosIdIntervalShardingAlgorithm [...]
new file mode 100644
index 00000000000..21814f3b05d
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/SqlTimestampCosIdIntervalShardingAlgorithmTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.cosid.algorithm.sharding.interval;
+
+import com.google.common.collect.Range;
+import java.sql.Timestamp;
+import java.util.Collection;
+import org.apache.shardingsphere.sharding.cosid.algorithm.sharding.interval.fixture.IntervalShardingAlgorithmDataFixture;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+public final class SqlTimestampCosIdIntervalShardingAlgorithmTest {
+    
+    static Iterable<Object[]> preciseArgsProviderAsTimestamp() {
+        return IntervalShardingAlgorithmDataFixture.preciseArgsProvider(ldt -> new Timestamp(ldt.toInstant(IntervalShardingAlgorithmDataFixture.ZONE_OFFSET_SHANGHAI).toEpochMilli()));
+    }
+    
+    static Iterable<Object[]> rangeArgsProviderAsTimestamp() {
+        return IntervalShardingAlgorithmDataFixture.rangeArgsProvider(ldt -> new Timestamp(ldt.toInstant(IntervalShardingAlgorithmDataFixture.ZONE_OFFSET_SHANGHAI).toEpochMilli()));
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class PreciseShardingValueDoShardingTest extends AbstractPreciseShardingValueParameterizedTest<Timestamp> {
+        
+        public PreciseShardingValueDoShardingTest(final Timestamp input, final String expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            return preciseArgsProviderAsTimestamp();
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class RangeShardingValueDoShardingTest extends AbstractRangeShardingValueParameterizedTest<Timestamp> {
+        
+        public RangeShardingValueDoShardingTest(final Range<Timestamp> input, final Collection<String> expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            return rangeArgsProviderAsTimestamp();
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/YearMonthCosIdIntervalShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/YearMonthCosIdIntervalShardingAlgorithmTest.java
new file mode 100644
index 00000000000..6be0cb64031
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/YearMonthCosIdIntervalShardingAlgorithmTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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.cosid.algorithm.sharding.interval;
+
+import com.google.common.collect.Range;
+import java.time.YearMonth;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import me.ahoo.cosid.sharding.ExactCollection;
+import org.apache.shardingsphere.sharding.cosid.algorithm.sharding.interval.fixture.IntervalShardingAlgorithmDataFixture;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+public final class YearMonthCosIdIntervalShardingAlgorithmTest {
+    
+    static Iterable<Object[]> preciseArgsProviderAsYearMonth() {
+        return IntervalShardingAlgorithmDataFixture.preciseArgsProvider(ldt -> YearMonth.of(ldt.getYear(), ldt.getMonth()));
+    }
+    
+    static Iterable<Object[]> rangeArgsProviderAsYearMonth() {
+        return IntervalShardingAlgorithmDataFixture.rangeArgsProvider(ldt -> YearMonth.of(ldt.getYear(), ldt.getMonth()));
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class PreciseShardingValueDoShardingTest extends AbstractPreciseShardingValueParameterizedTest<YearMonth> {
+        
+        public PreciseShardingValueDoShardingTest(final YearMonth input, final String expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            return preciseArgsProviderAsYearMonth();
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class RangeShardingValueDoShardingTest extends AbstractRangeShardingValueParameterizedTest<YearMonth> {
+        
+        public RangeShardingValueDoShardingTest(final Range<YearMonth> input, final Collection<String> expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            List<Object[]> args = new LinkedList<>();
+            Iterable<Object[]> iterable = rangeArgsProviderAsYearMonth();
+            iterable.forEach(each -> {
+                Range<YearMonth> shardingValue = (Range<YearMonth>) each[0];
+                ExactCollection<String> expect = (ExactCollection<String>) each[1];
+                if (shardingValue.equals(Range.lessThan(YearMonth.of(2021, 5)))) {
+                    expect = new ExactCollection<>("table_202101", "table_202102", "table_202103", "table_202104");
+                }
+                args.add(new Object[]{shardingValue, expect});
+            });
+            return args;
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/ZonedDateTimeCosIdIntervalShardingAlgorithmTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/ZonedDateTimeCosIdIntervalShardingAlgorit [...]
new file mode 100644
index 00000000000..ea4c3ecc882
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-plugin/shardingsphere-sharding-cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/sharding/interval/ZonedDateTimeCosIdIntervalShardingAlgorithmTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.cosid.algorithm.sharding.interval;
+
+import com.google.common.collect.Range;
+import java.time.ZonedDateTime;
+import java.util.Collection;
+import org.apache.shardingsphere.sharding.cosid.algorithm.sharding.interval.fixture.IntervalShardingAlgorithmDataFixture;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+public final class ZonedDateTimeCosIdIntervalShardingAlgorithmTest {
+    
+    static Iterable<Object[]> preciseArgsProviderAsZonedDateTime() {
+        return IntervalShardingAlgorithmDataFixture.preciseArgsProvider(ldt -> ZonedDateTime.of(ldt, IntervalShardingAlgorithmDataFixture.ZONE_OFFSET_SHANGHAI));
+    }
+    
+    static Iterable<Object[]> rangeArgsProviderAsZonedDateTime() {
+        return IntervalShardingAlgorithmDataFixture.rangeArgsProvider(ldt -> ZonedDateTime.of(ldt, IntervalShardingAlgorithmDataFixture.ZONE_OFFSET_SHANGHAI));
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class PreciseShardingValueDoShardingTest extends AbstractPreciseShardingValueParameterizedTest<ZonedDateTime> {
+        
+        public PreciseShardingValueDoShardingTest(final ZonedDateTime input, final String expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            return preciseArgsProviderAsZonedDateTime();
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+    
+    @RunWith(Parameterized.class)
+    public static final class RangeShardingValueDoShardingTest extends AbstractRangeShardingValueParameterizedTest<ZonedDateTime> {
+        
+        public RangeShardingValueDoShardingTest(final Range<ZonedDateTime> input, final Collection<String> expected) {
+            super(input, expected);
+        }
+        
+        @Parameterized.Parameters(name = "{index}: doSharding({0})={1}")
+        public static Iterable<Object[]> argsProvider() {
+            return rangeArgsProviderAsZonedDateTime();
+        }
+        
+        @Test
+        public void assertDoSharding() {
+            doSharding();
+        }
+    }
+}