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

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


The following commit(s) were added to refs/heads/master by this push:
     new 3abb856200 [INLONG-10041][Sort] Thread unsafety of time format 
serialization (#10042)
3abb856200 is described below

commit 3abb8562007372b59166cae71232aed286f45dd0
Author: vernedeng <[email protected]>
AuthorDate: Tue Apr 23 15:19:23 2024 +0800

    [INLONG-10041][Sort] Thread unsafety of time format serialization (#10042)
---
 .../sort/dataflow/field/format/DateFormatInfo.java | 27 ++++-----------------
 .../sort/dataflow/field/format/TimeFormatInfo.java | 27 +++++----------------
 .../dataflow/field/format/TimestampFormatInfo.java | 28 +++++-----------------
 3 files changed, 17 insertions(+), 65 deletions(-)

diff --git 
a/inlong-common/src/main/java/org/apache/inlong/common/pojo/sort/dataflow/field/format/DateFormatInfo.java
 
b/inlong-common/src/main/java/org/apache/inlong/common/pojo/sort/dataflow/field/format/DateFormatInfo.java
index 5c2694cf59..32feaa9625 100644
--- 
a/inlong-common/src/main/java/org/apache/inlong/common/pojo/sort/dataflow/field/format/DateFormatInfo.java
+++ 
b/inlong-common/src/main/java/org/apache/inlong/common/pojo/sort/dataflow/field/format/DateFormatInfo.java
@@ -17,16 +17,14 @@
 
 package org.apache.inlong.common.pojo.sort.dataflow.field.format;
 
+import org.apache.commons.lang3.time.FastDateFormat;
 import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
 import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
 
 import java.sql.Date;
 import java.text.ParseException;
-import java.text.SimpleDateFormat;
 import java.util.Objects;
 import java.util.concurrent.TimeUnit;
 
@@ -46,23 +44,16 @@ public class DateFormatInfo implements 
BasicFormatInfo<Date> {
     @Nonnull
     private final String format;
 
-    @JsonIgnore
-    @Nullable
-    private final SimpleDateFormat simpleDateFormat;
-
     @JsonCreator
     public DateFormatInfo(
             @JsonProperty(FIELD_FORMAT) @Nonnull String format) {
         this.format = format;
-
         if (!format.equals("SECONDS")
                 && !format.equals("MILLIS")
                 && !format.equals("MICROS")
                 && !DATE_AND_TIME_STANDARD_SQL.equals(format)
                 && !DATE_AND_TIME_STANDARD_ISO_8601.equals(format)) {
-            this.simpleDateFormat = new SimpleDateFormat(format);
-        } else {
-            this.simpleDateFormat = null;
+            FastDateFormat.getInstance(format);
         }
     }
 
@@ -98,11 +89,7 @@ public class DateFormatInfo implements BasicFormatInfo<Date> 
{
                 return Long.toString(seconds);
             }
             default: {
-                if (simpleDateFormat == null) {
-                    throw new IllegalStateException();
-                }
-
-                return simpleDateFormat.format(date);
+                return 
FastDateFormat.getInstance(format).format(date.getTime());
             }
         }
     }
@@ -126,12 +113,8 @@ public class DateFormatInfo implements 
BasicFormatInfo<Date> {
                 return new Date(millis);
             }
             default: {
-                if (simpleDateFormat == null) {
-                    throw new IllegalStateException();
-                }
-
-                java.util.Date jDate = simpleDateFormat.parse(text.trim());
-                return new Date(jDate.getTime());
+                java.util.Date date = 
FastDateFormat.getInstance(format).parse(text.trim());
+                return new Date(date.getTime());
             }
         }
     }
diff --git 
a/inlong-common/src/main/java/org/apache/inlong/common/pojo/sort/dataflow/field/format/TimeFormatInfo.java
 
b/inlong-common/src/main/java/org/apache/inlong/common/pojo/sort/dataflow/field/format/TimeFormatInfo.java
index 65101cb7bd..57369dfe20 100644
--- 
a/inlong-common/src/main/java/org/apache/inlong/common/pojo/sort/dataflow/field/format/TimeFormatInfo.java
+++ 
b/inlong-common/src/main/java/org/apache/inlong/common/pojo/sort/dataflow/field/format/TimeFormatInfo.java
@@ -17,16 +17,14 @@
 
 package org.apache.inlong.common.pojo.sort.dataflow.field.format;
 
+import org.apache.commons.lang3.time.FastDateFormat;
 import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
 import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
 
 import java.sql.Time;
 import java.text.ParseException;
-import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Objects;
 import java.util.concurrent.TimeUnit;
@@ -47,9 +45,6 @@ public class TimeFormatInfo implements BasicFormatInfo<Time> {
     @JsonProperty(FIELD_FORMAT)
     @Nonnull
     private final String format;
-    @JsonIgnore
-    @Nullable
-    private final SimpleDateFormat simpleDateFormat;
     @JsonProperty("precision")
     private int precision;
 
@@ -59,14 +54,12 @@ public class TimeFormatInfo implements 
BasicFormatInfo<Time> {
             @JsonProperty("precision") int precision) {
         this.format = format;
         this.precision = precision;
-        if (!format.equals("MICROS")
+        if (!format.equals("SECONDS")
                 && !format.equals("MILLIS")
-                && !format.equals("SECONDS")
+                && !format.equals("MICROS")
                 && !DATE_AND_TIME_STANDARD_SQL.equals(format)
                 && !DATE_AND_TIME_STANDARD_ISO_8601.equals(format)) {
-            this.simpleDateFormat = new SimpleDateFormat(format);
-        } else {
-            this.simpleDateFormat = null;
+            FastDateFormat.getInstance(format);
         }
     }
 
@@ -106,11 +99,7 @@ public class TimeFormatInfo implements 
BasicFormatInfo<Time> {
                 return Long.toString(seconds);
             }
             default: {
-                if (simpleDateFormat == null) {
-                    throw new IllegalStateException();
-                }
-
-                return simpleDateFormat.format(time);
+                return 
FastDateFormat.getInstance(format).format(time.getTime());
             }
         }
     }
@@ -133,11 +122,7 @@ public class TimeFormatInfo implements 
BasicFormatInfo<Time> {
                 return new Time(millis);
             }
             default: {
-                if (simpleDateFormat == null) {
-                    throw new IllegalStateException();
-                }
-
-                Date date = simpleDateFormat.parse(text);
+                Date date = FastDateFormat.getInstance(format).parse(text);
                 return new Time(date.getTime());
             }
         }
diff --git 
a/inlong-common/src/main/java/org/apache/inlong/common/pojo/sort/dataflow/field/format/TimestampFormatInfo.java
 
b/inlong-common/src/main/java/org/apache/inlong/common/pojo/sort/dataflow/field/format/TimestampFormatInfo.java
index 1985c2fea5..0afae59ce5 100644
--- 
a/inlong-common/src/main/java/org/apache/inlong/common/pojo/sort/dataflow/field/format/TimestampFormatInfo.java
+++ 
b/inlong-common/src/main/java/org/apache/inlong/common/pojo/sort/dataflow/field/format/TimestampFormatInfo.java
@@ -17,16 +17,14 @@
 
 package org.apache.inlong.common.pojo.sort.dataflow.field.format;
 
+import org.apache.commons.lang3.time.FastDateFormat;
 import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
 import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
 
 import java.sql.Timestamp;
 import java.text.ParseException;
-import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Objects;
 import java.util.concurrent.TimeUnit;
@@ -49,10 +47,6 @@ public class TimestampFormatInfo implements 
BasicFormatInfo<Timestamp> {
     @Nonnull
     private final String format;
 
-    @JsonIgnore
-    @Nullable
-    private final SimpleDateFormat simpleDateFormat;
-
     @JsonProperty("precision")
     private int precision;
 
@@ -62,14 +56,12 @@ public class TimestampFormatInfo implements 
BasicFormatInfo<Timestamp> {
             @JsonProperty("precision") int precision) {
         this.format = format;
         this.precision = precision;
-        if (!format.equals("MICROS")
+        if (!format.equals("SECONDS")
                 && !format.equals("MILLIS")
-                && !format.equals("SECONDS")
+                && !format.equals("MICROS")
                 && !DATE_AND_TIME_STANDARD_SQL.equals(format)
                 && !DATE_AND_TIME_STANDARD_ISO_8601.equals(format)) {
-            this.simpleDateFormat = new SimpleDateFormat(format);
-        } else {
-            this.simpleDateFormat = null;
+            FastDateFormat.getInstance(format);
         }
     }
 
@@ -114,11 +106,7 @@ public class TimestampFormatInfo implements 
BasicFormatInfo<Timestamp> {
                 return Long.toString(seconds);
             }
             default: {
-                if (simpleDateFormat == null) {
-                    throw new IllegalStateException();
-                }
-
-                return simpleDateFormat.format(timestamp);
+                return 
FastDateFormat.getInstance(format).format(timestamp.getTime());
             }
         }
     }
@@ -141,11 +129,7 @@ public class TimestampFormatInfo implements 
BasicFormatInfo<Timestamp> {
                 return new Timestamp(millis);
             }
             default: {
-                if (simpleDateFormat == null) {
-                    throw new IllegalStateException();
-                }
-
-                Date date = simpleDateFormat.parse(text);
+                Date date = FastDateFormat.getInstance(format).parse(text);
                 return new Timestamp(date.getTime());
             }
         }

Reply via email to