From 3bdd750982c8a8cd18d6e02040e7a00d7244c018 Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Tue, 30 Jan 2018 17:13:54 -0800 Subject: [PATCH] Use FastDateFormat for thread safety Use FastDateFormat for thread safety, this is to fix an exception when a job is used to ingest multiple tables. An example exception: ``` Caused by: java.lang.NumberFormatException: multiple points at sun.misc.FloatingDecimal.readJavaFormatString(FloatingDecimal.java:1890) at sun.misc.FloatingDecimal.parseDouble(FloatingDecimal.java:110) at java.lang.Double.parseDouble(Double.java:538) at java.text.DigitList.getDouble(DigitList.java:169) at java.text.DecimalFormat.parse(DecimalFormat.java:2056) at java.text.SimpleDateFormat.subParse(SimpleDateFormat.java:1867) at java.text.SimpleDateFormat.parse(SimpleDateFormat.java:1514) at java.text.DateFormat.parse(DateFormat.java:364) at com.uber.hoodie.HoodieWriteClient.commit(HoodieWriteClient.java:442) ``` --- .../hoodie/common/table/timeline/HoodieActiveTimeline.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java index 4c9f5c523..8c34717b2 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java @@ -24,7 +24,6 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.io.Serializable; -import java.text.SimpleDateFormat; import java.util.Arrays; import java.util.Comparator; import java.util.Date; @@ -35,6 +34,7 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.time.FastDateFormat; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; @@ -51,7 +51,7 @@ import org.apache.log4j.Logger; */ public class HoodieActiveTimeline extends HoodieDefaultTimeline { - public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss"); + public static final FastDateFormat COMMIT_FORMATTER = FastDateFormat.getInstance("yyyyMMddHHmmss"); private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class); private HoodieTableMetaClient metaClient;