You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "voonhous (via GitHub)" <gi...@apache.org> on 2023/04/10 09:18:27 UTC

[GitHub] [hudi] voonhous opened a new pull request, #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

voonhous opened a new pull request, #8418:
URL: https://github.com/apache/hudi/pull/8418

   …iles
   
   ### Change Logs
   
   Fix parquet read issues when inline-clustering is enabled for APPEND-ONLY pipelines on Flink.
   
   This PR includes:
   
   1. Removal of `Timestamp96Writer` as Hudi-on-Flink does not support writing such a type.
   2. Prevent `TIMESTAMP(9)` types from being written to parquet files
   3. `TIMESTAMP(6)` will be written as INT64 to parquet from the StreamWriteFunction and AppendWriteFunction entry points
   
   
   ### Impact
   
   Removal of TIMESTAMP(9) support.
   
   
   ### Risk level (write none, low medium or high below)
   
   LOW
   
   ### Documentation Update
   
   _Describe any necessary documentation update if there is any new feature, config, or user-facing change_
   
   - _The config description must be updated if new configs are added or the default value of the configs are changed_
   - _Any new feature or user-facing change requires updating the Hudi website. Please create a Jira ticket, attach the
     ticket number here and follow the [instruction](https://hudi.apache.org/contribute/developer-setup#website) to make
     changes to the website._
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162321558


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   If we want to support `TIMESTAMP(9)`, opening a separate JIRA issue + PR for it would be better. Let's fix the inconsistency between Stream and Append mode writing different physical types to parquet.
   
   The reason for doing it in another PR is as such:
   
   Stream mode does not allow `TIMESTAMP(9)` types. If Stream mode (upserts) does not allow `TIMESTAMP(9)`, there is no reason to allow `TIMESTAMP(9)` for APPEND mode. 
   
   I'd like to separate feature + bug fixes so that people who want to cherrypick fixes into their internal branch will find it easier to do so. 
   
   IIUC, type support should be consistent for Hudi tables under all write modes.
   
   
   In short, let's fix the bug in this PR first and align the behaviour for both STREAM and APPEND modes. So, I'll undo the removal of `Timestamp96Writer` and tidy up the tests.
   
   Proper `TIMESTAMP(9)` support should be done in another PR. Whether to implement it as a INT96/BINARY type or INT64 type, we can discuss it in detail in a JIRA/PR.  WDYT?



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162484156


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   > Can the append mode write timestamp(9) then?
   
   Nope, APPEND can't write TIMESTAMP(9).
   
   > BTW, Spark use the INT96 as the default output timestamp type in their parquet writer: https://github.com/apache/spark/blob/0a63a496bdced946a5d4825ca66df12de51d3a87/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L970
   
   I don't think we are using this by default, writing TIMESTAMP_NANOS with Hudi-on-Spark will write INT64.
   
   Example:
   
   ```sql
   CREATE TABLE `dev_hudi`.`timestamp_test`
   (
       `id`            INTEGER,
       `bigint_col`    BIGINT,
       `string_col`    STRING,
       `double_col`    DOUBLE,
       `timestamp_col` TIMESTAMP,
       `operation`     STRING
   ) USING hudi
   TBLPROPERTIES (
     'primaryKey' = 'id',
     'type' = 'cow',
     'preCombineField' = 'bigint_col'
   )
   LOCATION 'hdfs://path/to/timestamp_test';
   
   -- use nanos, however, this will fallback to micros
   INSERT INTO `dev_hudi`.`timestamp_test`
   VALUES (1, 1000, "string_col_1", 1.1, TIMESTAMP "1970-01-01 00:00:01.001001001", "init"),
          (2, 2000, "string_col_2", 2.2, TIMESTAMP "1970-01-01 00:00:02.001001001", "init");
   
   
   SELECT * FROM `dev_hudi`.`timestamp_test`;
   
   20230411163354949	20230411163354949_0_0	1		5ea1112a-3f7d-4c6a-8f20-5275055ee330-0_0-17-20_20230411163354949.parquet	1	1000	string_col_1	1.1	1970-01-01 00:00:01.001001	init
   20230411163354949	20230411163354949_0_1	2		5ea1112a-3f7d-4c6a-8f20-5275055ee330-0_0-17-20_20230411163354949.parquet	2	2000	string_col_2	2.2	1970-01-01 00:00:02.001001	init
   ```
   
   parquet-tools snippet:
   
   ```
   ############ Column(timestamp_col)[row group 0] ############
   name: timestamp_col
   path: timestamp_col
   max_definition_level: 1
   max_repetition_level: 0
   physical_type: INT64
   logical_type: Timestamp(isAdjustedToUTC=true, timeUnit=microseconds, is_from_converted_type=true, force_set_converted_type=false)
   converted_type (legacy): TIMESTAMP_MICROS
   compression: GZIP (space_saved: -20%)
   total_compressed_size: 100
   total_uncompressed_size: 83
   ```
   
   
   



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1163897188


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   Hello @danny0405 , any further comments with regards to the current changes for this bug fix?



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1511106294

   Am struggling with the Github CI:
   
   ```
   2023-04-17T09:00:40.6653644Z 918536 [task-result-getter-0] ERROR org.apache.spark.scheduler.TaskSetManager [] - Task 0 in stage 2366.0 failed 1 times; aborting job
   2023-04-17T09:00:40.8838862Z 918673 [Executor task launch worker for task 1.0 in stage 2366.0 (TID 3438)] ERROR org.apache.spark.executor.Executor [] - Exception in task 1.0 in stage 2366.0 (TID 3438)
   2023-04-17T09:00:40.8840609Z java.lang.ClassCastException: org.apache.spark.sql.vectorized.ColumnarBatchRow cannot be cast to org.apache.spark.sql.vectorized.ColumnarBatch
   2023-04-17T09:00:40.8841616Z 	at org.apache.spark.sql.execution.FileSourceScanExec$$anon$1.next(DataSourceScanExec.scala:528) ~[spark-sql_2.12-3.2.3.jar:0.14.0-SNAPSHOT]
   2023-04-17T09:00:40.8842408Z 	at org.apache.spark.sql.execution.FileSourceScanExec$$anon$1.next(DataSourceScanExec.scala:517) ~[spark-sql_2.12-3.2.3.jar:0.14.0-SNAPSHOT]
   2023-04-17T09:00:40.8843112Z 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.columnartorow_nextBatch_0$(Unknown Source) ~[?:?]
   2023-04-17T09:00:40.8843866Z 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source) ~[?:?]
   2023-04-17T09:00:40.8844701Z 	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) ~[spark-sql_2.12-3.2.3.jar:0.14.0-SNAPSHOT]
   2023-04-17T09:00:40.8845512Z 	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759) ~[spark-sql_2.12-3.2.3.jar:0.14.0-SNAPSHOT]
   2023-04-17T09:00:40.8846189Z 	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458) ~[scala-library-2.12.10.jar:?]
   2023-04-17T09:00:40.8846741Z 	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458) ~[scala-library-2.12.10.jar:?]
   2023-04-17T09:00:40.8847270Z 	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458) ~[scala-library-2.12.10.jar:?]
   2023-04-17T09:00:40.8847824Z 	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458) ~[scala-library-2.12.10.jar:?]
   2023-04-17T09:00:40.8848354Z 	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458) ~[scala-library-2.12.10.jar:?]
   2023-04-17T09:00:40.8849015Z 	at org.apache.spark.util.random.SamplingUtils$.reservoirSampleAndCount(SamplingUtils.scala:41) ~[spark-core_2.12-3.2.3.jar:3.2.3]
   2023-04-17T09:00:40.8849674Z 	at org.apache.spark.RangePartitioner$.$anonfun$sketch$1(Partitioner.scala:306) ~[spark-core_2.12-3.2.3.jar:3.2.3]
   2023-04-17T09:00:40.8850316Z 	at org.apache.spark.RangePartitioner$.$anonfun$sketch$1$adapted(Partitioner.scala:304) ~[spark-core_2.12-3.2.3.jar:3.2.3]
   2023-04-17T09:00:40.8850941Z 	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndex$2(RDD.scala:915) ~[spark-core_2.12-3.2.3.jar:3.2.3]
   2023-04-17T09:00:40.8851886Z 	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndex$2$adapted(RDD.scala:915) ~[spark-core_2.12-3.2.3.jar:3.2.3]
   2023-04-17T09:00:40.8852535Z 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) ~[spark-core_2.12-3.2.3.jar:3.2.3]
   2023-04-17T09:00:40.8853174Z 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373) ~[spark-core_2.12-3.2.3.jar:3.2.3]
   2023-04-17T09:00:40.8853737Z 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337) ~[spark-core_2.12-3.2.3.jar:3.2.3]
   2023-04-17T09:00:40.8854385Z 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) ~[spark-core_2.12-3.2.3.jar:3.2.3]
   2023-04-17T09:00:40.8854959Z 	at org.apache.spark.scheduler.Task.run(Task.scala:131) ~[spark-core_2.12-3.2.3.jar:3.2.3]
   2023-04-17T09:00:40.8855554Z 	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506) ~[spark-core_2.12-3.2.3.jar:3.2.3]
   2023-04-17T09:00:40.8856150Z 	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1491) ~[spark-core_2.12-3.2.3.jar:3.2.3]
   2023-04-17T09:00:40.8856720Z 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509) ~[spark-core_2.12-3.2.3.jar:3.2.3]
   2023-04-17T09:00:40.8857327Z 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_362]
   2023-04-17T09:00:40.8857840Z 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_362]
   2023-04-17T09:00:40.8858350Z 	at java.lang.Thread.run(Thread.java:750) ~[?:1.8.0_362]
   2023-04-17T09:01:07.0036050Z - Test Call run_clustering Procedure Order Strategy *** FAILED ***
   2023-04-17T09:01:07.0038126Z   java.util.concurrent.CompletionException: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2366.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2366.0 (TID 3437) (fv-az400-928.4sktce1ui1juvaou3g2cooit5e.bx.internal.cloudapp.net executor driver): java.lang.ClassCastException: org.apache.spark.sql.vectorized.ColumnarBatchRow cannot be cast to org.apache.spark.sql.vectorized.ColumnarBatch
   2023-04-17T09:01:07.0039313Z 	at org.apache.spark.sql.execution.FileSourceScanExec$$anon$1.next(DataSourceScanExec.scala:528)
   2023-04-17T09:01:07.0039947Z 	at org.apache.spark.sql.execution.FileSourceScanExec$$anon$1.next(DataSourceScanExec.scala:517)
   2023-04-17T09:01:07.0040676Z 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.columnartorow_nextBatch_0$(Unknown Source)
   2023-04-17T09:01:07.0041487Z 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
   2023-04-17T09:01:07.0042213Z 	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
   2023-04-17T09:01:07.0042873Z 	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
   2023-04-17T09:01:07.0043424Z 	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
   2023-04-17T09:01:07.0043856Z 	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
   2023-04-17T09:01:07.0044301Z 	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
   2023-04-17T09:01:07.0044743Z 	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
   2023-04-17T09:01:07.0045169Z 	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
   2023-04-17T09:01:07.0045675Z 	at org.apache.spark.util.random.SamplingUtils$.reservoirSampleAndCount(SamplingUtils.scala:41)
   2023-04-17T09:01:07.0046244Z 	at org.apache.spark.RangePartitioner$.$anonfun$sketch$1(Partitioner.scala:306)
   2023-04-17T09:01:07.0046765Z 	at org.apache.spark.RangePartitioner$.$anonfun$sketch$1$adapted(Partitioner.scala:304)
   2023-04-17T09:01:07.0075476Z 	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndex$2(RDD.scala:915)
   2023-04-17T09:01:07.0076081Z 	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndex$2$adapted(RDD.scala:915)
   2023-04-17T09:01:07.0076644Z 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   2023-04-17T09:01:07.0077198Z 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
   2023-04-17T09:01:07.0100079Z 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
   2023-04-17T09:01:07.0100905Z 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
   2023-04-17T09:01:07.0101488Z 	at org.apache.spark.scheduler.Task.run(Task.scala:131)
   2023-04-17T09:01:07.0101994Z 	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
   2023-04-17T09:01:07.0102506Z 	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1491)
   2023-04-17T09:01:07.0102994Z 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
   2023-04-17T09:01:07.0108497Z 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   2023-04-17T09:01:07.0109167Z 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   2023-04-17T09:01:07.0109625Z 	at java.lang.Thread.run(Thread.java:750)
   ```
   
   Ran the test locally and it's passing. The stack trace doesn't look like it's caused by flaky tests though. T.T


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162394534


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   I undid the removal of `Timestamp96Writer`.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1510657026

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340",
       "triggerID" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16380",
       "triggerID" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "triggerType" : "PUSH"
     }, {
       "hash" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340) 
   * be4235b4f42ffe845ec17dad7113af2de7a94332 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16380) 
   * 79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162484156


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   > Can the append mode write timestamp(9) then?
   
   Nope, APPEND can't write TIMESTAMP(9).
   
   > BTW, Spark use the INT96 as the default output timestamp type in their parquet writer: https://github.com/apache/spark/blob/0a63a496bdced946a5d4825ca66df12de51d3a87/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L970
   
   I don't think we are using this by default, writing TIMESTAMP_MICROS with Hudi-on-Spark will write INT64.
   
   We actually set this config as TIMESTAMP_MICROS in Hudi.
   
   https://github.com/apache/hudi/blob/d6ff3d6ba46b51f58ebb1db58f26423a211741f5/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java
   
   Example:
   
   ```sql
   CREATE TABLE `dev_hudi`.`timestamp_test`
   (
       `id`            INTEGER,
       `bigint_col`    BIGINT,
       `string_col`    STRING,
       `double_col`    DOUBLE,
       `timestamp_col` TIMESTAMP,
       `operation`     STRING
   ) USING hudi
   TBLPROPERTIES (
     'primaryKey' = 'id',
     'type' = 'cow',
     'preCombineField' = 'bigint_col'
   )
   LOCATION 'hdfs://path/to/timestamp_test';
   
   -- use nanos, however, this will fallback to micros
   INSERT INTO `dev_hudi`.`timestamp_test`
   VALUES (1, 1000, "string_col_1", 1.1, TIMESTAMP "1970-01-01 00:00:01.001001001", "init"),
          (2, 2000, "string_col_2", 2.2, TIMESTAMP "1970-01-01 00:00:02.001001001", "init");
   
   
   SELECT * FROM `dev_hudi`.`timestamp_test`;
   
   20230411163354949	20230411163354949_0_0	1		5ea1112a-3f7d-4c6a-8f20-5275055ee330-0_0-17-20_20230411163354949.parquet	1	1000	string_col_1	1.1	1970-01-01 00:00:01.001001	init
   20230411163354949	20230411163354949_0_1	2		5ea1112a-3f7d-4c6a-8f20-5275055ee330-0_0-17-20_20230411163354949.parquet	2	2000	string_col_2	2.2	1970-01-01 00:00:02.001001	init
   ```
   
   parquet-tools snippet:
   
   ```
   ############ Column(timestamp_col)[row group 0] ############
   name: timestamp_col
   path: timestamp_col
   max_definition_level: 1
   max_repetition_level: 0
   physical_type: INT64
   logical_type: Timestamp(isAdjustedToUTC=true, timeUnit=microseconds, is_from_converted_type=true, force_set_converted_type=false)
   converted_type (legacy): TIMESTAMP_MICROS
   compression: GZIP (space_saved: -20%)
   total_compressed_size: 100
   total_uncompressed_size: 83
   ```
   
   
   



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1502634702

   Well, the INT96 is already deprecated and kept for some historical reasons: https://issues.apache.org/jira/browse/PARQUET-323, +1 to remove the conversion for it.


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1501596480

   @danny0405 Can you please help to review this PR? 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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162321558


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   If we want to support `TIMESTAMP(9)`, I opening a new PR ticket + PR for it. Let's fix the inconsistency between Stream and Append mode writing different physical types to parquet.
   
   The reason for doing it in another PR is as such:
   
   Stream mode does not allow `TIMESTAMP(9)` types. If Stream mode (upserts) does not allow `TIMESTAMP(9)`, there is no reason to allow `TIMESTAMP(9)` for APPEND mode. 
   
   IIUC, type support should be consistent for Hudi tables under all write modes.
   
   
   In short, let's fix the bug in this PR first and align the behaviour for both STREAM and APPEND modes. So, I'll undo the removal of `Timestamp96Writer` and tidy up the tests.
   
   Proper `TIMESTAMP(9)` support should be done in another PR. Whether to implement it as a INT96/BINARY type or INT64 type, we can discuss it in detail in a JIRA/PR.  WDYT?



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1168171291


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -18,6 +18,11 @@
 
 package org.apache.hudi.sink.cluster;
 
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.api.ValidationException;

Review Comment:
   Fix the import sequence.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1511315828

   The failed test cases in module `hudi-utilities` and `hudi-spark-client` have no relationship with this change, would merge it soon ~


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1168171167


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -31,6 +31,7 @@
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.TimestampType;
 import org.apache.flink.util.Preconditions;
+import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.parquet.io.api.Binary;

Review Comment:
   You might need to fix your import sequence of the IDEA.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1168238938


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -31,6 +31,7 @@
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.TimestampType;
 import org.apache.flink.util.Preconditions;
+import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.parquet.io.api.Binary;

Review Comment:
   Man, after adding the hudi checkstyles, it's adding whitelines instead of ordering the imports... 
   
   ETA:
   Nvm, fixed it, i was reformating code instead of optimizing imports... My bad



##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -18,6 +18,11 @@
 
 package org.apache.hudi.sink.cluster;
 
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.api.ValidationException;

Review Comment:
   Done



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1168232222


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -18,6 +18,11 @@
 
 package org.apache.hudi.sink.cluster;
 
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.api.ValidationException;

Review Comment:
   hudi package should be in the first block and isolated with other blocks.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1510652533

   Added more checkstyle fixes.


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1168124406


##########
hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/io/storage/row/parquet/TestParquetSchemaConverter.java:
##########
@@ -25,6 +25,7 @@
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 

Review Comment:
   Done.



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java:
##########
@@ -18,12 +18,11 @@
 
 package org.apache.hudi.io.storage.row;
 
-import org.apache.hudi.avro.HoodieBloomFilterWriteSupport;
-import org.apache.hudi.common.bloom.BloomFilter;
-
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.avro.HoodieBloomFilterWriteSupport;
+import org.apache.hudi.common.bloom.BloomFilter;
 import org.apache.hudi.common.util.Option;

Review Comment:
   Done



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1168231169


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -31,6 +31,7 @@
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.TimestampType;
 import org.apache.flink.util.Preconditions;
+import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.parquet.io.api.Binary;

Review Comment:
   Got it, will fix that.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1507993349

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340",
       "triggerID" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162333501


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   Alright then, if this is your worry, then maybe the correct fix is to allow clustering to read TIMESTAMP(9) types? 
   
   Spark is able to read out TIMESTAMP(9) types, but it'll only display it in the millis granularity from my tests.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162264009


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   We should still support the `NANOS` precision using INT64 according to the parquet reference: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1165174172


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -283,34 +291,66 @@ public void write(ArrayData array, int ordinal) {
     }
   }
 
-  /**
-   * Timestamp of INT96 bytes, julianDay(4) + nanosOfDay(8). See
-   * https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp
-   * TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType.
-   */
   private class Timestamp64Writer implements FieldWriter {
-    private Timestamp64Writer() {
+
+    private final int precision;
+    private Timestamp64Writer(int precision) {
+      this.precision = precision;
     }
 
     @Override
     public void write(RowData row, int ordinal) {
-      recordConsumer.addLong(timestampToInt64(row.getTimestamp(ordinal, 3)));
+      TimestampData timestampData = row.getTimestamp(ordinal, precision);
+      recordConsumer.addLong(timestampToInt64(timestampData, precision));
     }
 
     @Override
     public void write(ArrayData array, int ordinal) {
-      recordConsumer.addLong(timestampToInt64(array.getTimestamp(ordinal, 3)));
+      TimestampData timestampData = array.getTimestamp(ordinal, precision);
+      recordConsumer.addLong(timestampToInt64(timestampData, precision));
     }
   }
 
-  private long timestampToInt64(TimestampData timestampData) {
-    return utcTimestamp ? timestampData.getMillisecond() : timestampData.toTimestamp().getTime();
+  /**
+   * Converts a {@code TimestampData} to its corresponding int64 value. This function only accepts TimestampData of
+   * precision 3 or 6. Special attention will need to be given to a TimestampData of precision = 6.
+   * <p>
+   * For example representing `1970-01-01T00:00:03.100001` of precision 6 will have:
+   * <ul>
+   *   <li>millisecond = 3100</li>
+   *   <li>nanoOfMillisecond = 1000</li>
+   * </ul>
+   * As such, the int64 value will be:
+   * <p>
+   * millisecond * 1000 + nanoOfMillisecond / 1000
+   *
+   * @param timestampData TimestampData to be converted to int64 format
+   * @param precision the precision of the TimestampData
+   * @return int64 value of the TimestampData
+   */
+  private long timestampToInt64(TimestampData timestampData, int precision) {
+    if (!utcTimestamp) {
+      // toTimestamp is agnostic of precision
+      return timestampData.toTimestamp().getTime();
+    } else if (precision == 3) {
+      return timestampData.getMillisecond();
+    } else if (precision == 6) {
+      // convert timestampDat ato microseconds format
+      return timestampData.getMillisecond() * 1000 + timestampData.getNanoOfMillisecond() / 1000;
+    }
+
+    throw new IllegalArgumentException(
+        "Unable to convert TimestampData with precision: "

Review Comment:
   This is a private method anyways, this `throw` is generally unreachable, will invoke `ValidationUtils.checkArgument` in the `Timestamp64Writer`.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1507853543

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340",
       "triggerID" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319) 
   * cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1165173485


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -283,34 +291,66 @@ public void write(ArrayData array, int ordinal) {
     }
   }
 
-  /**
-   * Timestamp of INT96 bytes, julianDay(4) + nanosOfDay(8). See
-   * https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp
-   * TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType.
-   */
   private class Timestamp64Writer implements FieldWriter {
-    private Timestamp64Writer() {
+
+    private final int precision;
+    private Timestamp64Writer(int precision) {
+      this.precision = precision;
     }
 
     @Override
     public void write(RowData row, int ordinal) {
-      recordConsumer.addLong(timestampToInt64(row.getTimestamp(ordinal, 3)));
+      TimestampData timestampData = row.getTimestamp(ordinal, precision);
+      recordConsumer.addLong(timestampToInt64(timestampData, precision));
     }
 
     @Override
     public void write(ArrayData array, int ordinal) {
-      recordConsumer.addLong(timestampToInt64(array.getTimestamp(ordinal, 3)));
+      TimestampData timestampData = array.getTimestamp(ordinal, precision);
+      recordConsumer.addLong(timestampToInt64(timestampData, precision));
     }
   }
 
-  private long timestampToInt64(TimestampData timestampData) {
-    return utcTimestamp ? timestampData.getMillisecond() : timestampData.toTimestamp().getTime();
+  /**
+   * Converts a {@code TimestampData} to its corresponding int64 value. This function only accepts TimestampData of
+   * precision 3 or 6. Special attention will need to be given to a TimestampData of precision = 6.
+   * <p>
+   * For example representing `1970-01-01T00:00:03.100001` of precision 6 will have:
+   * <ul>
+   *   <li>millisecond = 3100</li>
+   *   <li>nanoOfMillisecond = 1000</li>
+   * </ul>
+   * As such, the int64 value will be:
+   * <p>
+   * millisecond * 1000 + nanoOfMillisecond / 1000
+   *
+   * @param timestampData TimestampData to be converted to int64 format
+   * @param precision the precision of the TimestampData
+   * @return int64 value of the TimestampData
+   */
+  private long timestampToInt64(TimestampData timestampData, int precision) {
+    if (!utcTimestamp) {
+      // toTimestamp is agnostic of precision
+      return timestampData.toTimestamp().getTime();

Review Comment:
   Fixed



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1165109067


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -124,17 +124,25 @@ private FieldWriter createWriter(LogicalType t) {
         return new DoubleWriter();
       case TIMESTAMP_WITHOUT_TIME_ZONE:
         TimestampType timestampType = (TimestampType) t;
-        if (timestampType.getPrecision() == 3) {
-          return new Timestamp64Writer();
+        final int tsPrecision = timestampType.getPrecision();
+        if (tsPrecision == 3 || tsPrecision == 6) {
+          return new Timestamp64Writer(tsPrecision);
         } else {
-          return new Timestamp96Writer(timestampType.getPrecision());
+          throw new IllegalArgumentException(
+              "Parquet does not support TIMESTAMP type with precision: "

Review Comment:
   Can we still return Timestamp96Writer here?



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1506568932

   @danny0405 addressed the comments, PTAL


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1165109685


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -124,17 +124,25 @@ private FieldWriter createWriter(LogicalType t) {
         return new DoubleWriter();
       case TIMESTAMP_WITHOUT_TIME_ZONE:
         TimestampType timestampType = (TimestampType) t;
-        if (timestampType.getPrecision() == 3) {
-          return new Timestamp64Writer();
+        final int tsPrecision = timestampType.getPrecision();
+        if (tsPrecision == 3 || tsPrecision == 6) {
+          return new Timestamp64Writer(tsPrecision);
         } else {
-          return new Timestamp96Writer(timestampType.getPrecision());
+          throw new IllegalArgumentException(
+              "Parquet does not support TIMESTAMP type with precision: "
+                  + tsPrecision
+                  + ", it only support precisions <= 6.");
         }
       case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
         LocalZonedTimestampType localZonedTimestampType = (LocalZonedTimestampType) t;
-        if (localZonedTimestampType.getPrecision() == 3) {
-          return new Timestamp64Writer();
+        final int tsLtzPrecision = localZonedTimestampType.getPrecision();
+        if (tsLtzPrecision == 3 || tsLtzPrecision == 6) {
+          return new Timestamp64Writer(tsLtzPrecision);
         } else {
-          return new Timestamp96Writer(localZonedTimestampType.getPrecision());
+          throw new IllegalArgumentException(

Review Comment:
   Ditto, can we rollback the change.



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -283,34 +291,66 @@ public void write(ArrayData array, int ordinal) {
     }
   }
 
-  /**
-   * Timestamp of INT96 bytes, julianDay(4) + nanosOfDay(8). See
-   * https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp

Review Comment:
   Can we fix the doc instead of removing it.



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -283,34 +291,66 @@ public void write(ArrayData array, int ordinal) {
     }
   }
 
-  /**
-   * Timestamp of INT96 bytes, julianDay(4) + nanosOfDay(8). See
-   * https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp
-   * TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType.
-   */
   private class Timestamp64Writer implements FieldWriter {
-    private Timestamp64Writer() {
+
+    private final int precision;
+    private Timestamp64Writer(int precision) {
+      this.precision = precision;
     }
 
     @Override
     public void write(RowData row, int ordinal) {
-      recordConsumer.addLong(timestampToInt64(row.getTimestamp(ordinal, 3)));
+      TimestampData timestampData = row.getTimestamp(ordinal, precision);
+      recordConsumer.addLong(timestampToInt64(timestampData, precision));
     }
 
     @Override
     public void write(ArrayData array, int ordinal) {
-      recordConsumer.addLong(timestampToInt64(array.getTimestamp(ordinal, 3)));
+      TimestampData timestampData = array.getTimestamp(ordinal, precision);
+      recordConsumer.addLong(timestampToInt64(timestampData, precision));
     }
   }
 
-  private long timestampToInt64(TimestampData timestampData) {
-    return utcTimestamp ? timestampData.getMillisecond() : timestampData.toTimestamp().getTime();
+  /**
+   * Converts a {@code TimestampData} to its corresponding int64 value. This function only accepts TimestampData of
+   * precision 3 or 6. Special attention will need to be given to a TimestampData of precision = 6.
+   * <p>
+   * For example representing `1970-01-01T00:00:03.100001` of precision 6 will have:
+   * <ul>
+   *   <li>millisecond = 3100</li>
+   *   <li>nanoOfMillisecond = 1000</li>
+   * </ul>
+   * As such, the int64 value will be:
+   * <p>
+   * millisecond * 1000 + nanoOfMillisecond / 1000
+   *
+   * @param timestampData TimestampData to be converted to int64 format
+   * @param precision the precision of the TimestampData
+   * @return int64 value of the TimestampData
+   */
+  private long timestampToInt64(TimestampData timestampData, int precision) {
+    if (!utcTimestamp) {
+      // toTimestamp is agnostic of precision
+      return timestampData.toTimestamp().getTime();
+    } else if (precision == 3) {
+      return timestampData.getMillisecond();
+    } else if (precision == 6) {
+      // convert timestampDat ato microseconds format
+      return timestampData.getMillisecond() * 1000 + timestampData.getNanoOfMillisecond() / 1000;
+    }
+
+    throw new IllegalArgumentException(
+        "Unable to convert TimestampData with precision: "

Review Comment:
   Usually we check the arguments at the beginning of the method, and can we use the utilites class `ValidationUtils`



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java:
##########
@@ -600,23 +600,29 @@ private static Type convertToParquetType(
             .named(name);
       case TIMESTAMP_WITHOUT_TIME_ZONE:
         TimestampType timestampType = (TimestampType) type;
-        if (timestampType.getPrecision() == 3) {
+        if (timestampType.getPrecision() == 3 || timestampType.getPrecision() == 6) {
+          TimeUnit timeunit = timestampType.getPrecision() == 3 ? TimeUnit.MILLIS : TimeUnit.MICROS;
           return Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition)
-              .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS))
+              .as(LogicalTypeAnnotation.timestampType(true, timeunit))
               .named(name);
         } else {
-          return Types.primitive(PrimitiveType.PrimitiveTypeName.INT96, repetition)
-              .named(name);
+          throw new IllegalArgumentException(
+              "Parquet does not support TIMESTAMP type with precision: "
+                  + timestampType.getPrecision()
+                  + ", it only support precisions <= 6.");
         }
       case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
         LocalZonedTimestampType localZonedTimestampType = (LocalZonedTimestampType) type;
-        if (localZonedTimestampType.getPrecision() == 3) {
+        if (localZonedTimestampType.getPrecision() == 3 || localZonedTimestampType.getPrecision() == 6) {
+          TimeUnit timeunit = localZonedTimestampType.getPrecision() == 3 ? TimeUnit.MILLIS : TimeUnit.MICROS;
           return Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition)
-              .as(LogicalTypeAnnotation.timestampType(false, TimeUnit.MILLIS))
+              .as(LogicalTypeAnnotation.timestampType(false, timeunit))
               .named(name);
         } else {
-          return Types.primitive(PrimitiveType.PrimitiveTypeName.INT96, repetition)
-              .named(name);
+          throw new IllegalArgumentException(
+              "Parquet does not support TIMESTAMP_LTZ type with precision: "
+                  + localZonedTimestampType.getPrecision()

Review Comment:
   Revert the change to not throw exception.



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -283,34 +291,66 @@ public void write(ArrayData array, int ordinal) {
     }
   }
 
-  /**
-   * Timestamp of INT96 bytes, julianDay(4) + nanosOfDay(8). See
-   * https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp
-   * TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType.
-   */
   private class Timestamp64Writer implements FieldWriter {
-    private Timestamp64Writer() {
+
+    private final int precision;
+    private Timestamp64Writer(int precision) {
+      this.precision = precision;
     }
 
     @Override
     public void write(RowData row, int ordinal) {
-      recordConsumer.addLong(timestampToInt64(row.getTimestamp(ordinal, 3)));
+      TimestampData timestampData = row.getTimestamp(ordinal, precision);
+      recordConsumer.addLong(timestampToInt64(timestampData, precision));
     }
 
     @Override
     public void write(ArrayData array, int ordinal) {
-      recordConsumer.addLong(timestampToInt64(array.getTimestamp(ordinal, 3)));
+      TimestampData timestampData = array.getTimestamp(ordinal, precision);
+      recordConsumer.addLong(timestampToInt64(timestampData, precision));
     }
   }
 
-  private long timestampToInt64(TimestampData timestampData) {
-    return utcTimestamp ? timestampData.getMillisecond() : timestampData.toTimestamp().getTime();
+  /**
+   * Converts a {@code TimestampData} to its corresponding int64 value. This function only accepts TimestampData of
+   * precision 3 or 6. Special attention will need to be given to a TimestampData of precision = 6.
+   * <p>
+   * For example representing `1970-01-01T00:00:03.100001` of precision 6 will have:
+   * <ul>
+   *   <li>millisecond = 3100</li>
+   *   <li>nanoOfMillisecond = 1000</li>
+   * </ul>
+   * As such, the int64 value will be:
+   * <p>
+   * millisecond * 1000 + nanoOfMillisecond / 1000
+   *
+   * @param timestampData TimestampData to be converted to int64 format
+   * @param precision the precision of the TimestampData
+   * @return int64 value of the TimestampData
+   */
+  private long timestampToInt64(TimestampData timestampData, int precision) {
+    if (!utcTimestamp) {
+      // toTimestamp is agnostic of precision
+      return timestampData.toTimestamp().getTime();

Review Comment:
   it is not  agnostic, we should also fix the longs for different time unit.



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java:
##########
@@ -600,23 +600,29 @@ private static Type convertToParquetType(
             .named(name);
       case TIMESTAMP_WITHOUT_TIME_ZONE:
         TimestampType timestampType = (TimestampType) type;
-        if (timestampType.getPrecision() == 3) {
+        if (timestampType.getPrecision() == 3 || timestampType.getPrecision() == 6) {
+          TimeUnit timeunit = timestampType.getPrecision() == 3 ? TimeUnit.MILLIS : TimeUnit.MICROS;
           return Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition)
-              .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS))
+              .as(LogicalTypeAnnotation.timestampType(true, timeunit))
               .named(name);
         } else {
-          return Types.primitive(PrimitiveType.PrimitiveTypeName.INT96, repetition)
-              .named(name);
+          throw new IllegalArgumentException(
+              "Parquet does not support TIMESTAMP type with precision: "
+                  + timestampType.getPrecision()

Review Comment:
   Revert the change to not throw exception.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1506576581

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3f508c7d8e9590ea9f19a106d2ea50a0999b353f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250) 
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1168176154


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -31,6 +31,7 @@
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.TimestampType;
 import org.apache.flink.util.Preconditions;
+import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.parquet.io.api.Binary;

Review Comment:
   Hmmm, I don't think there's anything wrong with this, `h` is after `f`, so, `org.apache.hudi` should be behind `org.apache.flink` right?



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1510738769

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340",
       "triggerID" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16380",
       "triggerID" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "triggerType" : "PUSH"
     }, {
       "hash" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16381",
       "triggerID" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f063bb1db0459bb6a62dd55a57279cf45f4e9fa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5f063bb1db0459bb6a62dd55a57279cf45f4e9fa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * be4235b4f42ffe845ec17dad7113af2de7a94332 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16380) 
   * 79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16381) 
   * 5f063bb1db0459bb6a62dd55a57279cf45f4e9fa UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1502840088

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7466734a78d74734b423aa68a86f76d1bc2b9a4e Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225) 
   * 3f508c7d8e9590ea9f19a106d2ea50a0999b353f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1503071771

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3f508c7d8e9590ea9f19a106d2ea50a0999b353f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162754131


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   Thanks for the clarification, let's fix the `MICROS` time unit to use the INT64 as the primitive type and only uses INT96 for `NANOS`.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1507356645

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1165192731


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -283,34 +291,66 @@ public void write(ArrayData array, int ordinal) {
     }
   }
 
-  /**
-   * Timestamp of INT96 bytes, julianDay(4) + nanosOfDay(8). See
-   * https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp

Review Comment:
   Done



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1510653019

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340",
       "triggerID" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340) 
   * be4235b4f42ffe845ec17dad7113af2de7a94332 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1168123430


##########
hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/io/storage/row/parquet/TestParquetSchemaConverter.java:
##########
@@ -25,6 +25,7 @@
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 

Review Comment:
   Useless import



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162333501


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   Let's not digress too much from the crux of the issue... 
   
   The issue here is: TIMESTAMP(6) in STREAM mode is writing INT64, while TIMESTAMP(6) in APPEND mode is writing INT64. 
   
   Let's aim to fix that in this PR first.
   
   ## TIMESTAMP(9)
   
   As for TIMESTAMP(9) types, there is currently no support for it AT ALL. If you try to create a table under any mode with TIMESTAMP(9) type, an error will be thrown when u try writing to the table regardless of APPEND / STREAM mode. 
   
   
   As such, regression is not a concern here given that we do not support TIMESTAMP(9) to begin with. The premise here is that TIMESTAMP(6) is writing the wrong physical type to parquet. That' about it... 
   
   P.S. Apologies for the ninja edits.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1502828008

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7466734a78d74734b423aa68a86f76d1bc2b9a4e Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225) 
   * 3f508c7d8e9590ea9f19a106d2ea50a0999b353f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162331382


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   > type support should be consistent for Hudi tables under all write modes
   
   I have reservations, if we already support TIMESTAMP(9) for append mode then revert it, it is a regression.
   Let's also check how Spark handle the timestamp(9) data type for parquet writer, and if the removal of TIMESTAMP(9) has some impacts on Hive reader.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1501605225

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7466734a78d74734b423aa68a86f76d1bc2b9a4e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1501600378

   @hbgstc123 for visibility


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1165173485


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -283,34 +291,66 @@ public void write(ArrayData array, int ordinal) {
     }
   }
 
-  /**
-   * Timestamp of INT96 bytes, julianDay(4) + nanosOfDay(8). See
-   * https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp
-   * TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType.
-   */
   private class Timestamp64Writer implements FieldWriter {
-    private Timestamp64Writer() {
+
+    private final int precision;
+    private Timestamp64Writer(int precision) {
+      this.precision = precision;
     }
 
     @Override
     public void write(RowData row, int ordinal) {
-      recordConsumer.addLong(timestampToInt64(row.getTimestamp(ordinal, 3)));
+      TimestampData timestampData = row.getTimestamp(ordinal, precision);
+      recordConsumer.addLong(timestampToInt64(timestampData, precision));
     }
 
     @Override
     public void write(ArrayData array, int ordinal) {
-      recordConsumer.addLong(timestampToInt64(array.getTimestamp(ordinal, 3)));
+      TimestampData timestampData = array.getTimestamp(ordinal, precision);
+      recordConsumer.addLong(timestampToInt64(timestampData, precision));
     }
   }
 
-  private long timestampToInt64(TimestampData timestampData) {
-    return utcTimestamp ? timestampData.getMillisecond() : timestampData.toTimestamp().getTime();
+  /**
+   * Converts a {@code TimestampData} to its corresponding int64 value. This function only accepts TimestampData of
+   * precision 3 or 6. Special attention will need to be given to a TimestampData of precision = 6.
+   * <p>
+   * For example representing `1970-01-01T00:00:03.100001` of precision 6 will have:
+   * <ul>
+   *   <li>millisecond = 3100</li>
+   *   <li>nanoOfMillisecond = 1000</li>
+   * </ul>
+   * As such, the int64 value will be:
+   * <p>
+   * millisecond * 1000 + nanoOfMillisecond / 1000
+   *
+   * @param timestampData TimestampData to be converted to int64 format
+   * @param precision the precision of the TimestampData
+   * @return int64 value of the TimestampData
+   */
+  private long timestampToInt64(TimestampData timestampData, int precision) {
+    if (!utcTimestamp) {
+      // toTimestamp is agnostic of precision
+      return timestampData.toTimestamp().getTime();

Review Comment:
   Hmmm, IIUC, `timestampData.toTimestamp().getTime()` is able to properly handle millis, micros, nanos. Which is why i wrote `toTimestamp()` is agnostic of precision.
   
   
   ```java
   /** Converts this {@link TimestampData} object to a {@link Timestamp}. */
   public Timestamp toTimestamp() {
       return Timestamp.valueOf(toLocalDateTime());
   }
   
   /** Converts this {@link TimestampData} object to a {@link LocalDateTime}. */
   public LocalDateTime toLocalDateTime() {
       int date = (int) (millisecond / MILLIS_PER_DAY);
       int time = (int) (millisecond % MILLIS_PER_DAY);
       if (time < 0) {
           --date;
           time += MILLIS_PER_DAY;
       }
       long nanoOfDay = time * 1_000_000L + nanoOfMillisecond;
       LocalDate localDate = LocalDate.ofEpochDay(date);
       LocalTime localTime = LocalTime.ofNanoOfDay(nanoOfDay);
       return LocalDateTime.of(localDate, localTime);
   }
   ```



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -283,34 +291,66 @@ public void write(ArrayData array, int ordinal) {
     }
   }
 
-  /**
-   * Timestamp of INT96 bytes, julianDay(4) + nanosOfDay(8). See
-   * https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp
-   * TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType.
-   */
   private class Timestamp64Writer implements FieldWriter {
-    private Timestamp64Writer() {
+
+    private final int precision;
+    private Timestamp64Writer(int precision) {
+      this.precision = precision;
     }
 
     @Override
     public void write(RowData row, int ordinal) {
-      recordConsumer.addLong(timestampToInt64(row.getTimestamp(ordinal, 3)));
+      TimestampData timestampData = row.getTimestamp(ordinal, precision);
+      recordConsumer.addLong(timestampToInt64(timestampData, precision));
     }
 
     @Override
     public void write(ArrayData array, int ordinal) {
-      recordConsumer.addLong(timestampToInt64(array.getTimestamp(ordinal, 3)));
+      TimestampData timestampData = array.getTimestamp(ordinal, precision);
+      recordConsumer.addLong(timestampToInt64(timestampData, precision));
     }
   }
 
-  private long timestampToInt64(TimestampData timestampData) {
-    return utcTimestamp ? timestampData.getMillisecond() : timestampData.toTimestamp().getTime();
+  /**
+   * Converts a {@code TimestampData} to its corresponding int64 value. This function only accepts TimestampData of
+   * precision 3 or 6. Special attention will need to be given to a TimestampData of precision = 6.
+   * <p>
+   * For example representing `1970-01-01T00:00:03.100001` of precision 6 will have:
+   * <ul>
+   *   <li>millisecond = 3100</li>
+   *   <li>nanoOfMillisecond = 1000</li>
+   * </ul>
+   * As such, the int64 value will be:
+   * <p>
+   * millisecond * 1000 + nanoOfMillisecond / 1000
+   *
+   * @param timestampData TimestampData to be converted to int64 format
+   * @param precision the precision of the TimestampData
+   * @return int64 value of the TimestampData
+   */
+  private long timestampToInt64(TimestampData timestampData, int precision) {
+    if (!utcTimestamp) {
+      // toTimestamp is agnostic of precision
+      return timestampData.toTimestamp().getTime();
+    } else if (precision == 3) {
+      return timestampData.getMillisecond();
+    } else if (precision == 6) {
+      // convert timestampDat ato microseconds format
+      return timestampData.getMillisecond() * 1000 + timestampData.getNanoOfMillisecond() / 1000;
+    }
+
+    throw new IllegalArgumentException(
+        "Unable to convert TimestampData with precision: "

Review Comment:
   This is a private method anyways, this `throw` is generally unreachable.



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java:
##########
@@ -600,23 +600,29 @@ private static Type convertToParquetType(
             .named(name);
       case TIMESTAMP_WITHOUT_TIME_ZONE:
         TimestampType timestampType = (TimestampType) type;
-        if (timestampType.getPrecision() == 3) {
+        if (timestampType.getPrecision() == 3 || timestampType.getPrecision() == 6) {
+          TimeUnit timeunit = timestampType.getPrecision() == 3 ? TimeUnit.MILLIS : TimeUnit.MICROS;
           return Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition)
-              .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS))
+              .as(LogicalTypeAnnotation.timestampType(true, timeunit))
               .named(name);
         } else {
-          return Types.primitive(PrimitiveType.PrimitiveTypeName.INT96, repetition)
-              .named(name);
+          throw new IllegalArgumentException(
+              "Parquet does not support TIMESTAMP type with precision: "
+                  + timestampType.getPrecision()
+                  + ", it only support precisions <= 6.");
         }
       case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
         LocalZonedTimestampType localZonedTimestampType = (LocalZonedTimestampType) type;
-        if (localZonedTimestampType.getPrecision() == 3) {
+        if (localZonedTimestampType.getPrecision() == 3 || localZonedTimestampType.getPrecision() == 6) {
+          TimeUnit timeunit = localZonedTimestampType.getPrecision() == 3 ? TimeUnit.MILLIS : TimeUnit.MICROS;
           return Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition)
-              .as(LogicalTypeAnnotation.timestampType(false, TimeUnit.MILLIS))
+              .as(LogicalTypeAnnotation.timestampType(false, timeunit))
               .named(name);
         } else {
-          return Types.primitive(PrimitiveType.PrimitiveTypeName.INT96, repetition)
-              .named(name);
+          throw new IllegalArgumentException(
+              "Parquet does not support TIMESTAMP_LTZ type with precision: "
+                  + localZonedTimestampType.getPrecision()

Review Comment:
   Done



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1168238938


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -31,6 +31,7 @@
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.TimestampType;
 import org.apache.flink.util.Preconditions;
+import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.parquet.io.api.Binary;

Review Comment:
   Man, after adding the hudi checkstyles, it's adding whitelines instead of ordering the imports... 



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1510798747

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340",
       "triggerID" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16380",
       "triggerID" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "triggerType" : "PUSH"
     }, {
       "hash" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16381",
       "triggerID" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f063bb1db0459bb6a62dd55a57279cf45f4e9fa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16385",
       "triggerID" : "5f063bb1db0459bb6a62dd55a57279cf45f4e9fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "596c673ec020e47bff03f76e121fad2c2ae2daee",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "596c673ec020e47bff03f76e121fad2c2ae2daee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * 79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16381) 
   * 5f063bb1db0459bb6a62dd55a57279cf45f4e9fa Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16385) 
   * 596c673ec020e47bff03f76e121fad2c2ae2daee UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1511262328

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340",
       "triggerID" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16380",
       "triggerID" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "triggerType" : "PUSH"
     }, {
       "hash" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16381",
       "triggerID" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f063bb1db0459bb6a62dd55a57279cf45f4e9fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16385",
       "triggerID" : "5f063bb1db0459bb6a62dd55a57279cf45f4e9fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "596c673ec020e47bff03f76e121fad2c2ae2daee",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16389",
       "triggerID" : "596c673ec020e47bff03f76e121fad2c2ae2daee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * 596c673ec020e47bff03f76e121fad2c2ae2daee Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16389) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1168123583


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java:
##########
@@ -18,12 +18,11 @@
 
 package org.apache.hudi.io.storage.row;
 
-import org.apache.hudi.avro.HoodieBloomFilterWriteSupport;
-import org.apache.hudi.common.bloom.BloomFilter;
-
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.avro.HoodieBloomFilterWriteSupport;
+import org.apache.hudi.common.bloom.BloomFilter;
 import org.apache.hudi.common.util.Option;

Review Comment:
   Unnecessary change.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1510809250

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340",
       "triggerID" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16380",
       "triggerID" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "triggerType" : "PUSH"
     }, {
       "hash" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16381",
       "triggerID" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f063bb1db0459bb6a62dd55a57279cf45f4e9fa",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16385",
       "triggerID" : "5f063bb1db0459bb6a62dd55a57279cf45f4e9fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "596c673ec020e47bff03f76e121fad2c2ae2daee",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16389",
       "triggerID" : "596c673ec020e47bff03f76e121fad2c2ae2daee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * 5f063bb1db0459bb6a62dd55a57279cf45f4e9fa Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16385) 
   * 596c673ec020e47bff03f76e121fad2c2ae2daee Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16389) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 merged pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 merged PR #8418:
URL: https://github.com/apache/hudi/pull/8418


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162333501


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   Let's not digress too much from the crux of the issue... 
   
   The issue here is: TIMESTAMP(6) in STREAM mode is writing INT64, while TIMESTAMP(6) in APPEND mode is writing INT96. 
   
   Let's aim to fix that in this PR first.
   
   ## TIMESTAMP(9)
   
   As for TIMESTAMP(9) types, there is currently no support for it AT ALL. If you try to create a table under any mode with TIMESTAMP(9) type, an error will be thrown when u try writing to the table regardless of APPEND / STREAM mode. 
   
   
   As such, regression is not a concern here given that we do not support TIMESTAMP(9) to begin with. The premise here is that TIMESTAMP(6) is writing the wrong physical type to parquet. That' about it... 
   
   P.S. Apologies for the ninja edits.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162321558


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   If we want to support `TIMESTAMP(9)`, opening a separate JIRA issue + PR for it would be better. Let's fix the inconsistency between Stream and Append mode writing different physical types to parquet.
   
   The reason for doing it in another PR is as such:
   
   Stream mode does not allow `TIMESTAMP(9)` types. If Stream mode (upserts) does not allow `TIMESTAMP(9)`, there is no reason to allow `TIMESTAMP(9)` for APPEND mode. 
   
   IIUC, type support should be consistent for Hudi tables under all write modes.
   
   On top of that it's better to separate feature + bug fixes so that people who want to cherrypick fixes into their internal branch will find it easier to do so. 
   
   In short, let's fix the bug in this PR first and align the behaviour for both STREAM and APPEND modes. So, I'll undo the removal of `Timestamp96Writer` and tidy up the tests.
   
   Proper `TIMESTAMP(9)` support should be done in another PR. Whether to implement it as a INT96/BINARY type or INT64 type, we can discuss it in detail in a JIRA/PR.  WDYT?



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162333501


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   Let's not digress too much from the crux of the issue... 
   
   The issue here is: TIMESTAMP(6) in STREAM mode is writing INT64, while TIMESTAMP(6) in APPEND mode is writing INT64. 
   
   Let's aim to fix that in this PR first.
   
   ## TIMESTAMP(9)
   
   As for TIMESTAMP(9) types, there is currently no support for it AT ALL. If you try to create a table under any mode with TIMESTAMP(9) type, an error will be thrown when u try writing to the table regardless of APPEND / STREAM mode. 
   
   



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1165193656


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -124,17 +124,25 @@ private FieldWriter createWriter(LogicalType t) {
         return new DoubleWriter();
       case TIMESTAMP_WITHOUT_TIME_ZONE:
         TimestampType timestampType = (TimestampType) t;
-        if (timestampType.getPrecision() == 3) {
-          return new Timestamp64Writer();
+        final int tsPrecision = timestampType.getPrecision();
+        if (tsPrecision == 3 || tsPrecision == 6) {
+          return new Timestamp64Writer(tsPrecision);
         } else {
-          return new Timestamp96Writer(timestampType.getPrecision());
+          throw new IllegalArgumentException(
+              "Parquet does not support TIMESTAMP type with precision: "
+                  + tsPrecision
+                  + ", it only support precisions <= 6.");
         }
       case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
         LocalZonedTimestampType localZonedTimestampType = (LocalZonedTimestampType) t;
-        if (localZonedTimestampType.getPrecision() == 3) {
-          return new Timestamp64Writer();
+        final int tsLtzPrecision = localZonedTimestampType.getPrecision();
+        if (tsLtzPrecision == 3 || tsLtzPrecision == 6) {
+          return new Timestamp64Writer(tsLtzPrecision);
         } else {
-          return new Timestamp96Writer(localZonedTimestampType.getPrecision());
+          throw new IllegalArgumentException(

Review Comment:
   Done



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -124,17 +124,25 @@ private FieldWriter createWriter(LogicalType t) {
         return new DoubleWriter();
       case TIMESTAMP_WITHOUT_TIME_ZONE:
         TimestampType timestampType = (TimestampType) t;
-        if (timestampType.getPrecision() == 3) {
-          return new Timestamp64Writer();
+        final int tsPrecision = timestampType.getPrecision();
+        if (tsPrecision == 3 || tsPrecision == 6) {
+          return new Timestamp64Writer(tsPrecision);
         } else {
-          return new Timestamp96Writer(timestampType.getPrecision());
+          throw new IllegalArgumentException(
+              "Parquet does not support TIMESTAMP type with precision: "

Review Comment:
   Done



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1506990973

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * ffdb2fcc11b96cd39e1c39bcea07b9974507db69 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308) 
   * e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1506977360

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * ffdb2fcc11b96cd39e1c39bcea07b9974507db69 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308) 
   * e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1507848318

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319) 
   * cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1508052321

   @danny0405 I've addressed the comments + CI is green. Can you please help to review it again, 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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1506879013

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * ffdb2fcc11b96cd39e1c39bcea07b9974507db69 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1510745435

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340",
       "triggerID" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16380",
       "triggerID" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "triggerType" : "PUSH"
     }, {
       "hash" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16381",
       "triggerID" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f063bb1db0459bb6a62dd55a57279cf45f4e9fa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16385",
       "triggerID" : "5f063bb1db0459bb6a62dd55a57279cf45f4e9fa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * 79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16381) 
   * 5f063bb1db0459bb6a62dd55a57279cf45f4e9fa Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16385) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1168230208


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -31,6 +31,7 @@
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.TimestampType;
 import org.apache.flink.util.Preconditions;
+import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.parquet.io.api.Binary;

Review Comment:
   org.apache.hudi package should be alway in the highest import sequence, you many need to import the checkstyle from the file: https://github.com/apache/hudi/blob/d9b29e540314f842d8d3aa86f8429cca8a8cf786/style/checkstyle.xml#L291



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -31,6 +31,7 @@
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.TimestampType;
 import org.apache.flink.util.Preconditions;
+import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.parquet.io.api.Binary;

Review Comment:
   org.apache.hudi package should be always in the highest import sequence, you many need to import the checkstyle from the file: https://github.com/apache/hudi/blob/d9b29e540314f842d8d3aa86f8429cca8a8cf786/style/checkstyle.xml#L291



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1501644808

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7466734a78d74734b423aa68a86f76d1bc2b9a4e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162414909


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   > As for TIMESTAMP(9) types, there is currently no support for it AT ALL
   
   Can the append mode write timestamp(9) then?
   
   > while TIMESTAMP(6) in APPEND mode is writing INT96
   
   This is a mistake that we should fix.
   
   BTW, Spark use the INT96 as the default output timestamp type in their parquet writer: https://github.com/apache/spark/blob/0a63a496bdced946a5d4825ca66df12de51d3a87/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L970



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1506649460

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3f508c7d8e9590ea9f19a106d2ea50a0999b353f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250) 
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * ffdb2fcc11b96cd39e1c39bcea07b9974507db69 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1506638378

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3f508c7d8e9590ea9f19a106d2ea50a0999b353f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250) 
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * ffdb2fcc11b96cd39e1c39bcea07b9974507db69 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162266493


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   Hmmm, avro does not support Timestamp types in the nanos position as of now. 
   
   Reference:
   
   https://github.com/apache/hudi/blob/d6ff3d6ba46b51f58ebb1db58f26423a211741f5/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java#L246
   
   Hence, if you are trying to create a table with a column of `TIMESTAMP(9)` type, there will be a validation exception. This test will pass regardless of whether the fix is applied or not (the error message is a little different, that's about it). 
   
   I added this test in for coverage purposes.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162268620


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   There is some restriction for avro in supporting `TIMESTAMP(9)` but the parquet does support that, let's keep the writer's ability to do that in case in the near future we remove the restriction of avro.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162270386


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   Hmmm, understood. Please correct me if my understanding is wrong:
   
   1. remove conversion support for TIMESTAMP(9)
   2. retain `Timestamp96Writer` 
   
   Is that correct? If so, I'll make the necessary changes, cleanup the tests a little and update the PR's changelog.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162333501


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   Let's not digress too much from the crux of the issue... 
   
   The issue here is: TIMESTAMP(6) in STREAM mode is writing INT64, while TIMESTAMP(6) in APPEND mode is writing INT64. 
   
   Let's aim to fix that in this PR first.
   
   ## TIMESTAMP(9)
   
   As for TIMESTAMP(9) types, there is currently no support for it AT ALL. If you try to create a table under any mode with TIMESTAMP(9) type, an error will be thrown when u try writing to the table regardless of APPEND / STREAM mode. 
   
   
   As such, regression is not a concern here given that we do not support TIMESTAMP(9) to begin with. The premise here is that TIMESTAMP(6) is writing the wrong physical type to parquet. That' about it... 



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162333501


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   Alright then, then maybe the correct fix is to allow clustering to read TIMESTAMP(9) types? 
   
   Spark is able to read out TIMESTAMP(9) types, but it'll only display it in the millis granularity from my tests.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162484156


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   > Can the append mode write timestamp(9) then?
   
   Nope, APPEND can't write TIMESTAMP(9).
   
   > BTW, Spark use the INT96 as the default output timestamp type in their parquet writer: https://github.com/apache/spark/blob/0a63a496bdced946a5d4825ca66df12de51d3a87/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L970
   
   I don't think we are using this by default, writing TIMESTAMP_MICROS with Hudi-on-Spark will write INT64.
   
   We actually set this config as TIMESTAMP_MICROS in Hudi as default.
   
   https://github.com/apache/hudi/blob/d6ff3d6ba46b51f58ebb1db58f26423a211741f5/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java#L123-L126
   
   
   Example:
   
   ```sql
   CREATE TABLE `dev_hudi`.`timestamp_test`
   (
       `id`            INTEGER,
       `bigint_col`    BIGINT,
       `string_col`    STRING,
       `double_col`    DOUBLE,
       `timestamp_col` TIMESTAMP,
       `operation`     STRING
   ) USING hudi
   TBLPROPERTIES (
     'primaryKey' = 'id',
     'type' = 'cow',
     'preCombineField' = 'bigint_col'
   )
   LOCATION 'hdfs://path/to/timestamp_test';
   
   -- use nanos, however, this will fallback to micros
   INSERT INTO `dev_hudi`.`timestamp_test`
   VALUES (1, 1000, "string_col_1", 1.1, TIMESTAMP "1970-01-01 00:00:01.001001001", "init"),
          (2, 2000, "string_col_2", 2.2, TIMESTAMP "1970-01-01 00:00:02.001001001", "init");
   
   
   SELECT * FROM `dev_hudi`.`timestamp_test`;
   
   20230411163354949	20230411163354949_0_0	1		5ea1112a-3f7d-4c6a-8f20-5275055ee330-0_0-17-20_20230411163354949.parquet	1	1000	string_col_1	1.1	1970-01-01 00:00:01.001001	init
   20230411163354949	20230411163354949_0_1	2		5ea1112a-3f7d-4c6a-8f20-5275055ee330-0_0-17-20_20230411163354949.parquet	2	2000	string_col_2	2.2	1970-01-01 00:00:02.001001	init
   ```
   
   parquet-tools snippet:
   
   ```
   ############ Column(timestamp_col)[row group 0] ############
   name: timestamp_col
   path: timestamp_col
   max_definition_level: 1
   max_repetition_level: 0
   physical_type: INT64
   logical_type: Timestamp(isAdjustedToUTC=true, timeUnit=microseconds, is_from_converted_type=true, force_set_converted_type=false)
   converted_type (legacy): TIMESTAMP_MICROS
   compression: GZIP (space_saved: -20%)
   total_compressed_size: 100
   total_uncompressed_size: 83
   ```
   
   
   



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162484156


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   > Can the append mode write timestamp(9) then?
   
   Nope, APPEND can't write TIMESTAMP(9).
   
   > BTW, Spark use the INT96 as the default output timestamp type in their parquet writer: https://github.com/apache/spark/blob/0a63a496bdced946a5d4825ca66df12de51d3a87/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L970
   
   I don't think we are using this by default, writing TIMESTAMP_MICROS with Hudi-on-Spark will write INT64.
   
   We actually set this config as TIMESTAMP_MICROS in Hudi.
   
   https://github.com/apache/hudi/blob/d6ff3d6ba46b51f58ebb1db58f26423a211741f5/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java#L123-L126
   
   
   Example:
   
   ```sql
   CREATE TABLE `dev_hudi`.`timestamp_test`
   (
       `id`            INTEGER,
       `bigint_col`    BIGINT,
       `string_col`    STRING,
       `double_col`    DOUBLE,
       `timestamp_col` TIMESTAMP,
       `operation`     STRING
   ) USING hudi
   TBLPROPERTIES (
     'primaryKey' = 'id',
     'type' = 'cow',
     'preCombineField' = 'bigint_col'
   )
   LOCATION 'hdfs://path/to/timestamp_test';
   
   -- use nanos, however, this will fallback to micros
   INSERT INTO `dev_hudi`.`timestamp_test`
   VALUES (1, 1000, "string_col_1", 1.1, TIMESTAMP "1970-01-01 00:00:01.001001001", "init"),
          (2, 2000, "string_col_2", 2.2, TIMESTAMP "1970-01-01 00:00:02.001001001", "init");
   
   
   SELECT * FROM `dev_hudi`.`timestamp_test`;
   
   20230411163354949	20230411163354949_0_0	1		5ea1112a-3f7d-4c6a-8f20-5275055ee330-0_0-17-20_20230411163354949.parquet	1	1000	string_col_1	1.1	1970-01-01 00:00:01.001001	init
   20230411163354949	20230411163354949_0_1	2		5ea1112a-3f7d-4c6a-8f20-5275055ee330-0_0-17-20_20230411163354949.parquet	2	2000	string_col_2	2.2	1970-01-01 00:00:02.001001	init
   ```
   
   parquet-tools snippet:
   
   ```
   ############ Column(timestamp_col)[row group 0] ############
   name: timestamp_col
   path: timestamp_col
   max_definition_level: 1
   max_repetition_level: 0
   physical_type: INT64
   logical_type: Timestamp(isAdjustedToUTC=true, timeUnit=microseconds, is_from_converted_type=true, force_set_converted_type=false)
   converted_type (legacy): TIMESTAMP_MICROS
   compression: GZIP (space_saved: -20%)
   total_compressed_size: 100
   total_uncompressed_size: 83
   ```
   
   
   



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1501754855

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7466734a78d74734b423aa68a86f76d1bc2b9a4e Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1165191338


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java:
##########
@@ -600,23 +600,29 @@ private static Type convertToParquetType(
             .named(name);
       case TIMESTAMP_WITHOUT_TIME_ZONE:
         TimestampType timestampType = (TimestampType) type;
-        if (timestampType.getPrecision() == 3) {
+        if (timestampType.getPrecision() == 3 || timestampType.getPrecision() == 6) {
+          TimeUnit timeunit = timestampType.getPrecision() == 3 ? TimeUnit.MILLIS : TimeUnit.MICROS;
           return Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition)
-              .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS))
+              .as(LogicalTypeAnnotation.timestampType(true, timeunit))
               .named(name);
         } else {
-          return Types.primitive(PrimitiveType.PrimitiveTypeName.INT96, repetition)
-              .named(name);
+          throw new IllegalArgumentException(
+              "Parquet does not support TIMESTAMP type with precision: "
+                  + timestampType.getPrecision()

Review Comment:
   Done



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1162311218


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -419,4 +425,179 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception {
         .stream().anyMatch(fg -> fg.getSlices()
             .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant))));
   }
+
+  /**
+   * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors
+   * @throws Exception
+   */
+  @Test
+  public void testHoodieFlinkClusteringWithTimestampNanos() {
+    // create hoodie table and insert into data

Review Comment:
   From the parquet reference, it suggest to use the INT64 to represent the `NANOS` precision timestamp, even though there is range limination. So for writer, I would suggest to remove the `Timestamp96Writer`, instead we use the INT64 for the timestamp(9) type, and for reader, we need to be compatible with the INT96 legacy type.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] voonhous commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "voonhous (via GitHub)" <gi...@apache.org>.
voonhous commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1168173022


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java:
##########
@@ -18,6 +18,11 @@
 
 package org.apache.hudi.sink.cluster;
 
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.api.ValidationException;

Review Comment:
   Done



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8418:
URL: https://github.com/apache/hudi/pull/8418#discussion_r1168230549


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java:
##########
@@ -31,6 +31,7 @@
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.TimestampType;
 import org.apache.flink.util.Preconditions;
+import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.parquet.io.api.Binary;

Review Comment:
   Take other Java files for reference of import sequence.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8418: [HUDI-6052] Standardise TIMESTAMP(6) format when writing to Parquet f…

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8418:
URL: https://github.com/apache/hudi/pull/8418#issuecomment-1510686260

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16225",
       "triggerID" : "7466734a78d74734b423aa68a86f76d1bc2b9a4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16250",
       "triggerID" : "3f508c7d8e9590ea9f19a106d2ea50a0999b353f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1579a6e6966b051e9a6a5b696a9e6e35500a929a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16308",
       "triggerID" : "ffdb2fcc11b96cd39e1c39bcea07b9974507db69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16319",
       "triggerID" : "e3b8fc000ef3a7d57dd23aec1c9b37afaeb3c2dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16340",
       "triggerID" : "cb05421be9bb950f7dadfc6a5cdfa4c07e5de6a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16380",
       "triggerID" : "be4235b4f42ffe845ec17dad7113af2de7a94332",
       "triggerType" : "PUSH"
     }, {
       "hash" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16381",
       "triggerID" : "79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1579a6e6966b051e9a6a5b696a9e6e35500a929a UNKNOWN
   * be4235b4f42ffe845ec17dad7113af2de7a94332 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16380) 
   * 79bc2ba070ecbcbdeab6ff88f9424cbd5a6b5e49 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16381) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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