You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2019/11/23 16:50:54 UTC

[hive] branch master updated: HIVE-22405: Add ColumnVector support for ProlepticCalendar (László Bodor via Owen O'Malley, Jesus Camacho Rodriguez)

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

jcamacho 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 1fcc5ce  HIVE-22405: Add ColumnVector support for ProlepticCalendar (László Bodor via Owen O'Malley, Jesus Camacho Rodriguez)
1fcc5ce is described below

commit 1fcc5cecc25003a49ebbc958c451c229a50cd2fe
Author: László Bodor <bo...@gmail.com>
AuthorDate: Sat Nov 23 08:49:23 2019 -0800

    HIVE-22405: Add ColumnVector support for ProlepticCalendar (László Bodor via Owen O'Malley, Jesus Camacho Rodriguez)
---
 .../hive/ql/exec/vector/DateColumnVector.java      | 126 +++++++++++++++++++
 .../hive/ql/exec/vector/TimestampColumnVector.java |  83 +++++++++++-
 .../hive/ql/exec/vector/TestDateColumnVector.java  |  80 ++++++++++++
 .../ql/exec/vector/TestTimestampColumnVector.java  | 140 +++++++++++++++++----
 4 files changed, 407 insertions(+), 22 deletions(-)

diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DateColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DateColumnVector.java
new file mode 100644
index 0000000..3dac667
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DateColumnVector.java
@@ -0,0 +1,126 @@
+/*
+ * 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.hadoop.hive.ql.exec.vector;
+
+import java.text.SimpleDateFormat;
+import java.util.GregorianCalendar;
+import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This class extends LongColumnVector in order to introduce some date-specific semantics. In
+ * DateColumnVector, the elements of vector[] represent the days since 1970-01-01
+ */
+public class DateColumnVector extends LongColumnVector {
+  private static final TimeZone UTC = TimeZone.getTimeZone("UTC");
+  private static final GregorianCalendar PROLEPTIC_GREGORIAN_CALENDAR = new GregorianCalendar(UTC);
+  private static final GregorianCalendar GREGORIAN_CALENDAR = new GregorianCalendar(UTC);
+
+  private static final SimpleDateFormat PROLEPTIC_GREGORIAN_DATE_FORMATTER =
+      new SimpleDateFormat("yyyy-MM-dd");
+  private static final SimpleDateFormat GREGORIAN_DATE_FORMATTER =
+      new SimpleDateFormat("yyyy-MM-dd");
+
+  /**
+  * -141427: hybrid: 1582-10-15 proleptic: 1582-10-15
+  * -141428: hybrid: 1582-10-04 proleptic: 1582-10-14
+  */
+  private static final int CUTOVER_DAY_EPOCH = -141427; // it's 1582-10-15 in both calendars
+
+  static {
+    PROLEPTIC_GREGORIAN_CALENDAR.setGregorianChange(new java.util.Date(Long.MIN_VALUE));
+
+    PROLEPTIC_GREGORIAN_DATE_FORMATTER.setCalendar(PROLEPTIC_GREGORIAN_CALENDAR);
+    GREGORIAN_DATE_FORMATTER.setCalendar(GREGORIAN_CALENDAR);
+  }
+
+  private boolean usingProlepticCalendar = false;
+
+  public DateColumnVector() {
+    this(VectorizedRowBatch.DEFAULT_SIZE);
+  }
+
+  /**
+   * Change the calendar to or from proleptic. If the new and old values of the flag are the same,
+   * nothing is done. useProleptic - set the flag for the proleptic calendar updateData - change the
+   * data to match the new value of the flag.
+   */
+  public void changeCalendar(boolean useProleptic, boolean updateData) {
+    if (useProleptic == usingProlepticCalendar) {
+      return;
+    }
+    usingProlepticCalendar = useProleptic;
+    if (updateData) {
+      try {
+        updateDataAccordingProlepticSetting();
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  private void updateDataAccordingProlepticSetting() throws Exception {
+    for (int i = 0; i < vector.length; i++) {
+      if (vector[i] >= CUTOVER_DAY_EPOCH) { // no need for conversion
+        continue;
+      }
+      long millis = TimeUnit.DAYS.toMillis(vector[i]);
+      String originalFormatted = usingProlepticCalendar ? GREGORIAN_DATE_FORMATTER.format(millis)
+        : PROLEPTIC_GREGORIAN_DATE_FORMATTER.format(millis);
+
+      millis = (usingProlepticCalendar ? PROLEPTIC_GREGORIAN_DATE_FORMATTER.parse(originalFormatted)
+        : GREGORIAN_DATE_FORMATTER.parse(originalFormatted)).getTime();
+
+      vector[i] = TimeUnit.MILLISECONDS.toDays(millis);
+    }
+  }
+
+  public String formatDate(int i) {
+    long millis = TimeUnit.DAYS.toMillis(vector[i]);
+    return usingProlepticCalendar ? PROLEPTIC_GREGORIAN_DATE_FORMATTER.format(millis)
+      : GREGORIAN_DATE_FORMATTER.format(millis);
+  }
+
+  public DateColumnVector setUsingProlepticCalendar(boolean usingProlepticCalendar) {
+    this.usingProlepticCalendar = usingProlepticCalendar;
+    return this;
+  }
+
+  /**
+   * Detect whether this data is using the proleptic calendar.
+   */
+  public boolean isUsingProlepticCalendar() {
+    return usingProlepticCalendar;
+  }
+
+  /**
+   * Don't use this except for testing purposes.
+   *
+   * @param len the number of rows
+   */
+  public DateColumnVector(int len) {
+    super(len);
+  }
+
+  @Override
+  public void shallowCopyTo(ColumnVector otherCv) {
+    DateColumnVector other = (DateColumnVector) otherCv;
+    super.shallowCopyTo(other);
+    other.vector = vector;
+  }
+}
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
index 3b5f3ba..d5dfc92 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
@@ -18,10 +18,13 @@
 package org.apache.hadoop.hive.ql.exec.vector;
 
 import java.sql.Timestamp;
+import java.text.SimpleDateFormat;
 import java.time.Instant;
 import java.time.LocalDateTime;
 import java.time.ZoneOffset;
 import java.util.Arrays;
+import java.util.GregorianCalendar;
+import java.util.TimeZone;
 
 import org.apache.hadoop.io.Writable;
 
@@ -38,6 +41,26 @@ import org.apache.hadoop.io.Writable;
  * using the scratch timestamp, and then perhaps update the column vector row with a result.
  */
 public class TimestampColumnVector extends ColumnVector {
+  private static final TimeZone UTC = TimeZone.getTimeZone("UTC");
+  private static final GregorianCalendar PROLEPTIC_GREGORIAN_CALENDAR_UTC =
+      new GregorianCalendar(UTC);
+  private static final GregorianCalendar GREGORIAN_CALENDAR_UTC =
+      new GregorianCalendar(UTC);
+
+  private static final SimpleDateFormat PROLEPTIC_GREGORIAN_TIMESTAMP_FORMATTER_UTC =
+      new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+  private static final SimpleDateFormat GREGORIAN_TIMESTAMP_FORMATTER_UTC =
+      new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+  static {
+    PROLEPTIC_GREGORIAN_CALENDAR_UTC.setGregorianChange(new java.util.Date(Long.MIN_VALUE));
+
+    PROLEPTIC_GREGORIAN_TIMESTAMP_FORMATTER_UTC.setCalendar(PROLEPTIC_GREGORIAN_CALENDAR_UTC);
+    GREGORIAN_TIMESTAMP_FORMATTER_UTC.setCalendar(GREGORIAN_CALENDAR_UTC);
+  }
+
+  // it's 1582-10-15 in both calendars
+  private static final int CUTOVER_MILLIS_EPOCH = -141427 * 24 * 60 * 60 * 1000;
 
   /*
    * The storage arrays for this column vector corresponds to the storage of a Timestamp:
@@ -58,6 +81,8 @@ public class TimestampColumnVector extends ColumnVector {
 
   private boolean isUTC;
 
+  private boolean usingProlepticCalendar = false;
+
   /**
    * Use this constructor by default. All column vectors
    * should normally be the default size.
@@ -546,4 +571,60 @@ public class TimestampColumnVector extends ColumnVector {
     other.time = time;
     other.nanos = nanos;
   }
-}
+
+  /**
+   * Change the calendar to or from proleptic. If the new and old values of the flag are the
+   * same, nothing is done.
+   * useProleptic - set the flag for the proleptic calendar
+   * updateData - change the data to match the new value of the flag.
+   */
+  public void changeCalendar(boolean useProleptic, boolean updateData) {
+    if (useProleptic == usingProlepticCalendar) {
+      return;
+    }
+    usingProlepticCalendar = useProleptic;
+    if (updateData) {
+      try {
+        updateDataAccordingProlepticSetting();
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  private void updateDataAccordingProlepticSetting() throws Exception {
+    for (int i = 0; i < nanos.length; i++) {
+      if (time[i] >= CUTOVER_MILLIS_EPOCH) { // no need for conversion
+        continue;
+      }
+      asScratchTimestamp(i);
+      long offset = 0;
+      String formatted =
+          usingProlepticCalendar ? GREGORIAN_TIMESTAMP_FORMATTER_UTC.format(scratchTimestamp)
+            : PROLEPTIC_GREGORIAN_TIMESTAMP_FORMATTER_UTC.format(scratchTimestamp);
+
+      long millis = usingProlepticCalendar
+        ? PROLEPTIC_GREGORIAN_TIMESTAMP_FORMATTER_UTC.parse(formatted).getTime()
+        : GREGORIAN_TIMESTAMP_FORMATTER_UTC.parse(formatted).getTime();
+
+      Timestamp newTimeStamp = Timestamp.from(Instant.ofEpochMilli(millis));
+
+      scratchTimestamp.setTime(newTimeStamp.getTime() + offset);
+      scratchTimestamp.setNanos(nanos[i]);
+
+      setFromScratchTimestamp(i);
+    }
+  }
+
+  public TimestampColumnVector setUsingProlepticCalendar(boolean usingProlepticCalendar) {
+    this.usingProlepticCalendar = usingProlepticCalendar;
+    return this;
+  }
+
+  /**
+   * Detect whether this data is using the proleptic calendar.
+   */
+  public boolean usingProlepticCalendar() {
+    return usingProlepticCalendar;
+  }
+}
\ No newline at end of file
diff --git a/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestDateColumnVector.java b/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestDateColumnVector.java
new file mode 100644
index 0000000..0d4dc5d
--- /dev/null
+++ b/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestDateColumnVector.java
@@ -0,0 +1,80 @@
+/*
+ * 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.hadoop.hive.ql.exec.vector;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestDateColumnVector {
+  /**
+   * Test case for DateColumnVector's changeCalendar
+   * epoch days, hybrid representation, proleptic representation
+   *   16768: hybrid: 2015-11-29 proleptic: 2015-11-29
+   * -141418: hybrid: 1582-10-24 proleptic: 1582-10-24
+   * -141427: hybrid: 1582-10-15 proleptic: 1582-10-15
+   * -141428: hybrid: 1582-10-04 proleptic: 1582-10-14
+   * -141430: hybrid: 1582-10-02 proleptic: 1582-10-12
+   * -141437: hybrid: 1582-09-25 proleptic: 1582-10-05
+   * -141438: hybrid: 1582-09-24 proleptic: 1582-10-04
+   * -499952: hybrid: 0601-03-04 proleptic: 0601-03-07
+   * -499955: hybrid: 0601-03-01 proleptic: 0601-03-04
+   * @throws Exception 
+   */
+  @Test
+  public void testProlepticCalendar() throws Exception {
+    // the expected output is the same as it was with the original calendar,
+    // as the goal is to change the underlying data in a way that the user
+    // gets the same string representation after the calendar change
+
+    // from hybrid internal representation to proleptic
+    setDateAndVerifyProlepticUpdate(16768, "2015-11-29", false, true);
+    setDateAndVerifyProlepticUpdate(-141418, "1582-10-24", false, true);
+    setDateAndVerifyProlepticUpdate(-141427, "1582-10-15", false, true);
+    setDateAndVerifyProlepticUpdate(-141428, "1582-10-04", false, true);
+    setDateAndVerifyProlepticUpdate(-141430, "1582-10-02", false, true);
+    setDateAndVerifyProlepticUpdate(-141437, "1582-09-25", false, true);
+    setDateAndVerifyProlepticUpdate(-499952, "0601-03-04", false, true);
+    setDateAndVerifyProlepticUpdate(-499955, "0601-03-01", false, true);
+
+    // from proleptic internal representation to hybrid
+    // this way, some string representations will change, as some proleptic dates
+    // (represented as string) don't exist in hybrid calendar, e.g. '1582-10-14'
+    setDateAndVerifyProlepticUpdate(16768, "2015-11-29", true, false);
+    setDateAndVerifyProlepticUpdate(-141418, "1582-10-24", true, false);
+    setDateAndVerifyProlepticUpdate(-141427, "1582-10-15", true, false);
+    setDateAndVerifyProlepticUpdate(-141428, "1582-10-24", true, false); // 1582-10-14 -> 1582-10-24
+    setDateAndVerifyProlepticUpdate(-141430, "1582-10-22", true, false); // 1582-10-12 -> 1582-10-22
+    setDateAndVerifyProlepticUpdate(-141437, "1582-10-15", true, false); // 1582-10-05 -> 1582-10-15
+    setDateAndVerifyProlepticUpdate(-499952, "0601-03-07", true, false);
+    setDateAndVerifyProlepticUpdate(-499955, "0601-03-04", true, false);
+  }
+
+  private void setDateAndVerifyProlepticUpdate(long longDay, String expectedDateString,
+      boolean originalUseProleptic, boolean newUseProleptic) throws Exception {
+
+    DateColumnVector dateColumnVector =
+        new DateColumnVector().setUsingProlepticCalendar(originalUseProleptic);
+    dateColumnVector.vector[0] = longDay;
+
+    dateColumnVector.changeCalendar(newUseProleptic, true);
+
+    Assert.assertEquals("original = " + originalUseProleptic +
+                        " new = " + newUseProleptic,
+                        expectedDateString, dateColumnVector.formatDate(0));
+  }
+}
diff --git a/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampColumnVector.java b/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampColumnVector.java
index 3095114..333a5b5 100644
--- a/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampColumnVector.java
+++ b/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampColumnVector.java
@@ -18,25 +18,41 @@
 
 package org.apache.hadoop.hive.ql.exec.vector;
 
-import org.junit.Test;
+import static org.junit.Assert.assertTrue;
 
-import java.io.PrintWriter;
-import java.math.BigDecimal;
-import java.math.RoundingMode;
 import java.sql.Timestamp;
-import java.util.Date;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.util.GregorianCalendar;
 import java.util.Random;
+import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hive.common.type.RandomTypeUtil;
-
-import static org.junit.Assert.*;
+import org.junit.Assert;
+import org.junit.Test;
 
 /**
  * Test for ListColumnVector
  */
 public class TestTimestampColumnVector {
+  private static final GregorianCalendar PROLEPTIC_GREGORIAN_CALENDAR_UTC =
+      new GregorianCalendar(TimeZone.getTimeZone("UTC"));
+  private static final GregorianCalendar GREGORIAN_CALENDAR_UTC =
+      new GregorianCalendar(TimeZone.getTimeZone("UTC"));
+
+  private static final SimpleDateFormat PROLEPTIC_GREGORIAN_TIMESTAMP_FORMATTER_UTC =
+      new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+  private static final SimpleDateFormat GREGORIAN_TIMESTAMP_FORMATTER_UTC =
+      new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
 
-  private static int TEST_COUNT = 5000;
+  static {
+    PROLEPTIC_GREGORIAN_CALENDAR_UTC.setGregorianChange(new java.util.Date(Long.MIN_VALUE));
+
+    PROLEPTIC_GREGORIAN_TIMESTAMP_FORMATTER_UTC.setCalendar(PROLEPTIC_GREGORIAN_CALENDAR_UTC);
+    GREGORIAN_TIMESTAMP_FORMATTER_UTC.setCalendar(GREGORIAN_CALENDAR_UTC);
+  }
 
   @Test
   public void testSaveAndRetrieve() throws Exception {
@@ -96,20 +112,102 @@ public class TestTimestampColumnVector {
     }
   }
 
-  /*
+  /**
+   * Test case for TimestampColumnVector's changeCalendar
+   *   16768: hybrid: 2015-11-29 proleptic: 2015-11-29
+   * -141418: hybrid: 1582-10-24 proleptic: 1582-10-24
+   * -141427: hybrid: 1582-10-15 proleptic: 1582-10-15
+   * -141428: hybrid: 1582-10-04 proleptic: 1582-10-14
+   * -141430: hybrid: 1582-10-02 proleptic: 1582-10-12
+   * -141437: hybrid: 1582-09-25 proleptic: 1582-10-05
+   * -141438: hybrid: 1582-09-24 proleptic: 1582-10-04
+   * -499952: hybrid: 0601-03-04 proleptic: 0601-03-07
+   * -499955: hybrid: 0601-03-01 proleptic: 0601-03-04
+   */
   @Test
-  public void testGenerate() throws Exception {
-    PrintWriter writer = new PrintWriter("/Users/you/timestamps.txt");
-    Random r = new Random(18485);
-    for (int i = 0; i < 25; i++) {
-      Timestamp randTimestamp = RandomTypeUtil.getRandTimestamp(r);
-      writer.println(randTimestamp.toString());
-    }
-    for (int i = 0; i < 25; i++) {
-      Timestamp randTimestamp = RandomTypeUtil.getRandTimestamp(r, 1965, 2025);
-      writer.println(randTimestamp.toString());
+  public void testProlepticCalendar() {
+    int randomMillis = new Random().nextInt(24 * 60 * 60 * 1000 - 1);
+
+    // from hybrid internal representation to proleptic
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(16768, randomMillis),
+        appendTime("2015-11-29", randomMillis), false, true);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-141418, randomMillis),
+        appendTime("1582-10-24", randomMillis), false, true);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-141427, randomMillis),
+        appendTime("1582-10-15", randomMillis), false, true);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-141428, randomMillis),
+        appendTime("1582-10-04", randomMillis), false, true);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-141430, randomMillis),
+        appendTime("1582-10-02", randomMillis), false, true);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-141437, randomMillis),
+        appendTime("1582-09-25", randomMillis), false, true);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-499952, randomMillis),
+        appendTime("0601-03-04", randomMillis), false, true);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-499955, randomMillis),
+        appendTime("0601-03-01", randomMillis), false, true);
+
+    // from proleptic internal representation to hybrid
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(16768, randomMillis),
+        appendTime("2015-11-29", randomMillis), true, false);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-141418, randomMillis),
+        appendTime("1582-10-24", randomMillis), true, false);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-141427, randomMillis),
+        appendTime("1582-10-15", randomMillis), true, false);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-141428, randomMillis),
+        appendTime("1582-10-24", randomMillis), true, false);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-141430, randomMillis),
+        appendTime("1582-10-22", randomMillis), true, false);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-141437, randomMillis),
+        appendTime("1582-10-15", randomMillis), true, false);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-499952, randomMillis),
+        appendTime("0601-03-07", randomMillis), true, false);
+    setAndVerifyProlepticUpdate(getMillisForDayPlusMillis(-499955, randomMillis),
+        appendTime("0601-03-04", randomMillis), true, false);
+  }
+
+  private long getMillisForDayPlusMillis(int days, int millis) {
+    return TimeUnit.DAYS.toMillis(days) + millis;
+  }
+
+  private String appendTime(String datePart, int millis) {
+    SimpleDateFormat sdf = new SimpleDateFormat("HH:mm:ss.SSS");
+    sdf.setTimeZone(TimeZone.getTimeZone("UTC"));
+
+    return datePart + " " + sdf.format(millis);
+  }
+
+  private void setAndVerifyProlepticUpdate(long epochMilli, String expected,
+      boolean originalUseProleptic, boolean newUseProleptic) {
+
+    DateFormat testFormatter = getTestFormatter(newUseProleptic);
+
+    Instant instant = Instant.ofEpochMilli(epochMilli); // instant is always a moment in UTC
+
+    int nanos = instant.getNano() + new Random().nextInt(999999) + 0;
+    TimestampColumnVector timestampColVector =
+        new TimestampColumnVector().setUsingProlepticCalendar(originalUseProleptic);
+
+    timestampColVector.time[0] = instant.toEpochMilli();
+    timestampColVector.nanos[0] = nanos;
+
+    timestampColVector.changeCalendar(newUseProleptic, true);
+
+    Assert.assertEquals(expected,
+        testFormatter.format(Timestamp.from(Instant.ofEpochMilli(timestampColVector.time[0]))));
+    Assert.assertEquals(nanos, timestampColVector.nanos[0]); // preserving nanos
+  }
+
+  private DateFormat getTestFormatter(boolean useProleptic) {
+    DateFormat testFormatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
+
+    if (useProleptic) {
+      testFormatter.setCalendar(PROLEPTIC_GREGORIAN_CALENDAR_UTC);
+    } else {
+      testFormatter.setCalendar(GREGORIAN_CALENDAR_UTC);
     }
-    writer.close();
+
+    testFormatter.setLenient(false);
+
+    return testFormatter;
   }
-  */
 }