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 2016/08/28 02:37:16 UTC

[2/2] orc git commit: HIVE-14566: LLAP IO reads timestamp wrongly (Prasanth Jayachandran reviewed by Sergey Shelukhin)

HIVE-14566: LLAP IO reads timestamp wrongly (Prasanth Jayachandran reviewed by Sergey Shelukhin)

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/a0606e46
Tree: http://git-wip-us.apache.org/repos/asf/orc/tree/a0606e46
Diff: http://git-wip-us.apache.org/repos/asf/orc/diff/a0606e46

Branch: refs/heads/branch-1.2
Commit: a0606e468d4dc056efc9da4a411f0be4d338d72d
Parents: fcc6859
Author: Owen O'Malley <om...@apache.org>
Authored: Sat Aug 27 19:34:35 2016 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Sat Aug 27 19:34:35 2016 -0700

----------------------------------------------------------------------
 .../org/apache/orc/impl/TreeReaderFactory.java  |   5 +-
 .../test/org/apache/orc/TestOrcTimezone3.java   | 126 +++++++++++++++++++
 2 files changed, 129 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/orc/blob/a0606e46/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 0c94f0e..1e1ae01 100644
--- a/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java
+++ b/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java
@@ -863,7 +863,7 @@ public class TreeReaderFactory {
     protected IntegerReader nanos = null;
     private final boolean skipCorrupt;
     private Map<String, Long> baseTimestampMap;
-    private long base_timestamp;
+    protected long base_timestamp;
     private final TimeZone readerTimeZone;
     private TimeZone writerTimeZone;
     private boolean hasSameTZRules;
@@ -896,6 +896,7 @@ public class TreeReaderFactory {
         if (nanosStream != null) {
           this.nanos = createIntegerReader(encoding.getKind(), nanosStream, false, context);
         }
+        base_timestamp = getBaseTimestamp(context.getWriterTimezone());
       }
     }
 
@@ -922,7 +923,7 @@ public class TreeReaderFactory {
       base_timestamp = getBaseTimestamp(stripeFooter.getWriterTimezone());
     }
 
-    private long getBaseTimestamp(String timeZoneId) throws IOException {
+    protected long getBaseTimestamp(String timeZoneId) throws IOException {
       // to make sure new readers read old files in the same way
       if (timeZoneId == null || timeZoneId.isEmpty()) {
         timeZoneId = readerTimeZone.getID();

http://git-wip-us.apache.org/repos/asf/orc/blob/a0606e46/java/core/src/test/org/apache/orc/TestOrcTimezone3.java
----------------------------------------------------------------------
diff --git a/java/core/src/test/org/apache/orc/TestOrcTimezone3.java b/java/core/src/test/org/apache/orc/TestOrcTimezone3.java
new file mode 100644
index 0000000..40ab0c9
--- /dev/null
+++ b/java/core/src/test/org/apache/orc/TestOrcTimezone3.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.orc;
+
+import static junit.framework.Assert.assertEquals;
+
+import java.io.File;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.TimeZone;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import com.google.common.collect.Lists;
+
+import junit.framework.Assert;
+
+/**
+ *
+ */
+@RunWith(Parameterized.class)
+public class TestOrcTimezone3 {
+  Path workDir = new Path(System.getProperty("test.tmp.dir",
+      "target" + File.separator + "test" + File.separator + "tmp"));
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+  String writerTimeZone;
+  String readerTimeZone;
+  static TimeZone defaultTimeZone = TimeZone.getDefault();
+
+  public TestOrcTimezone3(String writerTZ, String readerTZ) {
+    this.writerTimeZone = writerTZ;
+    this.readerTimeZone = readerTZ;
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    List<Object[]> result = Arrays.asList(new Object[][]{
+        {"America/Chicago", "America/Los_Angeles"},
+    });
+    return result;
+  }
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    testFilePath = new Path(workDir, "TestOrcTimezone3." +
+        testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  @After
+  public void restoreTimeZone() {
+    TimeZone.setDefault(defaultTimeZone);
+  }
+
+  @Test
+  public void testTimestampWriter() throws Exception {
+    TypeDescription schema = TypeDescription.createTimestamp();
+
+    TimeZone.setDefault(TimeZone.getTimeZone(writerTimeZone));
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema).stripeSize(100000)
+            .bufferSize(10000));
+    assertEquals(writerTimeZone, TimeZone.getDefault().getID());
+    List<String> ts = Lists.newArrayList();
+    ts.add("1969-12-31 16:00:14.007");
+    ts.add("1969-12-31 16:00:06.021");
+    ts.add("1969-12-31 16:00:03.963");
+    VectorizedRowBatch batch = schema.createRowBatch();
+    TimestampColumnVector times = (TimestampColumnVector) batch.cols[0];
+    for (String t : ts) {
+      times.set(batch.size++, Timestamp.valueOf(t));
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    TimeZone.setDefault(TimeZone.getTimeZone(readerTimeZone));
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    assertEquals(readerTimeZone, TimeZone.getDefault().getID());
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    times = (TimestampColumnVector) batch.cols[0];
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(ts.get(idx++), times.asScratchTimestamp(r).toString());
+      }
+    }
+    rows.close();
+  }
+}