You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ng...@apache.org on 2019/08/09 06:05:07 UTC

[hive] branch master updated: HIVE-21241: Migrate TimeStamp Parser From Joda Time (Beluga Behr, reviewed by Naveen Gangam)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 311049d  HIVE-21241: Migrate TimeStamp Parser From Joda Time (Beluga Behr, reviewed by Naveen Gangam)
311049d is described below

commit 311049da38428590b2353d3ce7a30dafbdc99b16
Author: Naveen Gangam <ng...@apache.org>
AuthorDate: Fri Aug 9 02:04:26 2019 -0400

    HIVE-21241: Migrate TimeStamp Parser From Joda Time (Beluga Behr, reviewed by Naveen Gangam)
---
 .../apache/hive/common/util/TimestampParser.java   | 313 +++++++++++++--------
 .../hive/common/util/TestTimestampParser.java      | 296 +++++++++----------
 .../apache/hadoop/hive/serde2/TestJsonSerDe.java   |   2 +-
 3 files changed, 346 insertions(+), 265 deletions(-)

diff --git a/common/src/java/org/apache/hive/common/util/TimestampParser.java b/common/src/java/org/apache/hive/common/util/TimestampParser.java
index d30ab88..d180ab1 100644
--- a/common/src/java/org/apache/hive/common/util/TimestampParser.java
+++ b/common/src/java/org/apache/hive/common/util/TimestampParser.java
@@ -18,166 +18,233 @@
 
 package org.apache.hive.common.util;
 
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.time.Instant;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeParseException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+import java.time.temporal.TemporalField;
+import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.List;
-import java.util.Optional;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashSet;
 
 import org.apache.hadoop.hive.common.type.Timestamp;
-import org.joda.time.DateTime;
-import org.joda.time.LocalDateTime;
-import org.joda.time.MutableDateTime;
-import org.joda.time.DateTimeFieldType;
-import org.joda.time.chrono.ISOChronology;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-import org.joda.time.format.DateTimeFormatterBuilder;
-import org.joda.time.format.DateTimeParser;
-import org.joda.time.format.DateTimeParserBucket;
-
-import javax.annotation.Nullable;
-
-import static com.google.common.base.Preconditions.checkState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * Timestamp parser using Joda DateTimeFormatter. Parser accepts 0 or more date time format
- * patterns. If no format patterns are provided it will default to the normal Timestamp parsing.
- * Datetime formats are compatible with Java SimpleDateFormat. Also added special case pattern
- * "millis" to parse the string as milliseconds since Unix epoch.
- * Since this uses Joda DateTimeFormatter, this parser should be thread safe.
+ * Timestamp parser using JDK DateTimeFormatter. Parser accepts 0 or more date
+ * time format patterns. If no format patterns are provided it will default to
+ * the normal Timestamp parsing. Datetime formats are compatible with Java
+ * SimpleDateFormat.
+ *
+ * In addition to accepting format patterns, this parser provides support for
+ * three pre-defined formats:
+ *
+ * <table border="1">
+ * <thead>
+ * <tr>
+ * <th>Formatter</th>
+ * <th>Description</th>
+ * <th>Example</th>
+ * <th>Example Output</th>
+ * </tr>
+ * </thead>
+ * <tr>
+ * <td><b>millis</b></td>
+ * <td>Milliseconds since EPOCH</td>
+ * <td>1420509274123</td>
+ * <td>2015-01-06T01:54:34Z</td>
+ * </tr>
+ * <tr>
+ * <td><b>iso8601</b></td>
+ * <td>Parses ISO-8601 timestamp format</td>
+ * <td>'2011-12-03T10:15:30Z'</td>
+ * <td>2011-12-03T10:15:30Z</td>
+ * </tr>
+ * <tr>
+ * <td><b>rfc1123</b></td>
+ * <td>Parses RFC 1123 timestamp format</td>
+ * <td>'Tue, 3 Jun 2008 11:05:30 GMT'</td>
+ * <td>2008-06-03T11:05:30Z</td>
+ * </tr>
+ * </table>
+ *
+ * @see java.text.SimpleDateFormat
+ * @see DateTimeFormatter
  */
 public class TimestampParser {
 
-  protected final static String[] stringArray = new String[] {};
-  protected final static String millisFormatString = "millis";
-  protected final static DateTime startingDateValue =
-      new DateTime(1970, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimestampParser.class);
+
+  public static final String MILLIS_FORMAT_STR = "millis";
+  public static final String ISO_8601_FORMAT_STR = "iso8601";
+  public static final String RFC_1123_FORMAT_STR = "rfc1123";
 
-  protected String[] formatStrings = null;
-  protected DateTimeFormatter fmt = null;
+  private final Collection<DateTimeFormatter> dtFormatters;
+  private final boolean supportMillisEpoch;
 
+  /**
+   * Create a default Timestamp parser with no formats defined.
+   *
+   * @see Timestamp#valueOf(String)
+   */
   public TimestampParser() {
+    this(Collections.emptyList());
   }
 
-  public TimestampParser(TimestampParser tsParser) {
-    this(tsParser.formatStrings == null ?
-        null : Arrays.copyOf(tsParser.formatStrings, tsParser.formatStrings.length));
+  /**
+   * Create a Timestamp parser based on an existing one.
+   *
+   * @param tsParser The source TimestampParser
+   */
+  public TimestampParser(final TimestampParser tsParser) {
+    this.dtFormatters = tsParser.dtFormatters;
+    this.supportMillisEpoch = tsParser.supportMillisEpoch;
   }
 
-  public TimestampParser(List<String> formatStrings) {
-    this(formatStrings == null ? null : formatStrings.toArray(stringArray));
+  /**
+   * Create a Timestamp parser which parses zero or more time stamp formats.
+   *
+   * @param formatStrings The format strings
+   */
+  public TimestampParser(final String[] formatStrings) {
+    this(formatStrings == null ? Collections.emptyList()
+        : Arrays.asList(formatStrings));
   }
 
-  public TimestampParser(String[] formatStrings) {
-    this.formatStrings = formatStrings;
-
-    // create formatter that includes all of the input patterns
-    if (formatStrings != null && formatStrings.length > 0) {
-      DateTimeParser[] parsers = new DateTimeParser[formatStrings.length];
-      for (int idx = 0; idx < formatStrings.length; ++idx) {
-        String formatString = formatStrings[idx];
-        if (formatString.equalsIgnoreCase(millisFormatString)) {
-          // Use milliseconds parser if pattern matches our special-case millis pattern string
-          parsers[idx] = new MillisDateFormatParser();
-        } else {
-          parsers[idx] = DateTimeFormat.forPattern(formatString).getParser();
-        }
+  /**
+   * Create a timestamp parser with one ore more date patterns. When parsing,
+   * the first pattern in the list is selected for parsing first, so if one
+   * format is more common than others, include it first in the list. If it
+   * fails, the next is chosen, and so on. If none of these patterns succeeds, a
+   * default formatting is expected.
+   *
+   * @see DateTimeFormatter
+   * @see Timestamp#valueOf(String)
+   * @param patterns a collection of timestamp formats
+   */
+  public TimestampParser(final Collection<String> patterns) {
+    final Collection<String> patternSet = new HashSet<>(patterns);
+    this.supportMillisEpoch = patternSet.remove(MILLIS_FORMAT_STR);
+
+    if (patternSet.isEmpty()) {
+      this.dtFormatters = Collections.emptyList();
+      return;
+    }
+
+    this.dtFormatters = new ArrayList<>();
+
+    for (final String patternText : patternSet) {
+      final DateTimeFormatter formatter;
+      switch (patternText) {
+      case ISO_8601_FORMAT_STR:
+        formatter = DateTimeFormatter.ISO_INSTANT;
+        break;
+      case RFC_1123_FORMAT_STR:
+        formatter = DateTimeFormatter.RFC_1123_DATE_TIME;
+        break;
+      default:
+        formatter = DateTimeFormatter.ofPattern(patternText);
+        break;
       }
-      fmt = new DateTimeFormatterBuilder()
-              .append(null, parsers)
-              .toFormatter()
-              .withDefaultYear(1970);
+
+      this.dtFormatters.add(formatter);
     }
   }
 
   /**
-   * Parse the input string and return a timestamp value
-   * @param strValue
-   * @return
-   * @throws IllegalArgumentException if input string cannot be parsed into timestamp
+   * Parse the input string and return a timestamp value.
+   *
+   * @param text The timestamp text
+   * @return A timestamp based on the text provided
+   * @throws IllegalArgumentException if input text cannot be parsed into
+   *           timestamp
    */
-  public Timestamp parseTimestamp(String strValue) throws IllegalArgumentException {
-    if (fmt != null) {
-      Optional<Timestamp> parsed = tryParseWithFormat(strValue);
-      if (parsed.isPresent()) {
-        return parsed.get();
+  public Timestamp parseTimestamp(final String text)
+      throws IllegalArgumentException {
+    if (supportMillisEpoch) {
+      try {
+        // support for milliseconds that include nanoseconds as well
+        // example: "1420509274123.456789"
+        final long millis = new BigDecimal(text).setScale(0, RoundingMode.DOWN)
+            .longValueExact();
+        return Timestamp.ofEpochMilli(millis);
+      } catch (NumberFormatException e) {
+        LOG.debug("Could not format millis: {}", text);
       }
     }
-
-    // Otherwise try default timestamp parsing
-    return Timestamp.valueOf(strValue);
-  }
-
-  private Optional<Timestamp> tryParseWithFormat(String strValue) {
-    checkState(fmt != null);
-
-    if (startingDateValue != null) {
-      // reset value in case any date fields are missing from the date pattern
-      MutableDateTime mdt = new MutableDateTime(
-          startingDateValue, ISOChronology.getInstanceUTC());
-
-      // Using parseInto() avoids throwing exception when parsing,
-      // allowing fallback to default timestamp parsing if custom patterns fail.
-      int ret = fmt.parseInto(mdt, strValue, 0);
-      // Only accept parse results if we parsed the entire string
-      if (ret == strValue.length()) {
-        return Optional.of(Timestamp.ofEpochMilli(mdt.getMillis()));
+    for (DateTimeFormatter formatter : this.dtFormatters) {
+      try {
+        final TemporalAccessor parsed = formatter.parse(text);
+        final Instant inst = Instant.from(wrap(parsed));
+        return Timestamp.ofEpochMilli(inst.toEpochMilli());
+      } catch (DateTimeParseException dtpe) {
+        LOG.debug("Could not parse timestamp text: {}", text);
       }
-      return Optional.empty();
     }
+    return Timestamp.valueOf(text);
+  }
 
-    try {
-      LocalDateTime dt = fmt.parseLocalDateTime(strValue);
-      return Optional.of(
-          Timestamp.ofEpochMilli(
-              dt.toDateTime(ISOChronology.getInstanceUTC().getZone()).getMillis()));
-    } catch (IllegalArgumentException e) {
-      return Optional.empty();
+  /**
+   * The goal of this class is to return a timestamp. A timestamp represents a
+   * single moment (instant) on the time line. However, some strings will not
+   * contain enough information to assign it to one instant in time. For
+   * example, if no time zone information is supplied, or a date is supplied,
+   * but no time. In those cases, they need to be populated manually. This
+   * method accepts all the data parsed from the supplied String and assigns it
+   * reasonable defaults if fields are missing.
+   *
+   * @param in The fields populated by parsing the supplied string
+   * @return The fields populated with default values if required
+   */
+  private TemporalAccessor wrap(final TemporalAccessor in) {
+    if (in.isSupported(ChronoField.INSTANT_SECONDS)
+        && in.isSupported(ChronoField.NANO_OF_SECOND)) {
+      return in;
     }
+    return new DefaultingTemporalAccessor(in);
   }
 
   /**
-   * DateTimeParser to parse the date string as the millis since Unix epoch
+   * Class to wrap a TemporalAccessor and add fields with reasonable defaults.
    */
-  public static class MillisDateFormatParser implements DateTimeParser {
-    private static final ThreadLocal<Matcher> numericMatcher = new ThreadLocal<Matcher>() {
-      @Override
-      protected Matcher initialValue() {
-        return Pattern.compile("(-?\\d+)(\\.\\d+)?$").matcher("");
+  private static class DefaultingTemporalAccessor implements TemporalAccessor {
+    private static final EnumSet<ChronoField> FIELDS =
+        EnumSet.of(ChronoField.YEAR, ChronoField.MONTH_OF_YEAR,
+            ChronoField.DAY_OF_MONTH, ChronoField.HOUR_OF_DAY,
+            ChronoField.MINUTE_OF_HOUR, ChronoField.SECOND_OF_MINUTE,
+            ChronoField.MILLI_OF_SECOND, ChronoField.NANO_OF_SECOND);
+
+    private final TemporalAccessor wrapped;
+
+    DefaultingTemporalAccessor(TemporalAccessor in) {
+      ZonedDateTime dateTime =
+          ZonedDateTime.ofInstant(Instant.EPOCH, ZoneOffset.UTC);
+      for (ChronoField field : FIELDS) {
+        if (in.isSupported(field)) {
+          dateTime = dateTime.with(field, in.getLong(field));
+        }
       }
-    };
-
-    private final static DateTimeFieldType[] dateTimeFields = {
-      DateTimeFieldType.year(),
-      DateTimeFieldType.monthOfYear(),
-      DateTimeFieldType.dayOfMonth(),
-      DateTimeFieldType.hourOfDay(),
-      DateTimeFieldType.minuteOfHour(),
-      DateTimeFieldType.secondOfMinute(),
-      DateTimeFieldType.millisOfSecond()
-    };
-
-    public int estimateParsedLength() {
-      return 13; // Shouldn't hit 14 digits until year 2286
+      this.wrapped = dateTime.toInstant();
     }
 
-    public int parseInto(DateTimeParserBucket bucket, String text, int position) {
-      String substr = text.substring(position);
-      Matcher matcher = numericMatcher.get();
-      matcher.reset(substr);
-      if (!matcher.matches()) {
-        return -1;
-      }
+    @Override
+    public long getLong(TemporalField field) {
+      return wrapped.getLong(field);
+    }
 
-      // Joda DateTime only has precision to millis, cut off any fractional portion
-      long millis = Long.parseLong(matcher.group(1));
-      DateTime dt =
-          new DateTime(millis, ISOChronology.getInstanceUTC());
-      for (DateTimeFieldType field : dateTimeFields) {
-        bucket.saveField(field, dt.get(field));
-      }
-      return substr.length();
+    @Override
+    public boolean isSupported(TemporalField field) {
+      return wrapped.isSupported(field);
     }
   }
 }
diff --git a/common/src/test/org/apache/hive/common/util/TestTimestampParser.java b/common/src/test/org/apache/hive/common/util/TestTimestampParser.java
index 00a7904..4d519ae 100644
--- a/common/src/test/org/apache/hive/common/util/TestTimestampParser.java
+++ b/common/src/test/org/apache/hive/common/util/TestTimestampParser.java
@@ -18,167 +18,181 @@
 
 package org.apache.hive.common.util;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 import org.apache.hadoop.hive.common.type.Timestamp;
+import org.junit.Assert;
 import org.junit.Test;
 
+/**
+ * Test suite for parsing timestamps.
+ */
 public class TestTimestampParser {
-  public static class ValidTimestampCase {
-    String strValue;
-    Timestamp expectedValue;
-
-    public ValidTimestampCase(String strValue, Timestamp expectedValue) {
-      this.strValue = strValue;
-      this.expectedValue = expectedValue;
-    }
-  }
 
-  static void testValidCases(TimestampParser tp, ValidTimestampCase[] validCases) {
-    for (ValidTimestampCase validCase : validCases) {
-      Timestamp ts = tp.parseTimestamp(validCase.strValue);
-      assertEquals("Parsing " + validCase.strValue, validCase.expectedValue, ts);
-    }
-  }
+  /**
+   * No timestamp patterns, should default to normal timestamp format.
+   *
+   * @see Timestamp#valueOf(String)
+   */
+  @Test
+  public void testDefault() {
+    final TimestampParser tsp = new TimestampParser();
+
+    Assert.assertEquals(Timestamp.valueOf("1945-12-31 23:59:59.0"),
+        tsp.parseTimestamp("1945-12-31 23:59:59.0"));
 
-  static void testInvalidCases(TimestampParser tp, String[] invalidCases) {
-    for (String invalidString : invalidCases) {
-      try {
-        Timestamp ts = tp.parseTimestamp(invalidString);
-        fail("Expected exception parsing " + invalidString + ", but parsed value to " + ts);
-      } catch (IllegalArgumentException err) {
-        // Exception expected
-      }
-    }
+    Assert.assertEquals(Timestamp.valueOf("1945-12-31 23:59:59.1234"),
+        tsp.parseTimestamp("1945-12-31 23:59:59.1234"));
+
+    Assert.assertEquals(Timestamp.valueOf("1970-01-01 00:00:00"),
+        tsp.parseTimestamp("1970-01-01 00:00:00"));
+
+    Assert.assertEquals(Timestamp.valueOf("1945-12-31T23:59:59"),
+        tsp.parseTimestamp("1945-12-31 23:59:59"));
   }
 
-  @Test
-  public void testDefault() {
-    // No timestamp patterns, should default to normal timestamp format
-    TimestampParser tp = new TimestampParser();
-    ValidTimestampCase[] validCases = {
-        new ValidTimestampCase("1945-12-31 23:59:59.0",
-            Timestamp.valueOf("1945-12-31 23:59:59.0")),
-        new ValidTimestampCase("1945-12-31 23:59:59.1234",
-            Timestamp.valueOf("1945-12-31 23:59:59.1234")),
-        new ValidTimestampCase("1970-01-01 00:00:00",
-            Timestamp.valueOf("1970-01-01 00:00:00")),
-        new ValidTimestampCase("1945-12-31T23:59:59",
-            Timestamp.valueOf("1945-12-31 23:59:59")),
-    };
-
-    String[] invalidCases = {
-        "12345",
-    };
-
-    testValidCases(tp, validCases);
-    testInvalidCases(tp, invalidCases);
+  @Test(expected = IllegalArgumentException.class)
+  public void testDefaultInvalid() {
+    final TimestampParser tsp = new TimestampParser();
+    tsp.parseTimestamp("12345");
   }
 
   @Test
   public void testPattern1() {
-    // Joda pattern matching expects fractional seconds length to match
+    // Timestamp pattern matching expects fractional seconds length to match
     // the number of 'S' in the pattern. So if you want to match .1, .12, .123,
     // you need 3 different patterns with .S, .SS, .SSS
-    String[] patterns = {
-        // ISO-8601 timestamps
-        "yyyy-MM-dd'T'HH:mm:ss",
-        "yyyy-MM-dd'T'HH:mm:ss.S",
-        "yyyy-MM-dd'T'HH:mm:ss.SS",
-        "yyyy-MM-dd'T'HH:mm:ss.SSS",
-        "yyyy-MM-dd'T'HH:mm:ss.SSSS",
-    };
-    TimestampParser tp = new TimestampParser(patterns);
-
-    ValidTimestampCase[] validCases = {
-        new ValidTimestampCase("1945-12-31T23:59:59.0",
-            Timestamp.valueOf("1945-12-31 23:59:59.0")),
-        new ValidTimestampCase("2001-01-01 00:00:00.100",
-            Timestamp.valueOf("2001-01-01 00:00:00.100")),
-        new ValidTimestampCase("2001-01-01 00:00:00.001",
-            Timestamp.valueOf("2001-01-01 00:00:00.001")),
-        // Joda parsing only supports up to millisecond precision
-        new ValidTimestampCase("1945-12-31T23:59:59.1234",
-            Timestamp.valueOf("1945-12-31 23:59:59.123")),
-        new ValidTimestampCase("1970-01-01T00:00:00",
-            Timestamp.valueOf("1970-01-01 00:00:00")),
-        new ValidTimestampCase("1970-4-5T6:7:8",
-             Timestamp.valueOf("1970-04-05 06:07:08")),
-
-        // Default timestamp format still works?
-        new ValidTimestampCase("2001-01-01 00:00:00",
-            Timestamp.valueOf("2001-01-01 00:00:00")),
-        new ValidTimestampCase("1945-12-31 23:59:59.1234",
-            Timestamp.valueOf("1945-12-31 23:59:59.1234")),
-        new ValidTimestampCase("1945-12-31T23:59:59.12345",
-            Timestamp.valueOf("1945-12-31 23:59:59.12345"))
-    };
-
-    String[] invalidCases = {
-        "1945-12-31-23:59:59",
-        "12345",
-    };
-
-    testValidCases(tp, validCases);
-    testInvalidCases(tp, invalidCases);
+    // ISO-8601 timestamps
+    final String[] patterns = {"yyyy-MM-dd'T'HH:mm:ss",
+        "yyyy-MM-dd'T'HH:mm:ss.S", "yyyy-MM-dd'T'HH:mm:ss.SS",
+        "yyyy-MM-dd'T'HH:mm:ss.SSS", "yyyy-MM-dd'T'HH:mm:ss.SSSS"};
+
+    final TimestampParser tsp = new TimestampParser(patterns);
+
+    Assert.assertEquals(Timestamp.valueOf("1945-12-31 23:59:59.0"),
+        tsp.parseTimestamp("1945-12-31T23:59:59.0"));
+
+    Assert.assertEquals(Timestamp.valueOf("2001-01-01 00:00:00.100"),
+        tsp.parseTimestamp("2001-01-01T00:00:00.100"));
+
+    Assert.assertEquals(Timestamp.valueOf("2001-01-01 00:00:00.001"),
+        tsp.parseTimestamp("2001-01-01T00:00:00.001"));
+
+    Assert.assertEquals(Timestamp.valueOf("1945-12-31T23:59:59.123"),
+        tsp.parseTimestamp("1945-12-31T23:59:59.123"));
+
+    Assert.assertEquals(Timestamp.valueOf("1945-12-31T23:59:59.123"),
+        tsp.parseTimestamp("1945-12-31T23:59:59.1234"));
+
+    Assert.assertEquals(Timestamp.valueOf("1970-01-01 00:00:00"),
+        tsp.parseTimestamp("1970-01-01T00:00:00"));
+
+    /** Default timestamp format still works? */
+
+    Assert.assertEquals(Timestamp.valueOf("1945-12-31 23:59:59.1234"),
+        tsp.parseTimestamp("1945-12-31 23:59:59.1234"));
+
+    Assert.assertEquals(Timestamp.valueOf("1945-12-31 23:59:59.12345"),
+        tsp.parseTimestamp("1945-12-31T23:59:59.12345"));
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testPatternInvalid1() {
+    final String[] patterns = {"yyyy-MM-dd'T'HH:mm:ss",
+        "yyyy-MM-dd'T'HH:mm:ss.S", "yyyy-MM-dd'T'HH:mm:ss.SS",
+        "yyyy-MM-dd'T'HH:mm:ss.SSS", "yyyy-MM-dd'T'HH:mm:ss.SSSS"};
+
+    final TimestampParser tsp = new TimestampParser(patterns);
+    tsp.parseTimestamp("1945-12-31-23:59:59");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testPatternInvalid2() {
+    final String[] patterns = {"yyyy-MM-dd'T'HH:mm:ss",
+        "yyyy-MM-dd'T'HH:mm:ss.S", "yyyy-MM-dd'T'HH:mm:ss.SS",
+        "yyyy-MM-dd'T'HH:mm:ss.SSS", "yyyy-MM-dd'T'HH:mm:ss.SSSS"};
+
+    final TimestampParser tsp = new TimestampParser(patterns);
+    tsp.parseTimestamp("12345");
   }
 
   @Test
   public void testMillisParser() {
-    String[] patterns = {
-        "millis",
-        // Also try other patterns
-        "yyyy-MM-dd'T'HH:mm:ss",
-    };
-    TimestampParser tp = new TimestampParser(patterns);
-
-    ValidTimestampCase[] validCases = {
-        new ValidTimestampCase("0", Timestamp.ofEpochMilli(0)),
-        new ValidTimestampCase("-1000000", Timestamp.ofEpochMilli(-1000000)),
-        new ValidTimestampCase("1420509274123", Timestamp.ofEpochMilli(1420509274123L)),
-        new ValidTimestampCase("1420509274123.456789", Timestamp.ofEpochMilli(1420509274123L)),
-
-        // Other format pattern should also work
-        new ValidTimestampCase("1945-12-31T23:59:59",
-            Timestamp.valueOf("1945-12-31 23:59:59")),
-        new ValidTimestampCase("1945-12-31T23:59:59.12345",
-            Timestamp.valueOf("1945-12-31 23:59:59.12345")),
-    };
-
-    String[] invalidCases = {
-        "1945-12-31-23:59:59",
-        "1420509274123-",
-    };
-
-    testValidCases(tp, validCases);
-    testInvalidCases(tp, invalidCases);
+    // Also try other patterns
+    final String[] patterns = {"millis", "yyyy-MM-dd'T'HH:mm:ss"};
+
+    final TimestampParser tsp = new TimestampParser(patterns);
+
+    Assert.assertEquals(Timestamp.ofEpochMilli(0L), tsp.parseTimestamp("0"));
+
+    Assert.assertEquals(Timestamp.ofEpochMilli(-1000000L),
+        tsp.parseTimestamp("-1000000"));
+
+    Assert.assertEquals(Timestamp.ofEpochMilli(1420509274123L),
+        tsp.parseTimestamp("1420509274123"));
+
+    Assert.assertEquals(Timestamp.ofEpochMilli(1420509274123L),
+        tsp.parseTimestamp("1420509274123.456789"));
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testMillisParserInvalid1() {
+    final String[] patterns = {"millis", "yyyy-MM-dd'T'HH:mm:ss"};
+
+    final TimestampParser tsp = new TimestampParser(patterns);
+    tsp.parseTimestamp("1420509274123-");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testMillisParserInvalid2() {
+    // Also try other patterns
+    final String[] patterns = {"millis", "yyyy-MM-dd'T'HH:mm:ss"};
+
+    final TimestampParser tsp = new TimestampParser(patterns);
+    tsp.parseTimestamp("1945-12-31-23:59:59");
   }
 
+  /**
+   * Test for pattern that does not contain all date fields.
+   */
   @Test
-  public void testPattern2() {
-    // Pattern does not contain all date fields
-    String[] patterns = {
-        "HH:mm",
-        "MM:dd:ss",
-    };
-    TimestampParser tp = new TimestampParser(patterns);
-
-    ValidTimestampCase[] validCases = {
-        new ValidTimestampCase("05:06",
-            Timestamp.valueOf("1970-01-01 05:06:00")),
-        new ValidTimestampCase("05:06:07",
-            Timestamp.valueOf("1970-05-06 00:00:07")),
-        new ValidTimestampCase("1945-12-31T23:59:59",
-            Timestamp.valueOf("1945-12-31 23:59:59")),
-    };
-
-    String[] invalidCases = {
-        "1945:12:31-",
-        "12345",
-    };
-
-    testValidCases(tp, validCases);
-    testInvalidCases(tp, invalidCases);
+  public void testPatternShort() {
+    final String[] patterns = {"MM:dd:ss", "HH:mm"};
+
+    final TimestampParser tsp = new TimestampParser(patterns);
+
+    Assert.assertEquals(Timestamp.valueOf("1970-01-01 05:06:00"),
+        tsp.parseTimestamp("05:06"));
+
+    Assert.assertEquals(Timestamp.valueOf("1970-05-06 00:00:07"),
+        tsp.parseTimestamp("05:06:07"));
+
+    Assert.assertEquals(Timestamp.valueOf("1945-12-31 23:59:59"),
+        tsp.parseTimestamp("1945-12-31T23:59:59"));
+  }
+
+  @Test
+  public void testPatternTimeZone() {
+    final String[] patterns = {"yyyy-MM-dd'T'HH:mm:ssX"};
+
+    final TimestampParser tsp = new TimestampParser(patterns);
+    Assert.assertEquals(Timestamp.valueOf("1945-12-31 23:59:59"),
+        tsp.parseTimestamp("1945-12-31T23:59:59Z"));
+  }
+
+  @Test
+  public void testPatternISO8601() {
+    final String[] patterns = {"iso8601"};
+
+    final TimestampParser tsp = new TimestampParser(patterns);
+    Assert.assertEquals(Timestamp.valueOf("1945-12-31 23:59:59"),
+        tsp.parseTimestamp("1945-12-31T23:59:59Z"));
+  }
+
+  @Test
+  public void testPatternRFC1123() {
+    final String[] patterns = {"rfc1123"};
+
+    final TimestampParser tsp = new TimestampParser(patterns);
+    Assert.assertEquals(Timestamp.valueOf("2008-06-03 11:05:30"),
+        tsp.parseTimestamp("Tue, 3 Jun 2008 11:05:30 GMT"));
   }
 }
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/TestJsonSerDe.java b/serde/src/test/org/apache/hadoop/hive/serde2/TestJsonSerDe.java
index 709871f..2e4eb01 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/TestJsonSerDe.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/TestJsonSerDe.java
@@ -274,7 +274,7 @@ public class TestJsonSerDe {
     props.setProperty(serdeConstants.LIST_COLUMN_TYPES,
         "timestamp with local time zone");
     props.setProperty(serdeConstants.TIMESTAMP_FORMATS,
-        "yyyy-MM-dd'T'HH:mm:ss'Z");
+        "yyyy-MM-dd'T'HH:mm:ss'Z'");
 
     final TimeZone localTz = TimeZone.getDefault();
     TimeZone.setDefault(TimeZone.getTimeZone("US/Pacific"));