You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@orc.apache.org by jc...@apache.org on 2019/09/24 22:13:21 UTC

[orc] branch branch-1.6 updated: ORC-550: Use default system time zone if it is not present in the file footer

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

jcamacho pushed a commit to branch branch-1.6
in repository https://gitbox.apache.org/repos/asf/orc.git


The following commit(s) were added to refs/heads/branch-1.6 by this push:
     new 0214376  ORC-550: Use default system time zone if it is not present in the file footer
0214376 is described below

commit 0214376dbbb613df3f25046707f31b5d2477a5ce
Author: Jesus Camacho Rodriguez <jc...@apache.org>
AuthorDate: Tue Sep 3 16:11:04 2019 -0700

    ORC-550: Use default system time zone if it is not present in the file footer
    
    Fixes #427
    
    Signed-off-by: Jesus Camacho Rodriguez <jc...@apache.org>
---
 .../org/apache/orc/impl/TreeReaderFactory.java     |   8 +-
 .../test/org/apache/orc/TestOrcDSTNoTimezone.java  | 118 ++++++++++++++++++++
 .../src/test/org/apache/orc/TestOrcNoTimezone.java | 119 +++++++++++++++++++++
 .../test/resources/orc-file-dst-no-timezone.orc    | Bin 0 -> 217 bytes
 .../src/test/resources/orc-file-no-timezone.orc    | Bin 0 -> 217 bytes
 5 files changed, 243 insertions(+), 2 deletions(-)

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 4ec7f61..270bd6c 100644
--- a/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java
+++ b/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java
@@ -919,7 +919,11 @@ public class TreeReaderFactory {
         this.readerTimeZone = TimeZone.getDefault();
       }
       if (context.getWriterTimezone() == null || context.getWriterTimezone().isEmpty()) {
-        this.base_timestamp = getBaseTimestamp(readerTimeZone.getID());
+        if (instantType) {
+          this.base_timestamp = getBaseTimestamp(readerTimeZone.getID()); // UTC
+        } else {
+          this.base_timestamp = getBaseTimestamp(TimeZone.getDefault().getID());
+        }
       } else {
         this.base_timestamp = getBaseTimestamp(context.getWriterTimezone());
       }
@@ -963,7 +967,7 @@ public class TreeReaderFactory {
     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();
+        timeZoneId = writerTimeZone.getID();
       }
 
       if (writerTimeZone == null || !timeZoneId.equals(writerTimeZone.getID())) {
diff --git a/java/core/src/test/org/apache/orc/TestOrcDSTNoTimezone.java b/java/core/src/test/org/apache/orc/TestOrcDSTNoTimezone.java
new file mode 100644
index 0000000..7730aa5
--- /dev/null
+++ b/java/core/src/test/org/apache/orc/TestOrcDSTNoTimezone.java
@@ -0,0 +1,118 @@
+/**
+ * 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 java.sql.Timestamp;
+import java.text.SimpleDateFormat;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+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 static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Test over an orc file that does not store time zone information in the footer
+ * and it was written from a time zone that observes DST for one of the timestamp
+ * values stored ('2014-06-06 12:34:56.0').
+ */
+@RunWith(Parameterized.class)
+public class TestOrcDSTNoTimezone {
+  Configuration conf;
+  FileSystem fs;
+  String readerTimeZone;
+  SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.S");
+  static TimeZone defaultTimeZone = TimeZone.getDefault();
+
+  public TestOrcDSTNoTimezone(String readerTZ) {
+    this.readerTimeZone = readerTZ;
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    List<Object[]> result = Arrays.asList(new Object[][]{
+        {"America/Los_Angeles"},
+        {"Europe/Berlin"},
+        {"Asia/Jerusalem"}
+    });
+    return result;
+  }
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+  }
+
+  @After
+  public void restoreTimeZone() {
+    TimeZone.setDefault(defaultTimeZone);
+  }
+
+  @Test
+  public void testReadOldTimestampFormat() throws Exception {
+    TimeZone.setDefault(TimeZone.getTimeZone(readerTimeZone));
+    Path oldFilePath = new Path(getClass().getClassLoader().
+        getSystemResource("orc-file-dst-no-timezone.orc").getPath());
+    Reader reader = OrcFile.createReader(oldFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs).useUTCTimestamp(true));
+    formatter.setTimeZone(TimeZone.getTimeZone("UTC"));
+    TypeDescription schema = reader.getSchema();
+    VectorizedRowBatch batch = schema.createRowBatch(10);
+    TimestampColumnVector ts = (TimestampColumnVector) batch.cols[0];
+
+    boolean[] include = new boolean[schema.getMaximumId() + 1];
+    include[schema.getChildren().get(0).getId()] = true;
+    RecordReader rows = reader.rows
+        (reader.options().include(include));
+    assertTrue(rows.nextBatch(batch));
+    Timestamp timestamp = ts.asScratchTimestamp(0);
+    assertEquals(Timestamp.valueOf("2014-01-01 12:34:56.0").toString(),
+        formatter.format(timestamp));
+
+    // check the contents of second row
+    rows.seekToRow(1);
+    assertTrue(rows.nextBatch(batch));
+    assertEquals(1, batch.size);
+    timestamp = ts.asScratchTimestamp(0);
+    assertEquals(Timestamp.valueOf("2014-06-06 12:34:56.0").toString(),
+        formatter.format(timestamp));
+
+    // handle the close up
+    assertFalse(rows.nextBatch(batch));
+    rows.close();
+  }
+}
diff --git a/java/core/src/test/org/apache/orc/TestOrcNoTimezone.java b/java/core/src/test/org/apache/orc/TestOrcNoTimezone.java
new file mode 100644
index 0000000..48208bb
--- /dev/null
+++ b/java/core/src/test/org/apache/orc/TestOrcNoTimezone.java
@@ -0,0 +1,119 @@
+/**
+ * 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 java.sql.Timestamp;
+import java.text.SimpleDateFormat;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+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 static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Test over an orc file that does not store time zone information in the footer
+ * and it was written from a time zone that does not observe DST.
+ */
+@RunWith(Parameterized.class)
+public class TestOrcNoTimezone {
+  Configuration conf;
+  FileSystem fs;
+  String readerTimeZone;
+  SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.S");
+  static TimeZone defaultTimeZone = TimeZone.getDefault();
+
+  public TestOrcNoTimezone(String readerTZ) {
+    this.readerTimeZone = readerTZ;
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    List<Object[]> result = Arrays.asList(new Object[][]{
+        {"GMT-12:00"},
+        {"UTC"},
+        {"GMT+8:00"},
+    });
+    return result;
+  }
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+  }
+
+  @After
+  public void restoreTimeZone() {
+    TimeZone.setDefault(defaultTimeZone);
+  }
+
+  @Test
+  public void testReadOldTimestampFormat() throws Exception {
+    TimeZone.setDefault(TimeZone.getTimeZone(readerTimeZone));
+    Path oldFilePath = new Path(getClass().getClassLoader().
+        getSystemResource("orc-file-no-timezone.orc").getPath());
+    Reader reader = OrcFile.createReader(oldFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs).useUTCTimestamp(true));
+    formatter.setTimeZone(TimeZone.getTimeZone("UTC"));
+    TypeDescription schema = reader.getSchema();
+    VectorizedRowBatch batch = schema.createRowBatch(10);
+    TimestampColumnVector ts = (TimestampColumnVector) batch.cols[0];
+
+    boolean[] include = new boolean[schema.getMaximumId() + 1];
+    include[schema.getChildren().get(0).getId()] = true;
+    RecordReader rows = reader.rows
+        (reader.options().include(include));
+    assertTrue(rows.nextBatch(batch));
+    Timestamp timestamp = ts.asScratchTimestamp(0);
+    assertEquals("For timezone : " + TimeZone.getTimeZone(readerTimeZone),
+        Timestamp.valueOf("2014-01-01 12:34:56.0").toString(),
+        formatter.format(timestamp));
+
+    // check the contents of second row
+    rows.seekToRow(1);
+    assertTrue(rows.nextBatch(batch));
+    assertEquals(1, batch.size);
+    timestamp = ts.asScratchTimestamp(0);
+    assertEquals("For timezone : " + TimeZone.getTimeZone(readerTimeZone),
+        Timestamp.valueOf("2014-06-06 12:34:56.0").toString(),
+        formatter.format(timestamp));
+
+    // handle the close up
+    assertFalse(rows.nextBatch(batch));
+    rows.close();
+  }
+}
diff --git a/java/core/src/test/resources/orc-file-dst-no-timezone.orc b/java/core/src/test/resources/orc-file-dst-no-timezone.orc
new file mode 100644
index 0000000..c836050
Binary files /dev/null and b/java/core/src/test/resources/orc-file-dst-no-timezone.orc differ
diff --git a/java/core/src/test/resources/orc-file-no-timezone.orc b/java/core/src/test/resources/orc-file-no-timezone.orc
new file mode 100644
index 0000000..18d69ea
Binary files /dev/null and b/java/core/src/test/resources/orc-file-no-timezone.orc differ