You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hive.apache.org by "xsys (Jira)" <ji...@apache.org> on 2022/09/09 05:07:00 UTC

[jira] [Updated] (HIVE-26528) TIMESTAMP stored via spark-shell DataFrame to Avro returns incorrect value when read using HiveCLI

     [ https://issues.apache.org/jira/browse/HIVE-26528?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

xsys updated HIVE-26528:
------------------------
    Description: 
h2. Describe the bug

We are trying to store a TIMESTAMP \{{"2022" }}to a table created via Spark DataFrame. The table is created with the Avro file format. We encounter no errors while creating the table and inserting the aforementioned timestamp value. However, performing a SELECT query on the table through HiveCLI returns an incorrect value: "+53971-10-02 19:00:0000"

The root cause for this issue is the fact that Spark's [AvroSerializer|https://github.com/apache/spark/blob/v3.2.1/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala#L171-L180] serializes timestamps using Avro's [TIMESTAMP_MICRO|https://github.com/apache/avro/blob/ee4725c64807549ec74e20e83d35cfc1fe8e90a8/lang/java/avro/src/main/java/org/apache/avro/LogicalTypes.java#L190] while Hive's [AvroDeserializer|https://github.com/apache/hive/blob/rel/release-3.1.2/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroDeserializer.java#L320-L347] assumes timestamps to be Avro's [TIMESTAMP_MILLIS|#L189] during deserialization.
h2. Step to reproduce

On Spark 3.2.1 (commit `4f25b3f712`), using `spark-shell` with the Avro package:
{code:java}
./bin/spark-shell --packages org.apache.spark:spark-avro_2.12:3.2.1{code}
Execute the following:
{code:java}
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.types._
val rdd = sc.parallelize(Seq(Row(Seq("2022").toDF("time").select(to_timestamp(col("time")).as("to_timestamp")).first().getAs[java.sql.Timestamp](0))))
val schema = new StructType().add(StructField("c1", TimestampType, true))
val df = spark.createDataFrame(rdd, schema)
df.show(false)
df.write.mode("overwrite").format("avro").saveAsTable("ws") {code}
 

On [Hive 3.1.2|https://archive.apache.org/dist/hive/hive-3.1.2/apache-hive-3.1.2-bin.tar.gz], execute the following:
{noformat}
hive> select * from ws;
OK
+53971-10-02 19:00:0000{noformat}
h2. Expected behavior

We expect the output of the {{SELECT}} query to be "{{{}2022-01-01 00:00:00".{}}}We tried other formats like Parquet and the outcome is consistent with this expectation. Moreover, the timestamp is interpreted correctly when the table is written to via DataFrame and read via spark-shell/spark-sql:
h3. Can be read correctly from spark-shell:
{code:java}
scala> spark.sql("select * from ws;").show(false)
+-------------------+
|c1                 |
+-------------------+
|2022-01-01 00:00:00|
+-------------------+{code}
h3. Can be read correctly from spark-sql:
{noformat}
spark-sql> select * from ws;
2022-01-01 00:00:00
Time taken: 0.063 seconds, Fetched 1 row(s){noformat}
 

  was:
h2. Describe the bug

We are trying to store a TIMESTAMP \{{"2022" }}to a table created via Spark DataFrame. The table is created with the Avro file format. We encounter no errors while creating the table and inserting the aforementioned timestamp value. However, performing a SELECT query on the table through HiveCLI returns an incorrect value: "+53971-10-02 19:00:0000"

The root cause for this issue is the fact that Spark's [AvroSerializer|https://github.com/apache/spark/blob/v3.2.1/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala#L171-L180] serializes timestamps using Avro's [TIMESTAMP_MICRO|https://github.com/apache/avro/blob/ee4725c64807549ec74e20e83d35cfc1fe8e90a8/lang/java/avro/src/main/java/org/apache/avro/LogicalTypes.java#L190] while Hive's [AvroDeserializer|https://github.com/apache/hive/blob/rel/release-3.1.2/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroDeserializer.java#L320-L347] assumes timestamps to be Avro's [TIMESTAMP_MILLIS|#L189] during deserialization.
h2. Step to reproduce

On Spark 3.2.1 (commit `4f25b3f712`), using `spark-shell` with the Avro package:
{code:java}
./bin/spark-shell --packages org.apache.spark:spark-avro_2.12:3.2.1{code}
 

Execute the following:
{code:java}
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.types._
val rdd = sc.parallelize(Seq(Row(Seq("2022").toDF("time").select(to_timestamp(col("time")).as("to_timestamp")).first().getAs[java.sql.Timestamp](0))))
val schema = new StructType().add(StructField("c1", TimestampType, true))
val df = spark.createDataFrame(rdd, schema)
df.show(false)
df.write.mode("overwrite").format("avro").saveAsTable("ws") {code}
 

On [Hive 3.1.2|https://archive.apache.org/dist/hive/hive-3.1.2/apache-hive-3.1.2-bin.tar.gz], execute the following:
{noformat}
hive> select * from ws;
OK
+53971-10-02 19:00:0000{noformat}
 
h2. Expected behavior

We expect the output of the {{SELECT}} query to be "{{{}2022-01-01 00:00:00".{}}}We tried other formats like Parquet and the outcome is consistent with this expectation. Moreover, the timestamp is interpreted correctly when the table is written to via DataFrame and read via spark-shell/spark-sql:
h3. Can be read correctly from spark-shell:

 
{code:java}
scala> spark.sql("select * from ws;").show(false)
+-------------------+
|c1                 |
+-------------------+
|2022-01-01 00:00:00|
+-------------------+{code}
h3. Can be read correctly from spark-sql:

 
{noformat}
spark-sql> select * from ws;
2022-01-01 00:00:00
Time taken: 0.063 seconds, Fetched 1 row(s){noformat}
 


> TIMESTAMP stored via spark-shell DataFrame to Avro returns incorrect value when read using HiveCLI
> --------------------------------------------------------------------------------------------------
>
>                 Key: HIVE-26528
>                 URL: https://issues.apache.org/jira/browse/HIVE-26528
>             Project: Hive
>          Issue Type: Bug
>          Components: Serializers/Deserializers
>    Affects Versions: 3.1.2
>            Reporter: xsys
>            Priority: Major
>
> h2. Describe the bug
> We are trying to store a TIMESTAMP \{{"2022" }}to a table created via Spark DataFrame. The table is created with the Avro file format. We encounter no errors while creating the table and inserting the aforementioned timestamp value. However, performing a SELECT query on the table through HiveCLI returns an incorrect value: "+53971-10-02 19:00:0000"
> The root cause for this issue is the fact that Spark's [AvroSerializer|https://github.com/apache/spark/blob/v3.2.1/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala#L171-L180] serializes timestamps using Avro's [TIMESTAMP_MICRO|https://github.com/apache/avro/blob/ee4725c64807549ec74e20e83d35cfc1fe8e90a8/lang/java/avro/src/main/java/org/apache/avro/LogicalTypes.java#L190] while Hive's [AvroDeserializer|https://github.com/apache/hive/blob/rel/release-3.1.2/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroDeserializer.java#L320-L347] assumes timestamps to be Avro's [TIMESTAMP_MILLIS|#L189] during deserialization.
> h2. Step to reproduce
> On Spark 3.2.1 (commit `4f25b3f712`), using `spark-shell` with the Avro package:
> {code:java}
> ./bin/spark-shell --packages org.apache.spark:spark-avro_2.12:3.2.1{code}
> Execute the following:
> {code:java}
> import org.apache.spark.sql.{Row, SparkSession}
> import org.apache.spark.sql.types._
> val rdd = sc.parallelize(Seq(Row(Seq("2022").toDF("time").select(to_timestamp(col("time")).as("to_timestamp")).first().getAs[java.sql.Timestamp](0))))
> val schema = new StructType().add(StructField("c1", TimestampType, true))
> val df = spark.createDataFrame(rdd, schema)
> df.show(false)
> df.write.mode("overwrite").format("avro").saveAsTable("ws") {code}
>  
> On [Hive 3.1.2|https://archive.apache.org/dist/hive/hive-3.1.2/apache-hive-3.1.2-bin.tar.gz], execute the following:
> {noformat}
> hive> select * from ws;
> OK
> +53971-10-02 19:00:0000{noformat}
> h2. Expected behavior
> We expect the output of the {{SELECT}} query to be "{{{}2022-01-01 00:00:00".{}}}We tried other formats like Parquet and the outcome is consistent with this expectation. Moreover, the timestamp is interpreted correctly when the table is written to via DataFrame and read via spark-shell/spark-sql:
> h3. Can be read correctly from spark-shell:
> {code:java}
> scala> spark.sql("select * from ws;").show(false)
> +-------------------+
> |c1                 |
> +-------------------+
> |2022-01-01 00:00:00|
> +-------------------+{code}
> h3. Can be read correctly from spark-sql:
> {noformat}
> spark-sql> select * from ws;
> 2022-01-01 00:00:00
> Time taken: 0.063 seconds, Fetched 1 row(s){noformat}
>  



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