You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@orc.apache.org by om...@apache.org on 2018/03/02 18:50:53 UTC

orc git commit: ORC-306 Correct pre-1970 timestamps that were off by one second.

Repository: orc
Updated Branches:
  refs/heads/master 51b6b6ce3 -> 9c105b92a


ORC-306 Correct pre-1970 timestamps that were off by one second.

Fixes #220

Signed-off-by: Owen O'Malley <om...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/orc/repo
Commit: http://git-wip-us.apache.org/repos/asf/orc/commit/9c105b92
Tree: http://git-wip-us.apache.org/repos/asf/orc/tree/9c105b92
Diff: http://git-wip-us.apache.org/repos/asf/orc/diff/9c105b92

Branch: refs/heads/master
Commit: 9c105b92a0c2ab9c624b7bffd3c8b3a91d892175
Parents: 51b6b6c
Author: Owen O'Malley <om...@apache.org>
Authored: Mon Feb 26 15:27:52 2018 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Fri Mar 2 10:49:30 2018 -0800

----------------------------------------------------------------------
 .../org/apache/orc/impl/TreeReaderFactory.java  | 12 ++--
 .../orc/impl/writer/TimestampTreeWriter.java    | 31 ++++++----
 .../test/org/apache/orc/TestVectorOrcFile.java  | 59 +++++++++++---------
 3 files changed, 60 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/orc/blob/9c105b92/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java b/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java
index 1891737..b33ad11 100644
--- a/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java
+++ b/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java
@@ -19,6 +19,7 @@ package org.apache.orc.impl;
 
 import java.io.EOFException;
 import java.io.IOException;
+import java.sql.Timestamp;
 import java.text.DateFormat;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
@@ -49,7 +50,6 @@ import org.apache.orc.impl.writer.TimestampTreeWriter;
  * Factory for creating ORC tree readers.
  */
 public class TreeReaderFactory {
-
   public interface Context {
     SchemaEvolution getSchemaEvolution();
 
@@ -977,12 +977,12 @@ public class TreeReaderFactory {
 
       for (int i = 0; i < batchSize; i++) {
         if (result.noNulls || !result.isNull[i]) {
-          long millis = data.next() + base_timestamp;
-          int newNanos = parseNanos(nanos.next());
-          if (millis < 0 && newNanos != 0) {
-            millis -= 1;
+          final int newNanos = parseNanos(nanos.next());
+          long millis = (data.next() + base_timestamp)
+              * TimestampTreeWriter.MILLIS_PER_SECOND + newNanos / 1_000_000;
+          if (millis < 0 && newNanos > 999_999) {
+            millis -= TimestampTreeWriter.MILLIS_PER_SECOND;
           }
-          millis *= TimestampTreeWriter.MILLIS_PER_SECOND;
           long offset = 0;
           // If reader and writer time zones have different rules, adjust the timezone difference
           // between reader and writer taking day light savings into account.

http://git-wip-us.apache.org/repos/asf/orc/blob/9c105b92/java/core/src/java/org/apache/orc/impl/writer/TimestampTreeWriter.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/writer/TimestampTreeWriter.java b/java/core/src/java/org/apache/orc/impl/writer/TimestampTreeWriter.java
index fae108e..1694ca1 100644
--- a/java/core/src/java/org/apache/orc/impl/writer/TimestampTreeWriter.java
+++ b/java/core/src/java/org/apache/orc/impl/writer/TimestampTreeWriter.java
@@ -75,11 +75,16 @@ public class TimestampTreeWriter extends TreeWriterBase {
                          int length) throws IOException {
     super.writeBatch(vector, offset, length);
     TimestampColumnVector vec = (TimestampColumnVector) vector;
-    Timestamp val;
     if (vector.isRepeating) {
       if (vector.noNulls || !vector.isNull[0]) {
-        val = vec.asScratchTimestamp(0);
-        long millis = val.getTime();
+        // ignore the bottom three digits from the vec.time field
+        final long secs = vec.time[0] / MILLIS_PER_SECOND;
+        final int newNanos = vec.nanos[0];
+        // set the millis based on the top three digits of the nanos
+        long millis = secs * MILLIS_PER_SECOND + newNanos / 1_000_000;
+        if (millis < 0 && newNanos > 999_999) {
+          millis -= MILLIS_PER_SECOND;
+        }
         long utc = SerializationUtils.convertToUtc(localTimezone, millis);
         indexStatistics.updateTimestamp(utc);
         if (createBloomFilter) {
@@ -88,22 +93,26 @@ public class TimestampTreeWriter extends TreeWriterBase {
           }
           bloomFilterUtf8.addLong(utc);
         }
-        final long secs = millis / MILLIS_PER_SECOND - baseEpochSecsLocalTz;
-        final long nano = formatNanos(val.getNanos());
+        final long nano = formatNanos(vec.nanos[0]);
         for (int i = 0; i < length; ++i) {
-          seconds.write(secs);
+          seconds.write(secs - baseEpochSecsLocalTz);
           nanos.write(nano);
         }
       }
     } else {
       for (int i = 0; i < length; ++i) {
         if (vec.noNulls || !vec.isNull[i + offset]) {
-          val = vec.asScratchTimestamp(i + offset);
-          long millis = val.getTime();
-          long secs = millis / MILLIS_PER_SECOND - baseEpochSecsLocalTz;
+          // ignore the bottom three digits from the vec.time field
+          final long secs = vec.time[i + offset] / MILLIS_PER_SECOND;
+          final int newNanos = vec.nanos[i + offset];
+          // set the millis based on the top three digits of the nanos
+          long millis = secs * MILLIS_PER_SECOND + newNanos / 1_000_000;
+          if (millis < 0 && newNanos > 999_999) {
+            millis -= MILLIS_PER_SECOND;
+          }
           long utc = SerializationUtils.convertToUtc(localTimezone, millis);
-          seconds.write(secs);
-          nanos.write(formatNanos(val.getNanos()));
+          seconds.write(secs - baseEpochSecsLocalTz);
+          nanos.write(formatNanos(newNanos));
           indexStatistics.updateTimestamp(utc);
           if (createBloomFilter) {
             if (bloomFilter != null) {

http://git-wip-us.apache.org/repos/asf/orc/blob/9c105b92/java/core/src/test/org/apache/orc/TestVectorOrcFile.java
----------------------------------------------------------------------
diff --git a/java/core/src/test/org/apache/orc/TestVectorOrcFile.java b/java/core/src/test/org/apache/orc/TestVectorOrcFile.java
index 3992d4e..b254fb1 100644
--- a/java/core/src/test/org/apache/orc/TestVectorOrcFile.java
+++ b/java/core/src/test/org/apache/orc/TestVectorOrcFile.java
@@ -1362,18 +1362,34 @@ public class TestVectorOrcFile {
             .blockPadding(false));
     VectorizedRowBatch batch = schema.createRowBatch();
     batch.size = 1000;
+    TimestampColumnVector timestampColVector = (TimestampColumnVector) batch.cols[0];
     for (int year = minYear; year < maxYear; ++year) {
-      for (int ms = 1000; ms < 2000; ++ms) {
-        TimestampColumnVector timestampColVector = (TimestampColumnVector) batch.cols[0];
-        timestampColVector.set(ms - 1000,
-            Timestamp.valueOf(year +
-                "-05-05 12:34:56." + ms));
-        ((LongColumnVector) batch.cols[1]).vector[ms - 1000] =
-            new DateWritable(new Date(year - 1900, 11, 25)).getDays();
+      for (int row = 0; row < 1000; ++row) {
+        String timeStr = String.format("%04d-05-05 12:34:56.%04d", year, 2*row);
+        timestampColVector.set(row, Timestamp.valueOf(timeStr));
       }
+      ((LongColumnVector) batch.cols[1]).vector[0] =
+          new DateWritable(new Date(year - 1900, 11, 25)).getDays();
+      batch.cols[1].isRepeating = true;
       writer.addRowBatch(batch);
     }
+
+    // add one more row to check the statistics for the jvm bug case
+    batch.size = 1;
+    String timeStr = String.format("%04d-12-12 12:34:56.0001", maxYear-1);
+    timestampColVector.set(0, Timestamp.valueOf(timeStr));
+    writer.addRowBatch(batch);
     writer.close();
+
+    // check the stats to make sure they match up to the millisecond
+    ColumnStatistics[] stats = writer.getStatistics();
+    TimestampColumnStatistics tsStat = (TimestampColumnStatistics) stats[1];
+    assertEquals(String.format("%04d-12-12 12:34:56.0", maxYear - 1),
+        tsStat.getMaximum().toString());
+    assertEquals(String.format("%04d-05-05 12:34:56.0", minYear),
+        tsStat.getMinimum().toString());
+
+    // read back the rows
     Reader reader = OrcFile.createReader(file,
         OrcFile.readerOptions(conf));
     RecordReader rows = reader.rows();
@@ -1383,27 +1399,20 @@ public class TestVectorOrcFile {
     for (int year = minYear; year < maxYear; ++year) {
       rows.nextBatch(batch);
       assertEquals(1000, batch.size);
-      for(int ms = 1000; ms < 2000; ++ms) {
-        StringBuilder buffer = new StringBuilder();
-        times.stringifyValue(buffer, ms - 1000);
-        String expected = Integer.toString(year) + "-05-05 12:34:56.";
-        // suppress the final zeros on the string by dividing by the largest
-        // power of 10 that divides evenly.
-        int roundedMs = ms;
-        for(int round = 1000; round > 0; round /= 10) {
-          if (ms % round == 0) {
-            roundedMs = ms / round;
-            break;
-          }
-        }
-        expected += roundedMs;
-        assertEquals(expected, buffer.toString());
-        assertEquals(Integer.toString(year) + "-12-25",
-            new DateWritable((int) dates.vector[ms - 1000]).toString());
+      for(int row = 0; row < 1000; ++row) {
+        Timestamp expected = Timestamp.valueOf(
+            String.format("%04d-05-05 12:34:56.%04d", year, 2*row));
+        assertEquals("ms row " + row + " " + expected, expected.getTime(),
+            times.time[row]);
+        assertEquals("nanos row " + row + " " + expected, expected.getNanos(),
+            times.nanos[row]);
+        assertEquals("year " + year + " row " + row,
+            Integer.toString(year) + "-12-25",
+            new DateWritable((int) dates.vector[row]).toString());
       }
     }
     rows.nextBatch(batch);
-    assertEquals(0, batch.size);
+    assertEquals(1, batch.size);
   }
 
   @Test