Skip to content

Commit

Permalink
Refactor DateTimeFormatterFactory (#29205)
Browse files Browse the repository at this point in the history
  • Loading branch information
terrymanu authored Nov 25, 2023
1 parent 139f6db commit 14cdd1f
Show file tree
Hide file tree
Showing 19 changed files with 186 additions and 147 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,10 @@
import org.apache.shardingsphere.db.protocol.mysql.packet.binlog.row.column.MySQLBinlogColumnDef;
import org.apache.shardingsphere.db.protocol.mysql.packet.binlog.row.column.value.MySQLBinlogProtocolValue;
import org.apache.shardingsphere.db.protocol.mysql.payload.MySQLPacketPayload;
import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory;

import java.io.Serializable;
import java.sql.Timestamp;
import java.time.format.DateTimeFormatter;

/**
* MySQL TIMESTAMP binlog protocol value.
Expand All @@ -33,11 +33,9 @@
*/
public final class MySQLTimestampBinlogProtocolValue implements MySQLBinlogProtocolValue {

private final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss");

@Override
public Serializable read(final MySQLBinlogColumnDef columnDef, final MySQLPacketPayload payload) {
int seconds = payload.readInt4();
return 0 == seconds ? MySQLTimeValueUtils.DATETIME_OF_ZERO : dateTimeFormatter.format(new Timestamp(seconds * 1000L).toLocalDateTime());
return 0 == seconds ? MySQLTimeValueUtils.DATETIME_OF_ZERO : DateTimeFormatterFactory.getStandardFormatter().format(new Timestamp(seconds * 1000L).toLocalDateTime());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.db.protocol.mysql.packet.MySQLPacket;
import org.apache.shardingsphere.db.protocol.mysql.payload.MySQLPacketPayload;
import org.apache.shardingsphere.infra.util.datetime.StandardDateTimeFormatter;
import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory;

import java.math.BigDecimal;
import java.sql.Timestamp;
Expand Down Expand Up @@ -70,7 +70,7 @@ private void writeDataIntoPayload(final MySQLPacketPayload payload, final Object
} else if (data instanceof Boolean) {
payload.writeBytesLenenc((boolean) data ? new byte[]{1} : new byte[]{0});
} else if (data instanceof LocalDateTime) {
payload.writeStringLenenc(StandardDateTimeFormatter.get().format((LocalDateTime) data));
payload.writeStringLenenc(DateTimeFormatterFactory.getStandardFormatter().format((LocalDateTime) data));
} else {
payload.writeStringLenenc(data.toString());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,13 @@

import org.apache.shardingsphere.db.protocol.mysql.packet.binlog.row.column.MySQLBinlogColumnDef;
import org.apache.shardingsphere.db.protocol.mysql.payload.MySQLPacketPayload;
import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;

import java.sql.Timestamp;
import java.time.format.DateTimeFormatter;

import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
Expand All @@ -45,7 +45,7 @@ void assertRead() {
int currentSeconds = Long.valueOf(System.currentTimeMillis() / 1000L).intValue();
when(payload.readInt4()).thenReturn(currentSeconds);
assertThat(new MySQLTimestampBinlogProtocolValue().read(columnDef, payload),
is(DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss").format(new Timestamp(currentSeconds * 1000L).toLocalDateTime())));
is(DateTimeFormatterFactory.getStandardFormatter().format(new Timestamp(currentSeconds * 1000L).toLocalDateTime())));
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import com.google.common.collect.Range;
import lombok.Getter;
import org.apache.shardingsphere.infra.exception.core.ShardingSpherePreconditions;
import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory;
import org.apache.shardingsphere.sharding.algorithm.sharding.ShardingAutoTableAlgorithmUtils;
import org.apache.shardingsphere.sharding.api.sharding.ShardingAutoTableAlgorithm;
import org.apache.shardingsphere.sharding.api.sharding.standard.PreciseShardingValue;
Expand All @@ -33,7 +34,6 @@
import java.text.ParsePosition;
import java.time.Duration;
import java.time.LocalDateTime;
import java.time.format.DateTimeFormatter;
import java.time.format.DateTimeParseException;
import java.util.Collection;
import java.util.LinkedHashSet;
Expand All @@ -50,8 +50,6 @@ public final class AutoIntervalShardingAlgorithm implements StandardShardingAlgo

private static final String SHARDING_SECONDS_KEY = "sharding-seconds";

private static final DateTimeFormatter DATE_TIME_FORMAT = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss");

private LocalDateTime dateTimeLower;

private long shardingSeconds;
Expand All @@ -70,7 +68,7 @@ private LocalDateTime getDateTime(final Properties props) {
String value = props.getProperty(DATE_TIME_LOWER_KEY);
ShardingSpherePreconditions.checkNotNull(value, () -> new ShardingAlgorithmInitializationException(getType(), String.format("%s cannot be null.", DATE_TIME_LOWER_KEY)));
try {
return LocalDateTime.parse(value, DATE_TIME_FORMAT);
return LocalDateTime.parse(value, DateTimeFormatterFactory.getStandardFormatter());
} catch (final DateTimeParseException ignored) {
throw new InvalidDatetimeFormatException(DATE_TIME_LOWER_KEY, value, "yyyy-MM-dd HH:mm:ss");
}
Expand Down Expand Up @@ -115,7 +113,7 @@ private int getLastPartition(final Range<Comparable<?>> valueRange) {
}

private long parseDate(final Comparable<?> shardingValue) {
LocalDateTime dateValue = LocalDateTime.from(DATE_TIME_FORMAT.parse(shardingValue.toString(), new ParsePosition(0)));
LocalDateTime dateValue = LocalDateTime.from(DateTimeFormatterFactory.getStandardFormatter().parse(shardingValue.toString(), new ParsePosition(0)));
return Duration.between(dateTimeLower, dateValue).toMillis() / 1000;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import com.google.common.collect.Range;
import org.apache.shardingsphere.infra.datanode.DataNodeInfo;
import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory;
import org.apache.shardingsphere.sharding.api.sharding.standard.PreciseShardingValue;
import org.apache.shardingsphere.sharding.api.sharding.standard.RangeShardingValue;
import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
Expand Down Expand Up @@ -361,7 +362,7 @@ void assertTimestampInJDBCTypeWithZeroMillisecond() {
ZonedDateTime.of(2021, 6, 15, 2, 25, 27, 0, ZoneId.systemDefault()),
ZonedDateTime.of(2021, 7, 31, 2, 25, 27, 0, ZoneId.systemDefault()))));
assertThat(actualAsZonedDateTime.size(), is(24));
DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS");
DateTimeFormatter dateTimeFormatter = DateTimeFormatterFactory.getLongMillsFormatter();
Collection<String> actualAsDate = shardingAlgorithmByDayWithMillisecond.doSharding(availableTablesForDayWithMillisecondDataSources,
new RangeShardingValue<>("t_order", "create_time", DATA_NODE_INFO,
Range.closed(Date.from(LocalDate.from(dateTimeFormatter.parse("2021-06-15 02:25:27.000")).atStartOfDay().atZone(ZoneId.systemDefault()).toInstant()),
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
/*
* 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.infra.util.datetime;

import lombok.AccessLevel;
import lombok.NoArgsConstructor;

import java.time.format.DateTimeFormatter;

/**
* Date time formatter factory.
*/
@NoArgsConstructor(access = AccessLevel.PRIVATE)
public final class DateTimeFormatterFactory {

private static final DateTimeFormatter STANDARD = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss");

private static final DateTimeFormatter DATE = DateTimeFormatter.ofPattern("yyyy-MM-dd");

private static final DateTimeFormatter TIME = DateTimeFormatter.ofPattern("HH:mm:ss");

private static final DateTimeFormatter SHORT_MILLS = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.S");

private static final DateTimeFormatter LONG_MILLS = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS");

/**
* Get standard date time formatter.
*
* @return standard date time formatter
*/
public static DateTimeFormatter getStandardFormatter() {
return STANDARD;
}

/**
* Get date formatter.
*
* @return date formatter
*/
public static DateTimeFormatter getDateFormatter() {
return DATE;
}

/**
* Get time formatter.
*
* @return time formatter
*/
public static DateTimeFormatter getTimeFormatter() {
return TIME;
}

/**
* Get short mills date time formatter.
*
* @return short mills date time formatter
*/
public static DateTimeFormatter getShortMillsFormatter() {
return SHORT_MILLS;
}

/**
* Get long mills date time formatter.
*
* @return long mills date time formatter
*/
public static DateTimeFormatter getLongMillsFormatter() {
return LONG_MILLS;
}
}

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.shardingsphere.infra.util.datetime;

import org.junit.jupiter.api.Test;

import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;

class DateTimeFormatterFactoryTest {

@Test
void assertGetStandardFormatter() {
assertThat(DateTimeFormatterFactory.getStandardFormatter().parse("1970-01-01 00:00:00").toString(), is("{},ISO resolved to 1970-01-01T00:00"));
}

@Test
void assertGetDateFormatter() {
assertThat(DateTimeFormatterFactory.getDateFormatter().parse("1970-01-01").toString(), is("{},ISO resolved to 1970-01-01"));
}

@Test
void assertGetTimeFormatter() {
assertThat(DateTimeFormatterFactory.getTimeFormatter().parse("00:00:00").toString(), is("{},ISO resolved to 00:00"));
}

@Test
void assertGetShortMillsFormatter() {
assertThat(DateTimeFormatterFactory.getShortMillsFormatter().parse("1970-01-01 00:00:00.0").toString(), is("{},ISO resolved to 1970-01-01T00:00"));
}

@Test
void assertGetLongMillsFormatter() {
assertThat(DateTimeFormatterFactory.getLongMillsFormatter().parse("1970-01-01 00:00:00.000").toString(), is("{},ISO resolved to 1970-01-01T00:00"));
}
}

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.shardingsphere.data.pipeline.common.execute;

import lombok.extern.slf4j.Slf4j;
import org.apache.shardingsphere.infra.util.datetime.StandardDateTimeFormatter;
import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory;

import java.sql.SQLException;
import java.time.Instant;
Expand Down Expand Up @@ -63,7 +63,7 @@ public final void stop() {
return;
}
LocalDateTime startTime = LocalDateTime.ofInstant(Instant.ofEpochMilli(startTimeMillis), ZoneId.systemDefault());
log.info("stop lifecycle executor {}, startTime={}, cost {} ms", this, startTime.format(StandardDateTimeFormatter.get()), System.currentTimeMillis() - startTimeMillis);
log.info("stop lifecycle executor {}, startTime={}, cost {} ms", this, startTime.format(DateTimeFormatterFactory.getStandardFormatter()), System.currentTimeMillis() - startTimeMillis);
try {
doStop();
// CHECKSTYLE:OFF
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
import org.apache.shardingsphere.data.pipeline.core.job.PipelineJobIdUtils;
import org.apache.shardingsphere.data.pipeline.core.job.option.PipelineJobOption;
import org.apache.shardingsphere.elasticjob.infra.pojo.JobConfigurationPOJO;
import org.apache.shardingsphere.infra.util.datetime.StandardDateTimeFormatter;
import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory;
import org.apache.shardingsphere.infra.util.yaml.YamlEngine;

import java.time.LocalDateTime;
Expand Down Expand Up @@ -61,7 +61,7 @@ public JobConfigurationPOJO convertToJobConfigurationPOJO(final PipelineJobConfi
int shardingTotalCount = jobOption.isForceNoShardingWhenConvertToJobConfigurationPOJO() ? 1 : jobConfig.getJobShardingCount();
result.setShardingTotalCount(shardingTotalCount);
result.setJobParameter(YamlEngine.marshal(jobOption.getYamlJobConfigurationSwapper().swapToYamlConfiguration(jobConfig)));
String createTimeFormat = LocalDateTime.now().format(StandardDateTimeFormatter.get());
String createTimeFormat = LocalDateTime.now().format(DateTimeFormatterFactory.getStandardFormatter());
result.getProps().setProperty("create_time", createTimeFormat);
result.getProps().setProperty("start_time_millis", String.valueOf(System.currentTimeMillis()));
result.getProps().setProperty("run_count", "1");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@
import org.apache.shardingsphere.elasticjob.infra.pojo.JobConfigurationPOJO;
import org.apache.shardingsphere.infra.exception.core.ShardingSpherePreconditions;
import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
import org.apache.shardingsphere.infra.util.datetime.StandardDateTimeFormatter;
import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory;

import java.time.LocalDateTime;
import java.util.Collections;
Expand Down Expand Up @@ -148,7 +148,7 @@ private void stopCurrentJob(final String jobId) {
return;
}
jobConfigPOJO.setDisabled(true);
jobConfigPOJO.getProps().setProperty("stop_time", LocalDateTime.now().format(StandardDateTimeFormatter.get()));
jobConfigPOJO.getProps().setProperty("stop_time", LocalDateTime.now().format(DateTimeFormatterFactory.getStandardFormatter()));
jobConfigPOJO.getProps().setProperty("stop_time_millis", String.valueOf(System.currentTimeMillis()));
String barrierPath = PipelineMetaDataNode.getJobBarrierDisablePath(jobId);
pipelineDistributedBarrier.register(barrierPath, jobConfigPOJO.getShardingTotalCount());
Expand Down
Loading

0 comments on commit 14cdd1f

Please sign in to comment.