[ 
https://issues.apache.org/jira/browse/HIVE-26612?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17616502#comment-17616502
 ] 

Stamatis Zampetakis commented on HIVE-26612:
--------------------------------------------

It seems that this ticket is closely related to HIVE-23345. 

Basically the problem in both cases (here and HIVE-23345) is that we are 
storing something as a timestamp and we want to read it back as a bigint.

When we create the Parquet file we are saying that the column holds timestamps. 
It seems that the writer makes use of the 
[LogicalType.Timestamp|https://github.com/apache/parquet-format/blob/54e53e5d7794d383529dd30746378f19a12afd58/LogicalTypes.md?plain=1#L337].

The Parquet [type 
specification|https://github.com/apache/parquet-format/blob/54e53e5d7794d383529dd30746378f19a12afd58/LogicalTypes.md?plain=1#L23]
 writes the following:
_Logical types are used to extend the types that parquet can be used to store,
by specifying how the primitive types should be interpreted_

>From my perspective the fact that we are trying to interpret a timestamp as a 
>bigint is more like a user problem rather than a Hive problem.

HIVE-23345 tried to support this kind of conversion although the motivation 
behind that change remains unknown.

The motivation for the change here is also somewhat questionable. 
* If we are storing timestamps then why not read it back as timestamps?
* If further we want to transform timestamps to bigints can't we use a cast in 
the query?

> Hive cannot read parquet files with int64 (TIMESTAMP_MILLIS)
> ------------------------------------------------------------
>
>                 Key: HIVE-26612
>                 URL: https://issues.apache.org/jira/browse/HIVE-26612
>             Project: Hive
>          Issue Type: Bug
>          Components: Database/Schema
>            Reporter: Steve Carlin
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> If a parquet file has a Type of "int64 eventtime (TIMESTAMP(MILLIS,true))", 
> the following error is produced:
> exec.Task: Failed with exception 
> java.io.IOException:org.apache.parquet.io.ParquetDecodingException: Can not 
> read value at 1 in block 0 in file 
> file:/home/steve/upstream/hive/itests/qtest/target/tmp/parquet_format_ts_as_bigint/part-00000/timestamp_as_bigint.parquet
> java.io.IOException: org.apache.parquet.io.ParquetDecodingException: Can not 
> read value at 1 in block 0 in file 
> file:/home/steve/upstream/hive/itests/qtest/target/tmp/parquet_format_ts_as_bigint/part-00000/timestamp_as_bigint.parquet
>         at 
> org.apache.hadoop.hive.ql.exec.FetchOperator.getNextRow(FetchOperator.java:624)
>         at 
> org.apache.hadoop.hive.ql.exec.FetchOperator.pushRow(FetchOperator.java:531)
>         at 
> org.apache.hadoop.hive.ql.exec.FetchTask.executeInner(FetchTask.java:197)
>         at org.apache.hadoop.hive.ql.exec.FetchTask.execute(FetchTask.java:98)
> The parquet file can be created with the following steps (through spark):
> spark.conf.set("spark.sql.parquet.outputTimestampType", "TIMESTAMP_MILLIS")
> spark.conf.set("spark.sql.legacy.parquet.int96RebaseModeInWrite", "LEGACY")
> spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInWrite", "LEGACY")
> spark.conf.set("spark.sql.legacy.parquet.int96RebaseModeInRead", "LEGACY")
> spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "LEGACY")
> [1]
> val df = Seq(
> (1, Timestamp.valueOf("2014-01-01 23:00:01")),
> (1, Timestamp.valueOf("2014-11-30 12:40:32")),
> (2, Timestamp.valueOf("2016-12-29 09:54:00")),
> (2, Timestamp.valueOf("2016-05-09 10:12:43"))
> ).toDF("typeid","eventtime")
> [2]
> [root@c4839-node3 test_parquet2]# parquet-tools schema 
> part-00001-6c90b794-90b9-4cc0-afc5-2e49a4e96bad-c000.snappy.parquet
> message spark_schema {
> required int32 typeid;
> optional int64 eventtime (TIMESTAMP(MILLIS,true));
> }
> [3]
> [root@c4839-node3 test_parquet1]# parquet-tools schema 
> part-00001-cb1aeebb-ec87-4273-82ec-911c4fb605b6-c000.snappy.parquet
> message spark_schema {
> required int32 typeid;
> optional int96 eventtime;
> }



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to