You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@orc.apache.org by GitBox <gi...@apache.org> on 2021/12/01 10:51:09 UTC

[GitHub] [orc] guiyanakuang opened a new pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

guiyanakuang opened a new pull request #967:
URL: https://github.com/apache/orc/pull/967


   ### What changes were proposed in this pull request?
   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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] guiyanakuang commented on pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
guiyanakuang commented on pull request #967:
URL: https://github.com/apache/orc/pull/967#issuecomment-984469903


   > Also, cc @wgtmac since C++ tool seems to have the same issue according to the JIRA.
   
   I have also debugged some of the C++ code. It looks like `strptime` can't convert values before 1970 on mac os.
   https://github.com/apache/orc/blob/334bf1f2c605f38c7e75ec81d1dab93c31fc8459/tools/src/CSVFileImport.cc#L257


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #967:
URL: https://github.com/apache/orc/pull/967#discussion_r760422805



##########
File path: java/tools/src/test/org/apache/orc/tools/convert/TestConvert.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestConvert {
+
+  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);
+  }
+
+  @Test
+  public void testConvertCustomTimestampFromCsv() throws IOException, ParseException {

Review comment:
       Thank you for adding this.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #967:
URL: https://github.com/apache/orc/pull/967#issuecomment-985299468


   Thank you!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] guiyanakuang commented on a change in pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
guiyanakuang commented on a change in pull request #967:
URL: https://github.com/apache/orc/pull/967#discussion_r760725574



##########
File path: java/tools/src/test/org/apache/orc/tools/convert/TestConvert.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestConvert {
+
+  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);
+  }
+
+  @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];
+    rowIterator.nextBatch(batch);
+
+    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]);

Review comment:
       ```java
       Timestamp timestamp1 = Timestamp.from(localDateTime.toInstant());
       Timestamp timestamp2 = Timestamp.valueOf(localDateTime);
   ```
   Both methods use the local time zone to calculate the timestamp, but the offset precision of the time zone of the previous method is seconds (for New York is 17762 s), and the precision of the latter method is hours (for New York is 18000 s, which is 5 hours)
   And the readings are interpreted by the latter precision (hour)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #967:
URL: https://github.com/apache/orc/pull/967#discussion_r760886324



##########
File path: java/tools/src/test/org/apache/orc/tools/convert/TestConvert.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestConvert {
+
+  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);
+  }
+
+  @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];
+    rowIterator.nextBatch(batch);
+
+    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]);

Review comment:
       I ran this yesterday and today. It passed on my laptop without your patch.
   ```
   $ git diff main --stat
    java/tools/src/test/org/apache/orc/tools/convert/TestConvert.java | 89 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
    1 file changed, 89 insertions(+)
   
   $ mvn package -pl tools -Dtest=org.apache.orc.tools.convert.TestConvert
   ...
   [INFO] -------------------------------------------------------
   [INFO]  T E S T S
   [INFO] -------------------------------------------------------
   [INFO] Running org.apache.orc.tools.convert.TestConvert
   Processing test.csv
   [INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.337 s - in org.apache.orc.tools.convert.TestConvert
   [INFO]
   [INFO] Results:
   [INFO]
   [INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0
   ...
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] wgtmac commented on pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
wgtmac commented on pull request #967:
URL: https://github.com/apache/orc/pull/967#issuecomment-985371206


   > > Also, cc @wgtmac since C++ tool seems to have the same issue according to the JIRA.
   > 
   > I have also debugged some of the C++ code. It looks like `strptime` can't convert values before 1970 on mac os.
   > 
   > https://github.com/apache/orc/blob/334bf1f2c605f38c7e75ec81d1dab93c31fc8459/tools/src/CSVFileImport.cc#L257
   
   Pre-1970 timestamp is a notorious issue. Not sure if these test cases will help: 
   https://github.com/apache/orc/blob/main/c%2B%2B/test/TestWriter.cc#L621
   https://github.com/apache/orc/blob/main/c%2B%2B/test/TestWriter.cc#L741


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] guiyanakuang commented on pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
guiyanakuang commented on pull request #967:
URL: https://github.com/apache/orc/pull/967#issuecomment-985290716


   > +1, LGTM. Thank you, @guiyanakuang . For C++ case, could you file a new JIRA for that? [ORC-1053](https://issues.apache.org/jira/browse/ORC-1053) is enough for Java-only patch
   
   Okay, I created the ORC-1055


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] guiyanakuang commented on a change in pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
guiyanakuang commented on a change in pull request #967:
URL: https://github.com/apache/orc/pull/967#discussion_r760640323



##########
File path: java/tools/src/test/org/apache/orc/tools/convert/TestConvert.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestConvert {
+
+  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);
+  }
+
+  @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];
+    rowIterator.nextBatch(batch);
+
+    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]);

Review comment:
       I verified before submitting that my local default timezone, and the America/New_York timezone on which jira uploaded the file, would not pass the test based on past code.
   
   The reason for this is the inconsistent interpretation of the timestamp offfset accuracy when writing and reading.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] guiyanakuang commented on a change in pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
guiyanakuang commented on a change in pull request #967:
URL: https://github.com/apache/orc/pull/967#discussion_r760927707



##########
File path: java/tools/src/test/org/apache/orc/tools/convert/TestConvert.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestConvert {
+
+  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);
+  }
+
+  @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];
+    rowIterator.nextBatch(batch);
+
+    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]);

Review comment:
       @dongjoon-hyun I added two hooks using annotations to ensure that the default time zone during testing this class is New York, which feels good. Also I removed a redundant statement :sweat_smile:. 
   You can try again when you have time, I'm sure the old code will not pass the test.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] guiyanakuang commented on pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
guiyanakuang commented on pull request #967:
URL: https://github.com/apache/orc/pull/967#issuecomment-983521971


   cc @dongjoon-hyun  @williamhyun  : )


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #967:
URL: https://github.com/apache/orc/pull/967#issuecomment-985280738


   Merged to main/1.7.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] guiyanakuang commented on a change in pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
guiyanakuang commented on a change in pull request #967:
URL: https://github.com/apache/orc/pull/967#discussion_r760890059



##########
File path: java/tools/src/test/org/apache/orc/tools/convert/TestConvert.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestConvert {
+
+  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);
+  }
+
+  @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];
+    rowIterator.nextBatch(batch);
+
+    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]);

Review comment:
       Maybe I should specify the time zone to test, but there are many methods inside ORC to get the local default time zone, so it's not good to cover it uniformly at the moment, I need to think.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] wgtmac commented on pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
wgtmac commented on pull request #967:
URL: https://github.com/apache/orc/pull/967#issuecomment-986385432


   > @wgtmac , Unfortunately I think this is a new issue.
   > 
   > ```c++
   > int main(int,char* argv[]){
   >   struct tm timeStruct;
   >   std::string name = "0001-01-01 00:00:00.000";
   >   char *left=strptime(name.c_str(), "%Y-%m-%d %H:%M:%S", &timeStruct);
   >   time_t _time = timegm(&timeStruct);
   >   char *tail;
   >   double d = strtod(left, &tail);
   >   long na;
   >   if (tail != left) {
   >     na = static_cast<long>(d * 1000000000.0);
   >   } else {
   >     na = 0;
   >   }
   >   std::cout << _time << " " << na << std::endl;
   > }
   > ```
   > 
   > On my mac machine the output is "-1 0" Using the online C++ runtime environment claimed on Linux, the output is "-62135596800 0" Time format strings that appear to be older than 1970 get -1 on the mac.
   > 
   > > https://github.com/apache/orc/blob/main/c%2B%2B/test/TestWriter.cc#L621
   > > https://github.com/apache/orc/blob/main/c%2B%2B/test/TestWriter.cc#L741
   > 
   > It looks like the first test verifies that we can write time as a negative number. The second one holds our custom time zone. So I think this is a new issue.
   > 
   > Of course I verified the `converted_by_cpp.orc` uploaded by the user in [ORC-1055](https://issues.apache.org/jira/browse/ORC-1055), and it is clear that it is not converted on the mac, so there should be another issue that is not yet clear
   
   Thanks for the input. I will take a look later this week.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #967:
URL: https://github.com/apache/orc/pull/967#discussion_r760433247



##########
File path: java/tools/src/test/org/apache/orc/tools/convert/TestConvert.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestConvert {
+
+  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);
+  }
+
+  @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];
+    rowIterator.nextBatch(batch);
+
+    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]);

Review comment:
       This seems to pass without your patch. Could you verify this again?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #967:
URL: https://github.com/apache/orc/pull/967#issuecomment-983884605


   Also, cc @wgtmac since C++ tool seems to have the same issue according to the JIRA.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #967:
URL: https://github.com/apache/orc/pull/967#issuecomment-983842988


   Thank you so much, @guiyanakuang !


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun merged pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun merged pull request #967:
URL: https://github.com/apache/orc/pull/967


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] guiyanakuang commented on pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
guiyanakuang commented on pull request #967:
URL: https://github.com/apache/orc/pull/967#issuecomment-985389979


   @wgtmac , Unfortunately I think this is a new issue.
   ```c++
   int main(int,char* argv[]){
     struct tm timeStruct;
     std::string name = "0001-01-01 00:00:00.000";
     char *left=strptime(name.c_str(), "%Y-%m-%d %H:%M:%S", &timeStruct);
     time_t _time = timegm(&timeStruct);
     char *tail;
     double d = strtod(left, &tail);
     long na;
     if (tail != left) {
       na = static_cast<long>(d * 1000000000.0);
     } else {
       na = 0;
     }
     std::cout << _time << " " << na << std::endl;
   }
   ```
   On my mac machine the output is "-1 0"
   Using the online C++ runtime environment claimed on Linux, the output is "-62135596800 0"
   Time format strings that appear to be older than 1970 get -1 on the mac.
   
   > https://github.com/apache/orc/blob/main/c%2B%2B/test/TestWriter.cc#L621
   > https://github.com/apache/orc/blob/main/c%2B%2B/test/TestWriter.cc#L741
   
   It looks like the first test verifies that we can write time as a negative number.
   The second one holds our custom time zone.  So I think this is a new issue.
   
   Of course I verified the `converted_by_cpp.orc` uploaded by the user in ORC-1055, and it is clear that it is not converted on the mac, so there should be another issue that is not yet clear


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #967:
URL: https://github.com/apache/orc/pull/967#issuecomment-986392554


   Thank you so much, @guiyanakuang and @wgtmac .


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] guiyanakuang commented on a change in pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
guiyanakuang commented on a change in pull request #967:
URL: https://github.com/apache/orc/pull/967#discussion_r760634553



##########
File path: java/tools/src/java/org/apache/orc/tools/convert/CsvReader.java
##########
@@ -372,4 +370,15 @@ Converter buildConverter(IntWritable startOffset, TypeDescription schema) {
         throw new IllegalArgumentException("Unhandled type " + schema);
     }
   }
+
+  public static void main(String[] args) {

Review comment:
       Very sorry for committing some validation code




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #967:
URL: https://github.com/apache/orc/pull/967#discussion_r760887314



##########
File path: java/tools/src/test/org/apache/orc/tools/convert/TestConvert.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestConvert {
+
+  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);
+  }
+
+  @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];
+    rowIterator.nextBatch(batch);
+
+    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]);

Review comment:
       The difference between you and me seems to be the timezones. You are in Asia timezone and I'm in America/Los Angeles timezone.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #967:
URL: https://github.com/apache/orc/pull/967#discussion_r760887860



##########
File path: java/tools/src/test/org/apache/orc/tools/convert/TestConvert.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestConvert {
+
+  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);
+  }
+
+  @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];
+    rowIterator.nextBatch(batch);
+
+    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]);

Review comment:
       Can we make the test case to be independent from the tester's environment?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #967: ORC-1053: Fix time zone offset precision when convert tool converts `LocalDateTime` to `Timestamp` is not consistent with the internal default precision of ORC

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #967:
URL: https://github.com/apache/orc/pull/967#discussion_r760387466



##########
File path: java/tools/src/java/org/apache/orc/tools/convert/CsvReader.java
##########
@@ -372,4 +370,15 @@ Converter buildConverter(IntWritable startOffset, TypeDescription schema) {
         throw new IllegalArgumentException("Unhandled type " + schema);
     }
   }
+
+  public static void main(String[] args) {

Review comment:
       Could you remove this please?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org