You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by GitBox <gi...@apache.org> on 2020/04/28 07:06:55 UTC

[GitHub] [shardingsphere] SanmuWangZJU opened a new pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

SanmuWangZJU opened a new pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363


   Ref #5282.
   
   Changes proposed in this pull request:
   - supply a sharding algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR/...
   - usage examples
     - shard by QUARTER_OF_YEAR (按季度分表)
     ``` yaml 
         actualDataNodes: ds.t_order_20${10..20}0${1..4}
         tableStrategy:
           standard:
             shardingColumn: created_at
             shardingAlgorithm:
               type: CUSTOM_DATE_TIME
               props:
                 datetime.format: yyyy-MM-dd HH:mm:ss
                 table.suffix.format: yyyyQQ
                 datetime.lower: "2010-01-01 00:00:00.000"
                 datetime.upper: "2020-12-31 00:00:00.000"
                 datetime.step.unit: Months
                 datetime.step.amount: 3
     ```
   - config detail can refer to java doc of `org.apache.shardingsphere.core.strategy.algorithm.sharding.CustomDateTimeShardingAlgorithmTest`


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] tristaZero commented on pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#issuecomment-620470762


   Hi @SanmuWangZJU 
   What you did is greater than expected! 
   There are just a few suggestions in the comment section, please review them and look forward to your reply. :-)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] tristaZero commented on pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#issuecomment-620961337


   @SanmuWangZJU 
   Your PR told me you are veteran, welcome to play a valuable role in ShardingSphere community. 😉 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] SanmuWangZJU commented on a change in pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
SanmuWangZJU commented on a change in pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#discussion_r416537434



##########
File path: sharding-core/sharding-core-common/src/main/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithm.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.core.strategy.algorithm.sharding;
+
+import com.google.common.base.Preconditions;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.RangeShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.StandardShardingAlgorithm;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeParseException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.ChronoUnit;
+import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalField;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Datetime sharding algorithm that adapt various shard method by define properties below.
+ *
+ * <p>properties defined here:
+ *
+ * <p>datetime.format: the datetime format used by applications, must can be transformed to {@link LocalDateTime},
+ * used by {@link LocalDateTime#parse(CharSequence, DateTimeFormatter)}.
+ *
+ * <p>table.suffix.format: suffix for sharded tables, used by {@link LocalDateTime#format(DateTimeFormatter)},
+ * examples:
+ * suffix=yyyyQQ means shard by {@link IsoFields#QUARTER_OF_YEAR};
+ * suffix=yyyyMMdd means shard by {@link ChronoField#DAY_OF_YEAR}.
+ *
+ * <p>detail explain for each char in datetime.format and table.suffix.format can refer {@link TemporalField}.
+ *
+ * <p>datetime.lower and datetime.upper: if app query with only half bound, lower and upper helps to build other half bound,
+ * datetime.lower must be specified and datetime.upper has a default value to {@link LocalDateTime#now}
+ * (default value of datetime.upper could only be used when query sql needn't get result that time larger than query time).
+ *
+ * <p>datetime.step.unit and datetime.step.amount used for calculate tables for range shard, datetime.step.unit is name of
+ * {@link ChronoUnit}, default unit is Days and amount is 1, amount + unit should not be larger than but close to your shard range.
+ *
+ * <p>examples: when shard by {@link IsoFields#QUARTER_OF_YEAR}, datetime.step.unit = Months and datetime.step.amount = 3 is a better choice.
+ */
+public class CustomDateTimeShardingAlgorithm implements StandardShardingAlgorithm<Comparable<?>> {
+
+    private static final String DATE_TIME_FORMAT = "datetime.format";
+
+    private static final String TABLE_SUFFIX_FORMAT = "table.suffix.format";
+
+    private static final String DEFAULT_LOWER = "datetime.lower";
+
+    private static final String DEFAULT_UPPER = "datetime.upper";
+
+    private static final String STEP_UNIT = "datetime.step.unit";
+
+    private static final String STEP_AMOUNT = "datetime.step.amount";
+
+    private DateTimeFormatter datetimeFormatter;
+
+    private ChronoUnit stepUnit;
+
+    private int stepAmount;
+
+    private volatile boolean init;
+
+    @Getter
+    @Setter
+    private Properties properties = new Properties();
+
+    @Override
+    public String doSharding(final Collection<String> availableTargetNames, final PreciseShardingValue<Comparable<?>> shardingValue) {
+        checkInit();
+        return availableTargetNames.stream()
+                .filter(tableName -> tableName.endsWith(formatForDateTime(parseDateTimeForValue(shardingValue.getValue().toString()))))
+                .findFirst().orElseThrow(() -> new UnsupportedOperationException(
+                        String.format("failed to shard value %s, and availableTables %s", shardingValue, availableTargetNames)));
+    }
+
+    @Override
+    public Collection<String> doSharding(final Collection<String> availableTargetNames, final RangeShardingValue<Comparable<?>> shardingValue) {
+        checkInit();
+        boolean hasStart = shardingValue.getValueRange().hasLowerBound();
+        boolean hasEnd = shardingValue.getValueRange().hasUpperBound();
+        Set<String> tables = new HashSet<>();
+        if (!hasStart && !hasEnd) {
+            return availableTargetNames;
+        }
+        LocalDateTime start = hasStart
+                ? parseDateTimeForValue(shardingValue.getValueRange().lowerEndpoint().toString())
+                : parseDateTimeForValue(properties.getProperty(DEFAULT_LOWER));
+        LocalDateTime end = hasEnd
+                ? parseDateTimeForValue(shardingValue.getValueRange().upperEndpoint().toString())
+                : properties.getProperty(DEFAULT_UPPER) == null
+                ? LocalDateTime.now()
+                : parseDateTimeForValue(properties.getProperty(DEFAULT_UPPER));
+        LocalDateTime tmp = start;
+        while (!tmp.isAfter(end)) {
+            mergeTableIfMatch(tmp, tables, availableTargetNames);
+            tmp = tmp.plus(stepAmount, stepUnit);
+        }
+        mergeTableIfMatch(end, tables, availableTargetNames);
+        return tables;
+    }
+
+    private LocalDateTime parseDateTimeForValue(final String value) {
+        return LocalDateTime.parse(value.substring(0, properties.getProperty(DATE_TIME_FORMAT).length()), datetimeFormatter);
+    }
+
+    private String formatForDateTime(final LocalDateTime localDateTime) {
+        return localDateTime.format(DateTimeFormatter.ofPattern(properties.get(TABLE_SUFFIX_FORMAT).toString()));
+    }
+
+    private void mergeTableIfMatch(final LocalDateTime dateTime, final Collection<String> tables, final Collection<String> availableTargetNames) {
+        String suffix = formatForDateTime(dateTime);
+        availableTargetNames.parallelStream().filter(tableName -> tableName.endsWith(suffix)).findAny().map(tables::add);
+    }
+
+    private void checkInit() {

Review comment:
       sync block here is used to ensure `verifyProperties()` method exactly execute once when multi-thread call `doSharding()`, inspired by Double-Checked Locking with Singleton




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] coveralls commented on pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
coveralls commented on pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#issuecomment-620433590


   ## Pull Request Test Coverage Report for [Build 11328](https://coveralls.io/builds/30392618)
   
   * **52** of **59**   **(88.14%)**  changed or added relevant lines in **1** file are covered.
   * No unchanged relevant lines lost coverage.
   * Overall coverage increased (+**0.09%**) to **57.501%**
   
   ---
   
   |  Changes Missing Coverage | Covered Lines | Changed/Added Lines | % |
   | :-----|--------------|--------|---: |
   | [sharding-core/sharding-core-common/src/main/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithm.java](https://coveralls.io/builds/30392618/source?filename=sharding-core%2Fsharding-core-common%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Fcore%2Fstrategy%2Falgorithm%2Fsharding%2FCustomDateTimeShardingAlgorithm.java#L95) | 52 | 59 | 88.14%
   <!-- | **Total:** | **52** | **59** | **88.14%** | -->
   
   
   |  Totals | [![Coverage Status](https://coveralls.io/builds/30392618/badge)](https://coveralls.io/builds/30392618) |
   | :-- | --: |
   | Change from base [Build 11327](https://coveralls.io/builds/30391678): |  0.09% |
   | Covered Lines: | 11939 |
   | Relevant Lines: | 20763 |
   
   ---
   ##### 💛  - [Coveralls](https://coveralls.io)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] SanmuWangZJU commented on a change in pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
SanmuWangZJU commented on a change in pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#discussion_r416650930



##########
File path: sharding-core/sharding-core-common/src/test/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithmTest.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.core.strategy.algorithm.sharding;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import org.apache.shardingsphere.api.config.sharding.strategy.StandardShardingStrategyConfiguration;
+import org.apache.shardingsphere.core.strategy.route.standard.StandardShardingStrategy;
+import org.apache.shardingsphere.core.strategy.route.value.ListRouteValue;
+import org.apache.shardingsphere.core.strategy.route.value.RangeRouteValue;
+import org.apache.shardingsphere.core.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.underlying.common.config.properties.ConfigurationProperties;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * datetime sharding algorithm test.
+ */
+public class CustomDateTimeShardingAlgorithmTest {
+
+    private final List<String> availableTables = new ArrayList<>();
+
+    private StandardShardingStrategy shardingStrategy;
+
+    @Before
+    public void setup() {
+        CustomDateTimeShardingAlgorithm shardingAlgorithm = new CustomDateTimeShardingAlgorithm();

Review comment:
       I have just add test case for sharding by month, looking forward to further suggestions




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] SanmuWangZJU commented on pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
SanmuWangZJU commented on pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#issuecomment-620633187


   Hi @tristaZero  @kimmking 
   Thanks for your suggestions, I have add  test case for yyyyMM(shard by Month) and fix some format issues.
   Looking forward to your further suggestions :)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] kimmking commented on a change in pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
kimmking commented on a change in pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#discussion_r416476187



##########
File path: sharding-core/sharding-core-common/src/main/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithm.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.core.strategy.algorithm.sharding;
+
+import com.google.common.base.Preconditions;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.RangeShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.StandardShardingAlgorithm;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeParseException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.ChronoUnit;
+import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalField;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Datetime sharding algorithm that adapt various shard method by define properties below.
+ *
+ * <p>properties defined here:
+ *
+ * <p>datetime.format: the datetime format used by applications, must can be transformed to {@link LocalDateTime},
+ * used by {@link LocalDateTime#parse(CharSequence, DateTimeFormatter)}.
+ *
+ * <p>table.suffix.format: suffix for sharded tables, used by {@link LocalDateTime#format(DateTimeFormatter)},
+ * examples:
+ * suffix=yyyyQQ means shard by {@link IsoFields#QUARTER_OF_YEAR};
+ * suffix=yyyyMMdd means shard by {@link ChronoField#DAY_OF_YEAR}.
+ *
+ * <p>detail explain for each char in datetime.format and table.suffix.format can refer {@link TemporalField}.
+ *
+ * <p>datetime.lower and datetime.upper: if app query with only half bound, lower and upper helps to build other half bound,
+ * datetime.lower must be specified and datetime.upper has a default value to {@link LocalDateTime#now}
+ * (default value of datetime.upper could only be used when query sql needn't get result that time larger than query time).
+ *
+ * <p>datetime.step.unit and datetime.step.amount used for calculate tables for range shard, datetime.step.unit is name of
+ * {@link ChronoUnit}, default unit is Days and amount is 1, amount + unit should not be larger than but close to your shard range.
+ *
+ * <p>examples: when shard by {@link IsoFields#QUARTER_OF_YEAR}, datetime.step.unit = Months and datetime.step.amount = 3 is a better choice.
+ */
+public class CustomDateTimeShardingAlgorithm implements StandardShardingAlgorithm<Comparable<?>> {
+
+    private static final String DATE_TIME_FORMAT = "datetime.format";
+
+    private static final String TABLE_SUFFIX_FORMAT = "table.suffix.format";
+
+    private static final String DEFAULT_LOWER = "datetime.lower";
+
+    private static final String DEFAULT_UPPER = "datetime.upper";
+
+    private static final String STEP_UNIT = "datetime.step.unit";
+
+    private static final String STEP_AMOUNT = "datetime.step.amount";
+
+    private DateTimeFormatter datetimeFormatter;
+
+    private ChronoUnit stepUnit;
+
+    private int stepAmount;
+
+    private volatile boolean init;
+
+    @Getter
+    @Setter
+    private Properties properties = new Properties();
+
+    @Override
+    public String doSharding(final Collection<String> availableTargetNames, final PreciseShardingValue<Comparable<?>> shardingValue) {
+        checkInit();
+        return availableTargetNames.stream()
+                .filter(tableName -> tableName.endsWith(formatForDateTime(parseDateTimeForValue(shardingValue.getValue().toString()))))
+                .findFirst().orElseThrow(() -> new UnsupportedOperationException(
+                        String.format("failed to shard value %s, and availableTables %s", shardingValue, availableTargetNames)));
+    }
+
+    @Override
+    public Collection<String> doSharding(final Collection<String> availableTargetNames, final RangeShardingValue<Comparable<?>> shardingValue) {
+        checkInit();
+        boolean hasStart = shardingValue.getValueRange().hasLowerBound();
+        boolean hasEnd = shardingValue.getValueRange().hasUpperBound();
+        Set<String> tables = new HashSet<>();
+        if (!hasStart && !hasEnd) {
+            return availableTargetNames;
+        }
+        LocalDateTime start = hasStart
+                ? parseDateTimeForValue(shardingValue.getValueRange().lowerEndpoint().toString())
+                : parseDateTimeForValue(properties.getProperty(DEFAULT_LOWER));
+        LocalDateTime end = hasEnd
+                ? parseDateTimeForValue(shardingValue.getValueRange().upperEndpoint().toString())
+                : properties.getProperty(DEFAULT_UPPER) == null
+                ? LocalDateTime.now()
+                : parseDateTimeForValue(properties.getProperty(DEFAULT_UPPER));
+        LocalDateTime tmp = start;
+        while (!tmp.isAfter(end)) {
+            mergeTableIfMatch(tmp, tables, availableTargetNames);
+            tmp = tmp.plus(stepAmount, stepUnit);
+        }
+        mergeTableIfMatch(end, tables, availableTargetNames);
+        return tables;
+    }
+
+    private LocalDateTime parseDateTimeForValue(final String value) {
+        return LocalDateTime.parse(value.substring(0, properties.getProperty(DATE_TIME_FORMAT).length()), datetimeFormatter);
+    }
+
+    private String formatForDateTime(final LocalDateTime localDateTime) {
+        return localDateTime.format(DateTimeFormatter.ofPattern(properties.get(TABLE_SUFFIX_FORMAT).toString()));
+    }
+
+    private void mergeTableIfMatch(final LocalDateTime dateTime, final Collection<String> tables, final Collection<String> availableTargetNames) {
+        String suffix = formatForDateTime(dateTime);
+        availableTargetNames.parallelStream().filter(tableName -> tableName.endsWith(suffix)).findAny().map(tables::add);
+    }
+
+    private void checkInit() {
+        if (!init) {
+            synchronized (this) {
+                if (!init) {
+                    verifyProperties();
+                    init = true;
+                }
+            }
+        }
+    }
+
+    private void verifyProperties() {
+        Preconditions.checkNotNull(properties.getProperty(DATE_TIME_FORMAT));
+        Preconditions.checkNotNull(properties.getProperty(TABLE_SUFFIX_FORMAT));
+        Preconditions.checkNotNull(properties.getProperty(DEFAULT_LOWER));
+        stepUnit = properties.getProperty(STEP_UNIT) == null
+                ? stepUnit = ChronoUnit.DAYS

Review comment:
       remove tepUnit = 

##########
File path: sharding-core/sharding-core-common/src/main/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithm.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.core.strategy.algorithm.sharding;
+
+import com.google.common.base.Preconditions;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.RangeShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.StandardShardingAlgorithm;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeParseException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.ChronoUnit;
+import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalField;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Datetime sharding algorithm that adapt various shard method by define properties below.
+ *
+ * <p>properties defined here:
+ *
+ * <p>datetime.format: the datetime format used by applications, must can be transformed to {@link LocalDateTime},
+ * used by {@link LocalDateTime#parse(CharSequence, DateTimeFormatter)}.
+ *
+ * <p>table.suffix.format: suffix for sharded tables, used by {@link LocalDateTime#format(DateTimeFormatter)},
+ * examples:
+ * suffix=yyyyQQ means shard by {@link IsoFields#QUARTER_OF_YEAR};
+ * suffix=yyyyMMdd means shard by {@link ChronoField#DAY_OF_YEAR}.
+ *
+ * <p>detail explain for each char in datetime.format and table.suffix.format can refer {@link TemporalField}.
+ *
+ * <p>datetime.lower and datetime.upper: if app query with only half bound, lower and upper helps to build other half bound,
+ * datetime.lower must be specified and datetime.upper has a default value to {@link LocalDateTime#now}
+ * (default value of datetime.upper could only be used when query sql needn't get result that time larger than query time).
+ *
+ * <p>datetime.step.unit and datetime.step.amount used for calculate tables for range shard, datetime.step.unit is name of
+ * {@link ChronoUnit}, default unit is Days and amount is 1, amount + unit should not be larger than but close to your shard range.
+ *

Review comment:
       Me too

##########
File path: sharding-core/sharding-core-common/src/main/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithm.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.core.strategy.algorithm.sharding;
+
+import com.google.common.base.Preconditions;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.RangeShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.StandardShardingAlgorithm;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeParseException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.ChronoUnit;
+import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalField;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Datetime sharding algorithm that adapt various shard method by define properties below.
+ *
+ * <p>properties defined here:
+ *
+ * <p>datetime.format: the datetime format used by applications, must can be transformed to {@link LocalDateTime},
+ * used by {@link LocalDateTime#parse(CharSequence, DateTimeFormatter)}.
+ *
+ * <p>table.suffix.format: suffix for sharded tables, used by {@link LocalDateTime#format(DateTimeFormatter)},
+ * examples:
+ * suffix=yyyyQQ means shard by {@link IsoFields#QUARTER_OF_YEAR};
+ * suffix=yyyyMMdd means shard by {@link ChronoField#DAY_OF_YEAR}.

Review comment:
       yyyyMM is also a common style.

##########
File path: sharding-core/sharding-core-common/src/main/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithm.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.core.strategy.algorithm.sharding;
+
+import com.google.common.base.Preconditions;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.RangeShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.StandardShardingAlgorithm;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeParseException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.ChronoUnit;
+import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalField;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Datetime sharding algorithm that adapt various shard method by define properties below.
+ *
+ * <p>properties defined here:
+ *
+ * <p>datetime.format: the datetime format used by applications, must can be transformed to {@link LocalDateTime},
+ * used by {@link LocalDateTime#parse(CharSequence, DateTimeFormatter)}.
+ *
+ * <p>table.suffix.format: suffix for sharded tables, used by {@link LocalDateTime#format(DateTimeFormatter)},
+ * examples:
+ * suffix=yyyyQQ means shard by {@link IsoFields#QUARTER_OF_YEAR};
+ * suffix=yyyyMMdd means shard by {@link ChronoField#DAY_OF_YEAR}.
+ *
+ * <p>detail explain for each char in datetime.format and table.suffix.format can refer {@link TemporalField}.
+ *
+ * <p>datetime.lower and datetime.upper: if app query with only half bound, lower and upper helps to build other half bound,
+ * datetime.lower must be specified and datetime.upper has a default value to {@link LocalDateTime#now}
+ * (default value of datetime.upper could only be used when query sql needn't get result that time larger than query time).
+ *
+ * <p>datetime.step.unit and datetime.step.amount used for calculate tables for range shard, datetime.step.unit is name of
+ * {@link ChronoUnit}, default unit is Days and amount is 1, amount + unit should not be larger than but close to your shard range.
+ *
+ * <p>examples: when shard by {@link IsoFields#QUARTER_OF_YEAR}, datetime.step.unit = Months and datetime.step.amount = 3 is a better choice.
+ */
+public class CustomDateTimeShardingAlgorithm implements StandardShardingAlgorithm<Comparable<?>> {
+
+    private static final String DATE_TIME_FORMAT = "datetime.format";
+
+    private static final String TABLE_SUFFIX_FORMAT = "table.suffix.format";
+
+    private static final String DEFAULT_LOWER = "datetime.lower";
+
+    private static final String DEFAULT_UPPER = "datetime.upper";
+
+    private static final String STEP_UNIT = "datetime.step.unit";
+
+    private static final String STEP_AMOUNT = "datetime.step.amount";
+
+    private DateTimeFormatter datetimeFormatter;
+
+    private ChronoUnit stepUnit;
+
+    private int stepAmount;
+
+    private volatile boolean init;
+
+    @Getter
+    @Setter
+    private Properties properties = new Properties();
+
+    @Override
+    public String doSharding(final Collection<String> availableTargetNames, final PreciseShardingValue<Comparable<?>> shardingValue) {
+        checkInit();
+        return availableTargetNames.stream()
+                .filter(tableName -> tableName.endsWith(formatForDateTime(parseDateTimeForValue(shardingValue.getValue().toString()))))
+                .findFirst().orElseThrow(() -> new UnsupportedOperationException(
+                        String.format("failed to shard value %s, and availableTables %s", shardingValue, availableTargetNames)));
+    }
+
+    @Override
+    public Collection<String> doSharding(final Collection<String> availableTargetNames, final RangeShardingValue<Comparable<?>> shardingValue) {
+        checkInit();
+        boolean hasStart = shardingValue.getValueRange().hasLowerBound();
+        boolean hasEnd = shardingValue.getValueRange().hasUpperBound();
+        Set<String> tables = new HashSet<>();
+        if (!hasStart && !hasEnd) {
+            return availableTargetNames;
+        }
+        LocalDateTime start = hasStart
+                ? parseDateTimeForValue(shardingValue.getValueRange().lowerEndpoint().toString())
+                : parseDateTimeForValue(properties.getProperty(DEFAULT_LOWER));
+        LocalDateTime end = hasEnd
+                ? parseDateTimeForValue(shardingValue.getValueRange().upperEndpoint().toString())
+                : properties.getProperty(DEFAULT_UPPER) == null
+                ? LocalDateTime.now()
+                : parseDateTimeForValue(properties.getProperty(DEFAULT_UPPER));
+        LocalDateTime tmp = start;
+        while (!tmp.isAfter(end)) {
+            mergeTableIfMatch(tmp, tables, availableTargetNames);
+            tmp = tmp.plus(stepAmount, stepUnit);
+        }
+        mergeTableIfMatch(end, tables, availableTargetNames);
+        return tables;
+    }
+
+    private LocalDateTime parseDateTimeForValue(final String value) {
+        return LocalDateTime.parse(value.substring(0, properties.getProperty(DATE_TIME_FORMAT).length()), datetimeFormatter);
+    }
+
+    private String formatForDateTime(final LocalDateTime localDateTime) {
+        return localDateTime.format(DateTimeFormatter.ofPattern(properties.get(TABLE_SUFFIX_FORMAT).toString()));
+    }
+
+    private void mergeTableIfMatch(final LocalDateTime dateTime, final Collection<String> tables, final Collection<String> availableTargetNames) {
+        String suffix = formatForDateTime(dateTime);
+        availableTargetNames.parallelStream().filter(tableName -> tableName.endsWith(suffix)).findAny().map(tables::add);
+    }
+
+    private void checkInit() {

Review comment:
       why add sync block here?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] tristaZero commented on pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#issuecomment-620955509


   Hi @SanmuWangZJU 
   I am glad you contribute this practical idea to the os community! Hope more people can benefit from it.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] SanmuWangZJU commented on pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
SanmuWangZJU commented on pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#issuecomment-620966405


   > @SanmuWangZJU
   > Your PR told me you are veteran, welcome to play a valuable role in ShardingSphere community. 😉
   
   Thanks, hope more of my ideas benefit the community.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] tristaZero commented on a change in pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
tristaZero commented on a change in pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#discussion_r416429114



##########
File path: sharding-core/sharding-core-common/src/test/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithmTest.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.core.strategy.algorithm.sharding;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import org.apache.shardingsphere.api.config.sharding.strategy.StandardShardingStrategyConfiguration;
+import org.apache.shardingsphere.core.strategy.route.standard.StandardShardingStrategy;
+import org.apache.shardingsphere.core.strategy.route.value.ListRouteValue;
+import org.apache.shardingsphere.core.strategy.route.value.RangeRouteValue;
+import org.apache.shardingsphere.core.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.underlying.common.config.properties.ConfigurationProperties;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * datetime sharding algorithm test.
+ */
+public class CustomDateTimeShardingAlgorithmTest {
+
+    private final List<String> availableTables = new ArrayList<>();
+
+    private StandardShardingStrategy shardingStrategy;
+
+    @Before
+    public void setup() {
+        CustomDateTimeShardingAlgorithm shardingAlgorithm = new CustomDateTimeShardingAlgorithm();
+        shardingAlgorithm.getProperties().setProperty("datetime.format", "yyyy-MM-dd HH:mm:ss");
+        shardingAlgorithm.getProperties().setProperty("table.suffix.format", "yyyyQQ");
+        shardingAlgorithm.getProperties().setProperty("datetime.lower", "2016-01-01 00:00:00.000");
+        shardingAlgorithm.getProperties().setProperty("datetime.upper", "2021-12-31 00:00:00.000");
+        shardingAlgorithm.getProperties().setProperty("datetime.step.unit", "Months");
+        shardingAlgorithm.getProperties().setProperty("datetime.step.amount", "3");
+        StandardShardingStrategyConfiguration shardingStrategyConfig = new StandardShardingStrategyConfiguration("create_time", shardingAlgorithm);
+        this.shardingStrategy = new StandardShardingStrategy(shardingStrategyConfig);
+
+        for (int i = 2016; i <= 2020; i++) {
+            for (int j = 1; j <= 4; j++) {
+                availableTables.add(String.format("t_order_%04d%02d", i, j));
+            }
+        }
+    }
+
+    @Test
+    public void assertPreciseDoSharding() {
+        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")));
+        Collection<String> actual = shardingStrategy.doSharding(availableTables, shardingValues, new ConfigurationProperties(new Properties()));
+        assertThat(actual.size(), is(2));
+        assertTrue(actual.contains("t_order_202001"));
+        assertTrue(actual.contains("t_order_202002"));
+    }
+
+    @Test
+    public void assertRangeDoSharding() {
+        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));
+        Collection<String> actual = shardingStrategy.doSharding(availableTables, shardingValues, new ConfigurationProperties(new Properties()));
+        assertThat(actual.size(), is(3));
+    }
+
+    @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));
+        Collection<String> actual = shardingStrategy.doSharding(availableTables, shardingValues, new ConfigurationProperties(new Properties()));
+        assertThat(actual.size(), is(9));
+    }
+
+    @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));
+        Collection<String> actual = shardingStrategy.doSharding(availableTables, shardingValues, new ConfigurationProperties(new Properties()));
+        assertThat(actual.size(), is(15));
+    }
+
+    @Test
+    public void testFormat() {

Review comment:
       Considering the consistent function names, `assertFormat` is better.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] kimmking commented on pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
kimmking commented on pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#issuecomment-620508205


   Good job.
   And one suggestion here:
   1. support yyyyMM for users, it may be the most popular style for this case.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] tristaZero commented on a change in pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
tristaZero commented on a change in pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#discussion_r416425277



##########
File path: sharding-core/sharding-core-common/src/main/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithm.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.core.strategy.algorithm.sharding;
+
+import com.google.common.base.Preconditions;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.RangeShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.StandardShardingAlgorithm;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeParseException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.ChronoUnit;
+import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalField;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Datetime sharding algorithm that adapt various shard method by define properties below.
+ *
+ * <p>properties defined here:
+ *
+ * <p>datetime.format: the datetime format used by applications, must can be transformed to {@link LocalDateTime},
+ * used by {@link LocalDateTime#parse(CharSequence, DateTimeFormatter)}.
+ *
+ * <p>table.suffix.format: suffix for sharded tables, used by {@link LocalDateTime#format(DateTimeFormatter)},
+ * examples:
+ * suffix=yyyyQQ means shard by {@link IsoFields#QUARTER_OF_YEAR};
+ * suffix=yyyyMMdd means shard by {@link ChronoField#DAY_OF_YEAR}.
+ *
+ * <p>detail explain for each char in datetime.format and table.suffix.format can refer {@link TemporalField}.
+ *
+ * <p>datetime.lower and datetime.upper: if app query with only half bound, lower and upper helps to build other half bound,
+ * datetime.lower must be specified and datetime.upper has a default value to {@link LocalDateTime#now}
+ * (default value of datetime.upper could only be used when query sql needn't get result that time larger than query time).
+ *
+ * <p>datetime.step.unit and datetime.step.amount used for calculate tables for range shard, datetime.step.unit is name of
+ * {@link ChronoUnit}, default unit is Days and amount is 1, amount + unit should not be larger than but close to your shard range.
+ *

Review comment:
       I like this detailed java doc.

##########
File path: sharding-core/sharding-core-common/src/test/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithmTest.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.core.strategy.algorithm.sharding;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import org.apache.shardingsphere.api.config.sharding.strategy.StandardShardingStrategyConfiguration;
+import org.apache.shardingsphere.core.strategy.route.standard.StandardShardingStrategy;
+import org.apache.shardingsphere.core.strategy.route.value.ListRouteValue;
+import org.apache.shardingsphere.core.strategy.route.value.RangeRouteValue;
+import org.apache.shardingsphere.core.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.underlying.common.config.properties.ConfigurationProperties;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * datetime sharding algorithm test.
+ */
+public class CustomDateTimeShardingAlgorithmTest {
+
+    private final List<String> availableTables = new ArrayList<>();
+
+    private StandardShardingStrategy shardingStrategy;
+
+    @Before
+    public void setup() {
+        CustomDateTimeShardingAlgorithm shardingAlgorithm = new CustomDateTimeShardingAlgorithm();
+        shardingAlgorithm.getProperties().setProperty("datetime.format", "yyyy-MM-dd HH:mm:ss");
+        shardingAlgorithm.getProperties().setProperty("table.suffix.format", "yyyyQQ");
+        shardingAlgorithm.getProperties().setProperty("datetime.lower", "2016-01-01 00:00:00.000");
+        shardingAlgorithm.getProperties().setProperty("datetime.upper", "2021-12-31 00:00:00.000");
+        shardingAlgorithm.getProperties().setProperty("datetime.step.unit", "Months");
+        shardingAlgorithm.getProperties().setProperty("datetime.step.amount", "3");
+        StandardShardingStrategyConfiguration shardingStrategyConfig = new StandardShardingStrategyConfiguration("create_time", shardingAlgorithm);
+        this.shardingStrategy = new StandardShardingStrategy(shardingStrategyConfig);
+

Review comment:
       A redundant blank line is recommended to remove.

##########
File path: sharding-core/sharding-core-common/src/test/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithmTest.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.core.strategy.algorithm.sharding;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import org.apache.shardingsphere.api.config.sharding.strategy.StandardShardingStrategyConfiguration;
+import org.apache.shardingsphere.core.strategy.route.standard.StandardShardingStrategy;
+import org.apache.shardingsphere.core.strategy.route.value.ListRouteValue;
+import org.apache.shardingsphere.core.strategy.route.value.RangeRouteValue;
+import org.apache.shardingsphere.core.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.underlying.common.config.properties.ConfigurationProperties;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * datetime sharding algorithm test.
+ */
+public class CustomDateTimeShardingAlgorithmTest {
+
+    private final List<String> availableTables = new ArrayList<>();
+
+    private StandardShardingStrategy shardingStrategy;
+
+    @Before
+    public void setup() {
+        CustomDateTimeShardingAlgorithm shardingAlgorithm = new CustomDateTimeShardingAlgorithm();
+        shardingAlgorithm.getProperties().setProperty("datetime.format", "yyyy-MM-dd HH:mm:ss");
+        shardingAlgorithm.getProperties().setProperty("table.suffix.format", "yyyyQQ");
+        shardingAlgorithm.getProperties().setProperty("datetime.lower", "2016-01-01 00:00:00.000");
+        shardingAlgorithm.getProperties().setProperty("datetime.upper", "2021-12-31 00:00:00.000");
+        shardingAlgorithm.getProperties().setProperty("datetime.step.unit", "Months");
+        shardingAlgorithm.getProperties().setProperty("datetime.step.amount", "3");
+        StandardShardingStrategyConfiguration shardingStrategyConfig = new StandardShardingStrategyConfiguration("create_time", shardingAlgorithm);
+        this.shardingStrategy = new StandardShardingStrategy(shardingStrategyConfig);
+
+        for (int i = 2016; i <= 2020; i++) {
+            for (int j = 1; j <= 4; j++) {
+                availableTables.add(String.format("t_order_%04d%02d", i, j));
+            }
+        }
+    }
+
+    @Test
+    public void assertPreciseDoSharding() {
+        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")));
+        Collection<String> actual = shardingStrategy.doSharding(availableTables, shardingValues, new ConfigurationProperties(new Properties()));
+        assertThat(actual.size(), is(2));
+        assertTrue(actual.contains("t_order_202001"));
+        assertTrue(actual.contains("t_order_202002"));
+    }
+
+    @Test
+    public void assertRangeDoSharding() {
+        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));
+        Collection<String> actual = shardingStrategy.doSharding(availableTables, shardingValues, new ConfigurationProperties(new Properties()));
+        assertThat(actual.size(), is(3));
+    }
+
+    @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));
+        Collection<String> actual = shardingStrategy.doSharding(availableTables, shardingValues, new ConfigurationProperties(new Properties()));
+        assertThat(actual.size(), is(9));
+    }
+
+    @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));
+        Collection<String> actual = shardingStrategy.doSharding(availableTables, shardingValues, new ConfigurationProperties(new Properties()));
+        assertThat(actual.size(), is(15));
+    }
+
+    @Test
+    public void testFormat() {

Review comment:
       Considering the consistent function name, `assertFormat` is better.

##########
File path: sharding-core/sharding-core-common/src/test/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithmTest.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.core.strategy.algorithm.sharding;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import org.apache.shardingsphere.api.config.sharding.strategy.StandardShardingStrategyConfiguration;
+import org.apache.shardingsphere.core.strategy.route.standard.StandardShardingStrategy;
+import org.apache.shardingsphere.core.strategy.route.value.ListRouteValue;
+import org.apache.shardingsphere.core.strategy.route.value.RangeRouteValue;
+import org.apache.shardingsphere.core.strategy.route.value.RouteValue;
+import org.apache.shardingsphere.underlying.common.config.properties.ConfigurationProperties;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * datetime sharding algorithm test.
+ */
+public class CustomDateTimeShardingAlgorithmTest {
+
+    private final List<String> availableTables = new ArrayList<>();
+
+    private StandardShardingStrategy shardingStrategy;
+
+    @Before
+    public void setup() {
+        CustomDateTimeShardingAlgorithm shardingAlgorithm = new CustomDateTimeShardingAlgorithm();

Review comment:
       This is a great practice t shard by quarter. Besides, could you provide another test case `like sharding by month or day` to present its flexibility to our users? 
   You know, a cool algorithm needs more examples to express its features.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] SanmuWangZJU commented on a change in pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
SanmuWangZJU commented on a change in pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#discussion_r416504070



##########
File path: sharding-core/sharding-core-common/src/main/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithm.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.core.strategy.algorithm.sharding;
+
+import com.google.common.base.Preconditions;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.api.sharding.standard.PreciseShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.RangeShardingValue;
+import org.apache.shardingsphere.api.sharding.standard.StandardShardingAlgorithm;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeParseException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.ChronoUnit;
+import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalField;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Datetime sharding algorithm that adapt various shard method by define properties below.
+ *
+ * <p>properties defined here:
+ *
+ * <p>datetime.format: the datetime format used by applications, must can be transformed to {@link LocalDateTime},
+ * used by {@link LocalDateTime#parse(CharSequence, DateTimeFormatter)}.
+ *
+ * <p>table.suffix.format: suffix for sharded tables, used by {@link LocalDateTime#format(DateTimeFormatter)},
+ * examples:
+ * suffix=yyyyQQ means shard by {@link IsoFields#QUARTER_OF_YEAR};
+ * suffix=yyyyMMdd means shard by {@link ChronoField#DAY_OF_YEAR}.

Review comment:
       yes, it support yyyyMM, I will add this example to java doc and provide corresponding test cases




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] codecov-io commented on pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#issuecomment-620581754


   # [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/5363?src=pr&el=h1) Report
   > Merging [#5363](https://codecov.io/gh/apache/shardingsphere/pull/5363?src=pr&el=desc) into [master](https://codecov.io/gh/apache/shardingsphere/commit/9fcf07eb42bcb662f09c31502e1c9e2813d70818&el=desc) will **increase** coverage by `0.25%`.
   > The diff coverage is `77.96%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/shardingsphere/pull/5363/graphs/tree.svg?width=650&height=150&src=pr&token=ZvlXpWa7so)](https://codecov.io/gh/apache/shardingsphere/pull/5363?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #5363      +/-   ##
   ============================================
   + Coverage     53.69%   53.94%   +0.25%     
     Complexity      412      412              
   ============================================
     Files          1156     1150       -6     
     Lines         20806    20758      -48     
     Branches       3763     3767       +4     
   ============================================
   + Hits          11171    11198      +27     
   + Misses         8918     8839      -79     
   - Partials        717      721       +4     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/shardingsphere/pull/5363?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...ithm/sharding/CustomDateTimeShardingAlgorithm.java](https://codecov.io/gh/apache/shardingsphere/pull/5363/diff?src=pr&el=tree#diff-c2hhcmRpbmctY29yZS9zaGFyZGluZy1jb3JlLWNvbW1vbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvY29yZS9zdHJhdGVneS9hbGdvcml0aG0vc2hhcmRpbmcvQ3VzdG9tRGF0ZVRpbWVTaGFyZGluZ0FsZ29yaXRobS5qYXZh) | `77.96% <77.96%> (ø)` | `0.00 <0.00> (?)` | |
   | [...shardingjdbc/jdbc/core/context/RuntimeContext.java](https://codecov.io/gh/apache/shardingsphere/pull/5363/diff?src=pr&el=tree#diff-c2hhcmRpbmctamRiYy9zaGFyZGluZy1qZGJjLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5namRiYy9qZGJjL2NvcmUvY29udGV4dC9SdW50aW1lQ29udGV4dC5qYXZh) | `86.84% <0.00%> (-5.27%)` | `0.00% <0.00%> (ø%)` | |
   | [...ngjdbc/jdbc/core/datasource/EncryptDataSource.java](https://codecov.io/gh/apache/shardingsphere/pull/5363/diff?src=pr&el=tree#diff-c2hhcmRpbmctamRiYy9zaGFyZGluZy1qZGJjLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5namRiYy9qZGJjL2NvcmUvZGF0YXNvdXJjZS9FbmNyeXB0RGF0YVNvdXJjZS5qYXZh) | `80.00% <0.00%> (-3.34%)` | `0.00% <0.00%> (ø%)` | |
   | [...gjdbc/jdbc/core/datasource/ShardingDataSource.java](https://codecov.io/gh/apache/shardingsphere/pull/5363/diff?src=pr&el=tree#diff-c2hhcmRpbmctamRiYy9zaGFyZGluZy1qZGJjLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5namRiYy9qZGJjL2NvcmUvZGF0YXNvdXJjZS9TaGFyZGluZ0RhdGFTb3VyY2UuamF2YQ==) | `88.88% <0.00%> (-1.12%)` | `0.00% <0.00%> (ø%)` | |
   | [...gjdbc/jdbc/core/connection/ShardingConnection.java](https://codecov.io/gh/apache/shardingsphere/pull/5363/diff?src=pr&el=tree#diff-c2hhcmRpbmctamRiYy9zaGFyZGluZy1qZGJjLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5namRiYy9qZGJjL2NvcmUvY29ubmVjdGlvbi9TaGFyZGluZ0Nvbm5lY3Rpb24uamF2YQ==) | `77.50% <0.00%> (-1.08%)` | `0.00% <0.00%> (ø%)` | |
   | [...re/shardingjdbc/api/ShardingDataSourceFactory.java](https://codecov.io/gh/apache/shardingsphere/pull/5363/diff?src=pr&el=tree#diff-c2hhcmRpbmctamRiYy9zaGFyZGluZy1qZGJjLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5namRiYy9hcGkvU2hhcmRpbmdEYXRhU291cmNlRmFjdG9yeS5qYXZh) | `100.00% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | [...re/shardingscaling/core/config/ScalingContext.java](https://codecov.io/gh/apache/shardingsphere/pull/5363/diff?src=pr&el=tree#diff-c2hhcmRpbmctc2NhbGluZy9zaGFyZGluZy1zY2FsaW5nLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5nc2NhbGluZy9jb3JlL2NvbmZpZy9TY2FsaW5nQ29udGV4dC5qYXZh) | `100.00% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | [...ingjdbc/jdbc/core/datasource/ShadowDataSource.java](https://codecov.io/gh/apache/shardingsphere/pull/5363/diff?src=pr&el=tree#diff-c2hhcmRpbmctamRiYy9zaGFyZGluZy1qZGJjLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5namRiYy9qZGJjL2NvcmUvZGF0YXNvdXJjZS9TaGFkb3dEYXRhU291cmNlLmphdmE=) | `100.00% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | [...bc/jdbc/core/datasource/MasterSlaveDataSource.java](https://codecov.io/gh/apache/shardingsphere/pull/5363/diff?src=pr&el=tree#diff-c2hhcmRpbmctamRiYy9zaGFyZGluZy1qZGJjLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5namRiYy9qZGJjL2NvcmUvZGF0YXNvdXJjZS9NYXN0ZXJTbGF2ZURhdGFTb3VyY2UuamF2YQ==) | `100.00% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | [...e/datasource/metadata/AdaptedDatabaseMetaData.java](https://codecov.io/gh/apache/shardingsphere/pull/5363/diff?src=pr&el=tree#diff-c2hhcmRpbmctamRiYy9zaGFyZGluZy1qZGJjLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5namRiYy9qZGJjL2NvcmUvZGF0YXNvdXJjZS9tZXRhZGF0YS9BZGFwdGVkRGF0YWJhc2VNZXRhRGF0YS5qYXZh) | `99.31% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | ... and [43 more](https://codecov.io/gh/apache/shardingsphere/pull/5363/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/shardingsphere/pull/5363?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/5363?src=pr&el=footer). Last update [9fcf07e...97e416c](https://codecov.io/gh/apache/shardingsphere/pull/5363?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] coveralls edited a comment on pull request #5363: a custom datetime shard algorithm used to shard tables by days/weeks/months/QUARTER_OF_YEAR

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #5363:
URL: https://github.com/apache/shardingsphere/pull/5363#issuecomment-620433590


   ## Pull Request Test Coverage Report for [Build 11338](https://coveralls.io/builds/30399510)
   
   * **52** of **59**   **(88.14%)**  changed or added relevant lines in **1** file are covered.
   * **4** unchanged lines in **3** files lost coverage.
   * Overall coverage increased (+**0.07%**) to **57.481%**
   
   ---
   
   |  Changes Missing Coverage | Covered Lines | Changed/Added Lines | % |
   | :-----|--------------|--------|---: |
   | [sharding-core/sharding-core-common/src/main/java/org/apache/shardingsphere/core/strategy/algorithm/sharding/CustomDateTimeShardingAlgorithm.java](https://coveralls.io/builds/30399510/source?filename=sharding-core%2Fsharding-core-common%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Fcore%2Fstrategy%2Falgorithm%2Fsharding%2FCustomDateTimeShardingAlgorithm.java#L96) | 52 | 59 | 88.14%
   <!-- | **Total:** | **52** | **59** | **88.14%** | -->
   
   |  Files with Coverage Reduction | New Missed Lines | % |
   | :-----|--------------|--: |
   | [sharding-jdbc/sharding-jdbc-core/src/main/java/org/apache/shardingsphere/shardingjdbc/jdbc/adapter/AbstractDataSourceAdapter.java](https://coveralls.io/builds/30399510/source?filename=sharding-jdbc%2Fsharding-jdbc-core%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Fshardingjdbc%2Fjdbc%2Fadapter%2FAbstractDataSourceAdapter.java#L120) | 1 | 97.44% |
   | [sharding-jdbc/sharding-jdbc-core/src/main/java/org/apache/shardingsphere/shardingjdbc/jdbc/core/datasource/EncryptDataSource.java](https://coveralls.io/builds/30399510/source?filename=sharding-jdbc%2Fsharding-jdbc-core%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Fshardingjdbc%2Fjdbc%2Fcore%2Fdatasource%2FEncryptDataSource.java#L32) | 1 | 80.0% |
   | [sharding-jdbc/sharding-jdbc-core/src/main/java/org/apache/shardingsphere/shardingjdbc/jdbc/core/context/RuntimeContext.java](https://coveralls.io/builds/30399510/source?filename=sharding-jdbc%2Fsharding-jdbc-core%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Fshardingjdbc%2Fjdbc%2Fcore%2Fcontext%2FRuntimeContext.java#L94) | 2 | 94.74% |
   <!-- | **Total:** | **4** |  | -->
   
   |  Totals | [![Coverage Status](https://coveralls.io/builds/30399510/badge)](https://coveralls.io/builds/30399510) |
   | :-- | --: |
   | Change from base [Build 11327](https://coveralls.io/builds/30391678): |  0.07% |
   | Covered Lines: | 11932 |
   | Relevant Lines: | 20758 |
   
   ---
   ##### 💛  - [Coveralls](https://coveralls.io)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org