From 758d7fd34bd834dda5bf03a25a011497d15cf210 Mon Sep 17 00:00:00 2001 From: zhangliang Date: Sat, 25 Nov 2023 19:43:02 +0800 Subject: [PATCH] Refactor DateTimeFormatterFactory --- .../MySQLTimestampBinlogProtocolValue.java | 6 +- .../text/MySQLTextResultSetRowPacket.java | 4 +- ...MySQLTimestampBinlogProtocolValueTest.java | 4 +- .../AutoIntervalShardingAlgorithm.java | 8 +- .../IntervalShardingAlgorithmTest.java | 3 +- .../datetime/DateTimeFormatterFactory.java | 85 +++++++++++++++++++ .../datetime/StandardDateTimeFormatter.java | 41 --------- .../DateTimeFormatterFactoryTest.java | 51 +++++++++++ .../StandardDateTimeFormatterTest.java | 31 ------- .../AbstractPipelineLifecycleRunnable.java | 4 +- .../PipelineJobConfigurationManager.java | 4 +- .../core/job/service/PipelineJobManager.java | 4 +- .../pipeline/cdc/api/impl/CDCJobOption.java | 4 +- .../api/impl/ConsistencyCheckJobOption.java | 10 +-- .../test/e2e/cases/value/SQLValue.java | 26 ++---- .../composer/BatchE2EContainerComposer.java | 10 +-- .../test/e2e/engine/type/DALE2EIT.java | 8 +- .../e2e/engine/type/dml/BaseDMLE2EIT.java | 16 ++-- .../e2e/engine/type/dql/BaseDQLE2EIT.java | 14 +-- 19 files changed, 186 insertions(+), 147 deletions(-) create mode 100644 infra/util/src/main/java/org/apache/shardingsphere/infra/util/datetime/DateTimeFormatterFactory.java delete mode 100644 infra/util/src/main/java/org/apache/shardingsphere/infra/util/datetime/StandardDateTimeFormatter.java create mode 100644 infra/util/src/test/java/org/apache/shardingsphere/infra/util/datetime/DateTimeFormatterFactoryTest.java delete mode 100644 infra/util/src/test/java/org/apache/shardingsphere/infra/util/datetime/StandardDateTimeFormatterTest.java diff --git a/db-protocol/mysql/src/main/java/org/apache/shardingsphere/db/protocol/mysql/packet/binlog/row/column/value/time/MySQLTimestampBinlogProtocolValue.java b/db-protocol/mysql/src/main/java/org/apache/shardingsphere/db/protocol/mysql/packet/binlog/row/column/value/time/MySQLTimestampBinlogProtocolValue.java index 1bd4f1d9bbaf2..9300dc946dd18 100644 --- a/db-protocol/mysql/src/main/java/org/apache/shardingsphere/db/protocol/mysql/packet/binlog/row/column/value/time/MySQLTimestampBinlogProtocolValue.java +++ b/db-protocol/mysql/src/main/java/org/apache/shardingsphere/db/protocol/mysql/packet/binlog/row/column/value/time/MySQLTimestampBinlogProtocolValue.java @@ -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. @@ -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()); } } diff --git a/db-protocol/mysql/src/main/java/org/apache/shardingsphere/db/protocol/mysql/packet/command/query/text/MySQLTextResultSetRowPacket.java b/db-protocol/mysql/src/main/java/org/apache/shardingsphere/db/protocol/mysql/packet/command/query/text/MySQLTextResultSetRowPacket.java index 81fca057930a3..b4eabc9b3f258 100644 --- a/db-protocol/mysql/src/main/java/org/apache/shardingsphere/db/protocol/mysql/packet/command/query/text/MySQLTextResultSetRowPacket.java +++ b/db-protocol/mysql/src/main/java/org/apache/shardingsphere/db/protocol/mysql/packet/command/query/text/MySQLTextResultSetRowPacket.java @@ -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; @@ -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()); } diff --git a/db-protocol/mysql/src/test/java/org/apache/shardingsphere/db/protocol/mysql/packet/binlog/row/column/value/time/MySQLTimestampBinlogProtocolValueTest.java b/db-protocol/mysql/src/test/java/org/apache/shardingsphere/db/protocol/mysql/packet/binlog/row/column/value/time/MySQLTimestampBinlogProtocolValueTest.java index ceba78e9d6631..6f128c8d291ab 100644 --- a/db-protocol/mysql/src/test/java/org/apache/shardingsphere/db/protocol/mysql/packet/binlog/row/column/value/time/MySQLTimestampBinlogProtocolValueTest.java +++ b/db-protocol/mysql/src/test/java/org/apache/shardingsphere/db/protocol/mysql/packet/binlog/row/column/value/time/MySQLTimestampBinlogProtocolValueTest.java @@ -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; @@ -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 diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/AutoIntervalShardingAlgorithm.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/AutoIntervalShardingAlgorithm.java index 6db951fff5330..9eaa07287b0e9 100644 --- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/AutoIntervalShardingAlgorithm.java +++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/AutoIntervalShardingAlgorithm.java @@ -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; @@ -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; @@ -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; @@ -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"); } @@ -115,7 +113,7 @@ private int getLastPartition(final Range> 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; } diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/IntervalShardingAlgorithmTest.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/IntervalShardingAlgorithmTest.java index 585f37d930958..1c3a53adeade3 100644 --- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/IntervalShardingAlgorithmTest.java +++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/sharding/datetime/IntervalShardingAlgorithmTest.java @@ -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; @@ -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 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()), diff --git a/infra/util/src/main/java/org/apache/shardingsphere/infra/util/datetime/DateTimeFormatterFactory.java b/infra/util/src/main/java/org/apache/shardingsphere/infra/util/datetime/DateTimeFormatterFactory.java new file mode 100644 index 0000000000000..b61eb72ae2225 --- /dev/null +++ b/infra/util/src/main/java/org/apache/shardingsphere/infra/util/datetime/DateTimeFormatterFactory.java @@ -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; + } +} diff --git a/infra/util/src/main/java/org/apache/shardingsphere/infra/util/datetime/StandardDateTimeFormatter.java b/infra/util/src/main/java/org/apache/shardingsphere/infra/util/datetime/StandardDateTimeFormatter.java deleted file mode 100644 index a1119fca67831..0000000000000 --- a/infra/util/src/main/java/org/apache/shardingsphere/infra/util/datetime/StandardDateTimeFormatter.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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; - -/** - * Standard date time formatter. - */ -@NoArgsConstructor(access = AccessLevel.PRIVATE) -public final class StandardDateTimeFormatter { - - private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); - - /** - * Get standard date time formatter. - * - * @return standard date time formatter - */ - public static DateTimeFormatter get() { - return DATE_TIME_FORMATTER; - } -} diff --git a/infra/util/src/test/java/org/apache/shardingsphere/infra/util/datetime/DateTimeFormatterFactoryTest.java b/infra/util/src/test/java/org/apache/shardingsphere/infra/util/datetime/DateTimeFormatterFactoryTest.java new file mode 100644 index 0000000000000..af926f8c52d92 --- /dev/null +++ b/infra/util/src/test/java/org/apache/shardingsphere/infra/util/datetime/DateTimeFormatterFactoryTest.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.shardingsphere.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")); + } +} diff --git a/infra/util/src/test/java/org/apache/shardingsphere/infra/util/datetime/StandardDateTimeFormatterTest.java b/infra/util/src/test/java/org/apache/shardingsphere/infra/util/datetime/StandardDateTimeFormatterTest.java deleted file mode 100644 index d4730283978a7..0000000000000 --- a/infra/util/src/test/java/org/apache/shardingsphere/infra/util/datetime/StandardDateTimeFormatterTest.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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 StandardDateTimeFormatterTest { - - @Test - void assertGet() { - assertThat(StandardDateTimeFormatter.get().parse("1970-01-01 00:00:00").toString(), is("{},ISO resolved to 1970-01-01T00:00")); - } -} diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/common/execute/AbstractPipelineLifecycleRunnable.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/common/execute/AbstractPipelineLifecycleRunnable.java index 2b5d85fd5946b..0d45d12b40b97 100644 --- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/common/execute/AbstractPipelineLifecycleRunnable.java +++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/common/execute/AbstractPipelineLifecycleRunnable.java @@ -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; @@ -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 diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/job/service/PipelineJobConfigurationManager.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/job/service/PipelineJobConfigurationManager.java index cf33f799c712d..c02dc1355a894 100644 --- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/job/service/PipelineJobConfigurationManager.java +++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/job/service/PipelineJobConfigurationManager.java @@ -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; @@ -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"); diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/job/service/PipelineJobManager.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/job/service/PipelineJobManager.java index 39968ff66909a..717125ea11415 100644 --- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/job/service/PipelineJobManager.java +++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/job/service/PipelineJobManager.java @@ -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; @@ -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()); diff --git a/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/api/impl/CDCJobOption.java b/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/api/impl/CDCJobOption.java index af0c219b7ef7d..48f6bf3bb61b2 100644 --- a/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/api/impl/CDCJobOption.java +++ b/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/api/impl/CDCJobOption.java @@ -79,7 +79,7 @@ import org.apache.shardingsphere.infra.instance.metadata.InstanceType; import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase; import org.apache.shardingsphere.infra.metadata.database.resource.unit.StorageUnit; -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 org.apache.shardingsphere.infra.yaml.config.pojo.YamlRootConfiguration; import org.apache.shardingsphere.infra.yaml.config.swapper.resource.YamlDataSourceConfigurationSwapper; @@ -224,7 +224,7 @@ public void updateJobConfigurationDisabled(final String jobId, final boolean dis JobConfigurationPOJO jobConfigPOJO = PipelineJobIdUtils.getElasticJobConfigurationPOJO(jobId); jobConfigPOJO.setDisabled(disabled); if (disabled) { - 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())); } else { jobConfigPOJO.getProps().setProperty("start_time_millis", String.valueOf(System.currentTimeMillis())); diff --git a/kernel/data-pipeline/scenario/consistencycheck/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/api/impl/ConsistencyCheckJobOption.java b/kernel/data-pipeline/scenario/consistencycheck/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/api/impl/ConsistencyCheckJobOption.java index afd2c577040c0..c793a22bd500f 100644 --- a/kernel/data-pipeline/scenario/consistencycheck/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/api/impl/ConsistencyCheckJobOption.java +++ b/kernel/data-pipeline/scenario/consistencycheck/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/api/impl/ConsistencyCheckJobOption.java @@ -46,11 +46,11 @@ import org.apache.shardingsphere.elasticjob.infra.pojo.JobConfigurationPOJO; import org.apache.shardingsphere.infra.database.core.type.DatabaseType; import org.apache.shardingsphere.infra.exception.core.ShardingSpherePreconditions; +import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory; import java.sql.Timestamp; import java.time.Duration; import java.time.LocalDateTime; -import java.time.format.DateTimeFormatter; import java.util.Collection; import java.util.Collections; import java.util.LinkedList; @@ -68,8 +68,6 @@ @Slf4j public final class ConsistencyCheckJobOption implements PipelineJobOption { - private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS"); - /** * Create consistency check configuration and start job. * @@ -244,13 +242,13 @@ private void fillInJobItemInfoWithTimes(final ConsistencyCheckJobItemInfo result long recordsCount = jobItemProgress.getRecordsCount(); long checkedRecordsCount = Math.min(jobItemProgress.getCheckedRecordsCount(), recordsCount); LocalDateTime checkBeginTime = new Timestamp(jobItemProgress.getCheckBeginTimeMillis()).toLocalDateTime(); - result.setCheckBeginTime(DATE_TIME_FORMATTER.format(checkBeginTime)); + result.setCheckBeginTime(DateTimeFormatterFactory.getLongMillsFormatter().format(checkBeginTime)); if (JobStatus.FINISHED == jobItemProgress.getStatus()) { result.setInventoryFinishedPercentage(100); LocalDateTime checkEndTime = new Timestamp(jobItemProgress.getCheckEndTimeMillis()).toLocalDateTime(); Duration duration = Duration.between(checkBeginTime, checkEndTime); result.setDurationSeconds(duration.getSeconds()); - result.setCheckEndTime(DATE_TIME_FORMATTER.format(checkEndTime)); + result.setCheckEndTime(DateTimeFormatterFactory.getLongMillsFormatter().format(checkEndTime)); result.setInventoryRemainingSeconds(0L); } else if (0 != recordsCount && 0 != checkedRecordsCount) { result.setInventoryFinishedPercentage((int) (checkedRecordsCount * 100 / recordsCount)); @@ -258,7 +256,7 @@ private void fillInJobItemInfoWithTimes(final ConsistencyCheckJobItemInfo result long durationMillis = (null != stopTimeMillis ? stopTimeMillis : System.currentTimeMillis()) - jobItemProgress.getCheckBeginTimeMillis(); result.setDurationSeconds(TimeUnit.MILLISECONDS.toSeconds(durationMillis)); if (null != stopTimeMillis) { - result.setCheckEndTime(DATE_TIME_FORMATTER.format(new Timestamp(stopTimeMillis).toLocalDateTime())); + result.setCheckEndTime(DateTimeFormatterFactory.getLongMillsFormatter().format(new Timestamp(stopTimeMillis).toLocalDateTime())); } long remainingMills = Math.max(0, (long) ((recordsCount - checkedRecordsCount) * 1.0D / checkedRecordsCount * durationMillis)); result.setInventoryRemainingSeconds(remainingMills / 1000); diff --git a/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/cases/value/SQLValue.java b/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/cases/value/SQLValue.java index 667f11e32ceb0..f98c4e4fedbe2 100644 --- a/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/cases/value/SQLValue.java +++ b/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/cases/value/SQLValue.java @@ -19,6 +19,7 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; +import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; @@ -28,7 +29,6 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; -import java.time.format.DateTimeFormatter; /** * SQL value. @@ -42,14 +42,6 @@ public final class SQLValue { @Getter private final int index; - private final DateTimeFormatter dateFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); - - private final DateTimeFormatter timeFormatter = DateTimeFormatter.ofPattern("HH:mm:ss"); - - private final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); - - private final DateTimeFormatter timestampFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.S"); - public SQLValue(final String value, final String type, final int index) { this.value = null == type ? value : getValue(value, type); this.index = index; @@ -86,16 +78,16 @@ private Object getValue(final String value, final String type) { case "boolean": return Boolean.parseBoolean(value); case "Date": - return Date.valueOf(LocalDate.parse(value, dateFormatter)); + return Date.valueOf(LocalDate.parse(value, DateTimeFormatterFactory.getDateFormatter())); case "datetime": if (10 == value.length()) { - return Date.valueOf(LocalDate.parse(value, dateFormatter)); + return Date.valueOf(LocalDate.parse(value, DateTimeFormatterFactory.getDateFormatter())); } - return Date.valueOf(LocalDate.parse(value, dateTimeFormatter)); + return Date.valueOf(LocalDate.parse(value, DateTimeFormatterFactory.getStandardFormatter())); case "time": - return Time.valueOf(LocalTime.parse(value, timeFormatter)); + return Time.valueOf(LocalTime.parse(value, DateTimeFormatterFactory.getTimeFormatter())); case "timestamp": - return Timestamp.valueOf(LocalDateTime.parse(value, timestampFormatter)); + return Timestamp.valueOf(LocalDateTime.parse(value, DateTimeFormatterFactory.getShortMillsFormatter())); case "bytes": return value.getBytes(StandardCharsets.UTF_8); default: @@ -109,13 +101,13 @@ public String toString() { return formatString((String) value); } if (value instanceof Date) { - return formatString(dateFormatter.format(((Date) value).toLocalDate())); + return formatString(DateTimeFormatterFactory.getDateFormatter().format(((Date) value).toLocalDate())); } if (value instanceof Time) { - return formatString(timeFormatter.format(((Time) value).toLocalTime())); + return formatString(DateTimeFormatterFactory.getTimeFormatter().format(((Time) value).toLocalTime())); } if (value instanceof Timestamp) { - return formatString(timestampFormatter.format(((Timestamp) value).toLocalDateTime())); + return formatString(DateTimeFormatterFactory.getShortMillsFormatter().format(((Timestamp) value).toLocalDateTime())); } if (value instanceof byte[]) { return formatString(new String((byte[]) value, StandardCharsets.UTF_8)); diff --git a/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/composer/BatchE2EContainerComposer.java b/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/composer/BatchE2EContainerComposer.java index df6e7922d19e7..a8f0e7cbac37c 100644 --- a/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/composer/BatchE2EContainerComposer.java +++ b/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/composer/BatchE2EContainerComposer.java @@ -20,6 +20,7 @@ import org.apache.shardingsphere.infra.database.core.type.DatabaseType; import org.apache.shardingsphere.infra.datanode.DataNode; import org.apache.shardingsphere.infra.expr.core.InlineExpressionParserFactory; +import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory; import org.apache.shardingsphere.test.e2e.cases.assertion.IntegrationTestCaseAssertion; import org.apache.shardingsphere.test.e2e.cases.dataset.DataSet; import org.apache.shardingsphere.test.e2e.cases.dataset.DataSetLoader; @@ -41,7 +42,6 @@ import java.sql.ResultSetMetaData; import java.sql.SQLException; import java.sql.Types; -import java.time.format.DateTimeFormatter; import java.util.Collection; import java.util.Comparator; import java.util.HashSet; @@ -63,10 +63,6 @@ public final class BatchE2EContainerComposer extends E2EContainerComposer implem private final DataSetEnvironmentManager dataSetEnvironmentManager; - private final DateTimeFormatter dateFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); - - private final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.S"); - public BatchE2EContainerComposer(final CaseTestParameter testParam) throws JAXBException, IOException { super(testParam); databaseType = testParam.getDatabaseType(); @@ -161,10 +157,10 @@ private void assertRows(final ResultSet actual, final List expectedD for (String expected : expectedDatSetRows.get(count).splitValues(", ")) { if (Types.DATE == actual.getMetaData().getColumnType(columnIndex)) { if (!E2EContainerComposer.NOT_VERIFY_FLAG.equals(expected)) { - assertThat(dateFormatter.format(actual.getDate(columnIndex).toLocalDate()), is(expected)); + assertThat(DateTimeFormatterFactory.getDateFormatter().format(actual.getDate(columnIndex).toLocalDate()), is(expected)); } } else if (Types.TIMESTAMP == actual.getMetaData().getColumnType(columnIndex)) { - assertThat(actual.getTimestamp(columnIndex).toLocalDateTime().format(dateTimeFormatter), is(expected)); + assertThat(actual.getTimestamp(columnIndex).toLocalDateTime().format(DateTimeFormatterFactory.getShortMillsFormatter()), is(expected)); } else if (Types.CHAR == actual.getMetaData().getColumnType(columnIndex) && ("PostgreSQL".equals(databaseType.getType()) || "openGauss".equals(databaseType.getType()))) { assertThat(String.valueOf(actual.getObject(columnIndex)).trim(), is(expected)); diff --git a/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/type/DALE2EIT.java b/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/type/DALE2EIT.java index 9aad8134fe381..35f4df29e3d60 100644 --- a/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/type/DALE2EIT.java +++ b/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/type/DALE2EIT.java @@ -17,6 +17,7 @@ package org.apache.shardingsphere.test.e2e.engine.type; +import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory; import org.apache.shardingsphere.test.e2e.cases.SQLCommandType; import org.apache.shardingsphere.test.e2e.cases.dataset.metadata.DataSetColumn; import org.apache.shardingsphere.test.e2e.cases.dataset.metadata.DataSetMetaData; @@ -37,7 +38,6 @@ import java.sql.SQLException; import java.sql.Statement; import java.sql.Types; -import java.time.format.DateTimeFormatter; import java.util.Collection; import java.util.LinkedList; import java.util.List; @@ -49,8 +49,6 @@ @E2ETestCaseSettings(SQLCommandType.DAL) class DALE2EIT { - private final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); - @ParameterizedTest(name = "{0}") @EnabledIf("isEnabled") @ArgumentsSource(E2ETestCaseArgumentsProvider.class) @@ -134,8 +132,8 @@ private void assertDateValue(final ResultSet actual, final int columnIndex, fina if (E2EContainerComposer.NOT_VERIFY_FLAG.equals(expected)) { return; } - assertThat(dateTimeFormatter.format(actual.getDate(columnIndex).toLocalDate()), is(expected)); - assertThat(dateTimeFormatter.format(actual.getDate(columnLabel).toLocalDate()), is(expected)); + assertThat(DateTimeFormatterFactory.getTimeFormatter().format(actual.getDate(columnIndex).toLocalDate()), is(expected)); + assertThat(DateTimeFormatterFactory.getTimeFormatter().format(actual.getDate(columnLabel).toLocalDate()), is(expected)); } private void assertObjectValue(final ResultSet actual, final int columnIndex, final String columnLabel, final String expected) throws SQLException { diff --git a/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/type/dml/BaseDMLE2EIT.java b/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/type/dml/BaseDMLE2EIT.java index 1d6cb531ef12e..b5991082ad625 100644 --- a/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/type/dml/BaseDMLE2EIT.java +++ b/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/type/dml/BaseDMLE2EIT.java @@ -19,6 +19,7 @@ import org.apache.shardingsphere.infra.datanode.DataNode; import org.apache.shardingsphere.infra.expr.core.InlineExpressionParserFactory; +import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory; import org.apache.shardingsphere.test.e2e.cases.dataset.metadata.DataSetColumn; import org.apache.shardingsphere.test.e2e.cases.dataset.metadata.DataSetMetaData; import org.apache.shardingsphere.test.e2e.cases.dataset.row.DataSetRow; @@ -42,7 +43,6 @@ import java.sql.ResultSetMetaData; import java.sql.SQLException; import java.sql.Types; -import java.time.format.DateTimeFormatter; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -55,12 +55,6 @@ public abstract class BaseDMLE2EIT { private static final String DATA_COLUMN_DELIMITER = ", "; - private final DateTimeFormatter dateFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); - - private final DateTimeFormatter timeFormatter = DateTimeFormatter.ofPattern("HH:mm:ss"); - - private final DateTimeFormatter timestampFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.S"); - private DataSetEnvironmentManager dataSetEnvironmentManager; /** @@ -148,14 +142,14 @@ private void assertValue(final AssertionTestParameter testParam, final ResultSet return; } if (Types.DATE == actual.getMetaData().getColumnType(columnIndex)) { - assertThat(dateFormatter.format(actual.getDate(columnIndex).toLocalDate()), is(expected)); + assertThat(DateTimeFormatterFactory.getDateFormatter().format(actual.getDate(columnIndex).toLocalDate()), is(expected)); } else if (Arrays.asList(Types.TIME, Types.TIME_WITH_TIMEZONE).contains(actual.getMetaData().getColumnType(columnIndex))) { - assertThat(timeFormatter.format(actual.getTime(columnIndex).toLocalTime()), is(expected)); + assertThat(DateTimeFormatterFactory.getTimeFormatter().format(actual.getTime(columnIndex).toLocalTime()), is(expected)); } else if (Arrays.asList(Types.TIMESTAMP, Types.TIMESTAMP_WITH_TIMEZONE).contains(actual.getMetaData().getColumnType(columnIndex))) { if ("Oracle".equals(testParam.getDatabaseType().getType()) && "DATE".equalsIgnoreCase(actual.getMetaData().getColumnTypeName(columnIndex))) { - assertThat(dateFormatter.format(actual.getDate(columnIndex).toLocalDate()), is(expected)); + assertThat(DateTimeFormatterFactory.getDateFormatter().format(actual.getDate(columnIndex).toLocalDate()), is(expected)); } else { - assertThat(timestampFormatter.format(actual.getTimestamp(columnIndex).toLocalDateTime()), is(expected)); + assertThat(DateTimeFormatterFactory.getShortMillsFormatter().format(actual.getTimestamp(columnIndex).toLocalDateTime()), is(expected)); } } else if (Types.CHAR == actual.getMetaData().getColumnType(columnIndex) && ("PostgreSQL".equals(testParam.getDatabaseType().getType()) || "openGauss".equals(testParam.getDatabaseType().getType()) diff --git a/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/type/dql/BaseDQLE2EIT.java b/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/type/dql/BaseDQLE2EIT.java index 0a8bc2a843bfb..21ea979539d28 100644 --- a/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/type/dql/BaseDQLE2EIT.java +++ b/test/e2e/sql/src/test/java/org/apache/shardingsphere/test/e2e/engine/type/dql/BaseDQLE2EIT.java @@ -19,6 +19,7 @@ import lombok.AccessLevel; import lombok.Getter; +import org.apache.shardingsphere.infra.util.datetime.DateTimeFormatterFactory; import org.apache.shardingsphere.test.e2e.cases.dataset.metadata.DataSetColumn; import org.apache.shardingsphere.test.e2e.cases.dataset.metadata.DataSetMetaData; import org.apache.shardingsphere.test.e2e.cases.dataset.row.DataSetRow; @@ -38,7 +39,6 @@ import java.sql.Timestamp; import java.sql.Types; import java.time.LocalDateTime; -import java.time.format.DateTimeFormatter; import java.util.Collection; import java.util.HashSet; import java.util.LinkedList; @@ -59,8 +59,6 @@ public abstract class BaseDQLE2EIT { private boolean useXMLAsExpectedDataset; - private final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); - /** * Init. * @@ -201,10 +199,12 @@ private void assertRow(final ResultSet actualResultSet, final ResultSetMetaData // TODO Since mysql 8.0.23, for the DATETIME type, the mysql driver returns the LocalDateTime type, but the proxy returns the Timestamp type. assertThat(((Timestamp) actualValue).toLocalDateTime(), is(expectedValue)); } else if (Types.TIMESTAMP == actualMetaData.getColumnType(i + 1) || Types.TIMESTAMP == expectedMetaData.getColumnType(i + 1)) { - Object convertedActualValue = - Types.TIMESTAMP == actualMetaData.getColumnType(i + 1) ? actualResultSet.getTimestamp(i + 1).toLocalDateTime().format(dateTimeFormatter) : actualValue; - Object convertedExpectedValue = - Types.TIMESTAMP == expectedMetaData.getColumnType(i + 1) ? expectedResultSet.getTimestamp(i + 1).toLocalDateTime().format(dateTimeFormatter) : actualValue; + Object convertedActualValue = Types.TIMESTAMP == actualMetaData.getColumnType(i + 1) + ? actualResultSet.getTimestamp(i + 1).toLocalDateTime().format(DateTimeFormatterFactory.getStandardFormatter()) + : actualValue; + Object convertedExpectedValue = Types.TIMESTAMP == expectedMetaData.getColumnType(i + 1) + ? expectedResultSet.getTimestamp(i + 1).toLocalDateTime().format(DateTimeFormatterFactory.getStandardFormatter()) + : actualValue; assertThat(String.valueOf(convertedActualValue), is(String.valueOf(convertedExpectedValue))); } else { assertThat(String.valueOf(actualValue), is(String.valueOf(expectedValue)));