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

sunnianjun pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new 14cdd1fd808 Refactor DateTimeFormatterFactory (#29205)
14cdd1fd808 is described below

commit 14cdd1fd808d173baa48565045871a476efa301c
Author: Liang Zhang <[email protected]>
AuthorDate: Sat Nov 25 19:45:12 2023 +0800

    Refactor DateTimeFormatterFactory (#29205)
---
 .../time/MySQLTimestampBinlogProtocolValue.java    |  6 +-
 .../query/text/MySQLTextResultSetRowPacket.java    |  4 +-
 .../MySQLTimestampBinlogProtocolValueTest.java     |  4 +-
 .../datetime/AutoIntervalShardingAlgorithm.java    |  8 +-
 .../datetime/IntervalShardingAlgorithmTest.java    |  3 +-
 .../util/datetime/DateTimeFormatterFactory.java    | 85 ++++++++++++++++++++++
 .../util/datetime/StandardDateTimeFormatter.java   | 41 -----------
 .../datetime/DateTimeFormatterFactoryTest.java     | 51 +++++++++++++
 .../datetime/StandardDateTimeFormatterTest.java    | 31 --------
 .../execute/AbstractPipelineLifecycleRunnable.java |  4 +-
 .../service/PipelineJobConfigurationManager.java   |  4 +-
 .../core/job/service/PipelineJobManager.java       |  4 +-
 .../data/pipeline/cdc/api/impl/CDCJobOption.java   |  4 +-
 .../api/impl/ConsistencyCheckJobOption.java        | 10 +--
 .../test/e2e/cases/value/SQLValue.java             | 26 +++----
 .../engine/composer/BatchE2EContainerComposer.java | 10 +--
 .../test/e2e/engine/type/DALE2EIT.java             |  8 +-
 .../test/e2e/engine/type/dml/BaseDMLE2EIT.java     | 16 ++--
 .../test/e2e/engine/type/dql/BaseDQLE2EIT.java     | 14 ++--
 19 files changed, 186 insertions(+), 147 deletions(-)

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 1bd4f1d9bba..9300dc946dd 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 @@ package 
org.apache.shardingsphere.db.protocol.mysql.packet.binlog.row.column.val
 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 @@ import java.time.format.DateTimeFormatter;
  */
 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 81fca057930..b4eabc9b3f2 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.Getter;
 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 @@ public final class MySQLTextResultSetRowPacket extends 
MySQLPacket {
         } 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 ceba78e9d66..6f128c8d291 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 @@ package 
org.apache.shardingsphere.db.protocol.mysql.packet.binlog.row.column.val
 
 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 @@ class MySQLTimestampBinlogProtocolValueTest {
         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 6db951fff53..9eaa07287b0 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 @@ package 
org.apache.shardingsphere.sharding.algorithm.sharding.datetime;
 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.DecimalFormat;
 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 @@ public final class AutoIntervalShardingAlgorithm implements 
StandardShardingAlgo
         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 @@ public final class AutoIntervalShardingAlgorithm implements 
StandardShardingAlgo
     }
     
     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 585f37d9309..1c3a53adead 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 @@ package 
org.apache.shardingsphere.sharding.algorithm.sharding.datetime;
 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 @@ class IntervalShardingAlgorithmTest {
                                 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()),
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 00000000000..b61eb72ae22
--- /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 a1119fca678..00000000000
--- 
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 00000000000..af926f8c52d
--- /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 d4730283978..00000000000
--- 
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 2b5d85fd594..0d45d12b40b 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 abstract class AbstractPipelineLifecycleRunnable 
implements PipelineLifec
             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 cf33f799c71..c02dc1355a8 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.common.listener.PipelineElasticJo
 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 final class PipelineJobConfigurationManager {
         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 39968ff6690..717125ea114 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.data.pipeline.core.job.option.TransmissionJobOp
 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 @@ public final class PipelineJobManager {
             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 af0c219b7ef..48f6bf3bb61 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.exception.core.ShardingSpherePrecondition
 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 final class CDCJobOption implements 
TransmissionJobOption {
         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 afd2c577040..c793a22bd50 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.data.pipeline.scenario.consistencycheck.util.Co
 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 @@ import java.util.stream.Collectors;
 @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 @@ public final class ConsistencyCheckJobOption implements 
PipelineJobOption {
         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 @@ public final class ConsistencyCheckJobOption implements 
PipelineJobOption {
             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 667f11e32ce..f98c4e4fedb 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 @@ package org.apache.shardingsphere.test.e2e.cases.value;
 
 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.sql.Timestamp;
 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 @@ public final class SQLValue {
             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 final class SQLValue {
             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 df6e7922d19..a8f0e7cbac3 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 @@ package org.apache.shardingsphere.test.e2e.engine.composer;
 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.ResultSet;
 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 @@ public final class BatchE2EContainerComposer extends 
E2EContainerComposer implem
             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 9aad8134fe3..35f4df29e3d 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.ResultSetMetaData;
 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 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 @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 @@ class DALE2EIT {
         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 1d6cb531ef1..b5991082ad6 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 @@ package org.apache.shardingsphere.test.e2e.engine.type.dml;
 
 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.ResultSet;
 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 @@ public abstract class BaseDMLE2EIT {
             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 0a8bc2a843b..21ea979539d 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 @@ package org.apache.shardingsphere.test.e2e.engine.type.dql;
 
 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.SQLException;
 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 @@ public abstract class BaseDQLE2EIT {
                     // 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)));

Reply via email to