You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@orc.apache.org by do...@apache.org on 2021/12/03 07:30:06 UTC

[orc] branch main updated: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC (#967)

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

dongjoon pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/orc.git


The following commit(s) were added to refs/heads/main by this push:
     new e787b8b  ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC (#967)
e787b8b is described below

commit e787b8b78555f11c93ee89181189c88d55a8bbdc
Author: Yiqun Zhang <gu...@gmail.com>
AuthorDate: Fri Dec 3 15:29:58 2021 +0800

    ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC (#967)
    
    ### What changes were proposed in this pull request?
    
    ```java
    // use tool compute offset : 17762
    int toolOffset = ((LocalDateTime) temporalAccessor).atZone(TimeZone.getTimeZone("America/New_York").toZoneId()).getOffset().getTotalSeconds();
    
    // in orc internal compute offset: 18000
    int orcInternalOffset = TimeZone.getTimeZone("America/New_York").getRawOffset() / 1000
    ```
    
    This pr is designed to modify the implementation of the LocalDateTime to Timestamp conversion so that the time zone accuracy is consistent with the ORC internal accuracy during the conversion
    
    ### Why are the changes needed?
    Avoid inconsistencies between converted data and expected data from convert tools.
    
    
    ### How was this patch tested?
    Add issue-specific unit test
---
 .../org/apache/orc/tools/convert/CsvReader.java    |   4 +-
 .../org/apache/orc/tools/convert/TestConvert.java  | 103 +++++++++++++++++++++
 2 files changed, 104 insertions(+), 3 deletions(-)

diff --git a/java/tools/src/java/org/apache/orc/tools/convert/CsvReader.java b/java/tools/src/java/org/apache/orc/tools/convert/CsvReader.java
index 9234ab4..e708416 100644
--- a/java/tools/src/java/org/apache/orc/tools/convert/CsvReader.java
+++ b/java/tools/src/java/org/apache/orc/tools/convert/CsvReader.java
@@ -43,7 +43,6 @@ import java.sql.Timestamp;
 import java.time.LocalDate;
 import java.time.LocalDateTime;
 import java.time.OffsetDateTime;
-import java.time.ZoneId;
 import java.time.ZonedDateTime;
 import java.time.format.DateTimeFormatter;
 import java.time.temporal.TemporalAccessor;
@@ -303,8 +302,7 @@ public class CsvReader implements RecordReader {
           Timestamp timestamp = Timestamp.from(offsetDateTime.toInstant());
           vector.set(row, timestamp);
         } else if (temporalAccessor instanceof LocalDateTime) {
-          ZonedDateTime tz = ((LocalDateTime) temporalAccessor).atZone(ZoneId.systemDefault());
-          Timestamp timestamp = Timestamp.from(tz.toInstant());
+          Timestamp timestamp = Timestamp.valueOf((LocalDateTime) temporalAccessor);
           vector.set(row, timestamp);
         } else {
           column.noNulls = false;
diff --git a/java/tools/src/test/org/apache/orc/tools/convert/TestConvert.java b/java/tools/src/test/org/apache/orc/tools/convert/TestConvert.java
new file mode 100644
index 0000000..84ec101
--- /dev/null
+++ b/java/tools/src/test/org/apache/orc/tools/convert/TestConvert.java
@@ -0,0 +1,103 @@
+/*
+ * 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.tools.convert;
+
+import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+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.apache.orc.OrcFile;
+import org.apache.orc.Reader;
+import org.apache.orc.RecordReader;
+import org.apache.orc.TypeDescription;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+import java.util.TimeZone;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestConvert {
+
+  public static final TimeZone DEFAULT_TIME_ZONE = TimeZone.getDefault();
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir"));
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+
+  @BeforeEach
+  public void openFileSystem () throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    fs.setWorkingDirectory(workDir);
+    testFilePath = new Path("TestConvert.testConvert.orc");
+    fs.delete(testFilePath, false);
+  }
+
+  @BeforeAll
+  public static void changeDefaultTimeZone() {
+    TimeZone.setDefault(TimeZone.getTimeZone("America/New_York"));
+  }
+
+  @AfterAll
+  public static void resetDefaultTimeZone() {
+    TimeZone.setDefault(DEFAULT_TIME_ZONE);
+  }
+
+  @Test
+  public void testConvertCustomTimestampFromCsv() throws IOException, ParseException {
+    Path csvFile = new Path("test.csv");
+    FSDataOutputStream stream = fs.create(csvFile, true);
+    String[] timeValues = new String[] {"0001-01-01 00:00:00.000", "2021-12-01 18:36:00.800"};
+    stream.writeBytes(String.join("\n", timeValues));
+    stream.close();
+    String schema = "struct<d:timestamp>";
+    String timestampFormat = "yyyy-MM-dd HH:mm:ss.SSS";
+    TypeDescription readSchema = TypeDescription.fromString(schema);
+
+    ConvertTool.main(conf, new String[]{"--schema", schema, "-o", testFilePath.toString(),
+        "-t", timestampFormat, csvFile.toString()});
+
+    assertTrue(fs.exists(testFilePath));
+
+    Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf));
+    VectorizedRowBatch batch = readSchema.createRowBatch();
+    RecordReader rowIterator = reader.rows(reader.options().schema(readSchema));
+    TimestampColumnVector tcv = (TimestampColumnVector) batch.cols[0];
+
+    while (rowIterator.nextBatch(batch)) {
+      for (int row = 0; row < batch.size; ++row) {
+        Timestamp timestamp = Timestamp.valueOf(timeValues[row]);
+        assertEquals(timestamp.getTime(), tcv.time[row]);
+        assertEquals(timestamp.getNanos(), tcv.nanos[row]);
+      }
+    }
+    rowIterator.close();
+  }
+
+}