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

koert kuipers edited comment on SPARK-31183 at 3/22/20, 2:03 PM:
-----------------------------------------------------------------

hi!

all spark tests pass for me except hive and avro. hive always seems to fail and 
i never figured out why so i ignore it. but avro failure is new and i think its 
related to this ticket.

i see errors like this:
{code:java}
- SPARK-31183: rebasing milliseconds timestamps in write *** FAILED ***         
                                                              [3529/9624]
  org.apache.spark.SparkException: Job aborted.                                 
                                                                         
  at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:226)
                                                      
  at 
org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:178)
                       
  at 
org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:108)
                                       
  at 
org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:106)
  at 
org.apache.spark.sql.execution.command.DataWritingCommandExec.doExecute(commands.scala:131)
  at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:175)
  at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:213)
  at 
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:210)
  at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:171)
  ...
  Cause: org.apache.spark.SparkException: Job aborted due to stage failure: 
Task 0 in stage 186.0 failed 1 times, most recent failure: Lost task 0.0 in $
tage 186.0 (TID 316, server04.tresata.com, executor driver): 
org.apache.spark.SparkException: Task failed while writing rows.
        at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:291)
        at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:205)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:127)
        at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:444)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:447)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.ArithmeticException: long overflow
        at java.lang.Math.multiplyExact(Math.java:892)
        at 
org.apache.spark.sql.catalyst.util.DateTimeUtils$.fromMillis(DateTimeUtils.scala:185)
        at 
org.apache.spark.sql.avro.AvroSerializer.$anonfun$newConverter$17(AvroSerializer.scala:155)
        at 
org.apache.spark.sql.avro.AvroSerializer.$anonfun$newConverter$17$adapted(AvroSerializer.scala:152)
        at 
org.apache.spark.sql.avro.AvroSerializer.$anonfun$newStructConverter$2(AvroSerializer.scala:245)
        at 
org.apache.spark.sql.avro.AvroSerializer.serialize(AvroSerializer.scala:51)
        at 
org.apache.spark.sql.avro.AvroOutputWriter.write(AvroOutputWriter.scala:64)
        at 
org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.write(FileFormatDataWriter.scala:140)
        at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$executeTask$1(FileFormatWriter.scala:273)
        at 
org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1411)
        at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:281)
{code}
this is on branch-3.0

on master it seems all tests pass i think. i will check again.


was (Author: koert):
hi!

all spark tests pass for me except hive and avro. hive always seems to fail and 
i never figured out why so i ignore it. but avro failure is new and i think its 
related to this ticket.

i see errors like this:
{code:java}
- SPARK-31183: rebasing milliseconds timestamps in write *** FAILED ***         
                                                              [3529/9624]
  org.apache.spark.SparkException: Job aborted.                                 
                                                                         
  at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:226)
                                                      
  at 
org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:178)
                       
  at 
org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:108)
                                       
  at 
org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:106)
  at 
org.apache.spark.sql.execution.command.DataWritingCommandExec.doExecute(commands.scala:131)
  at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:175)
  at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:213)
  at 
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:210)
  at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:171)
  ...
  Cause: org.apache.spark.SparkException: Job aborted due to stage failure: 
Task 0 in stage 186.0 failed 1 times, most recent failure: Lost task 0.0 in $
tage 186.0 (TID 316, server04.tresata.com, executor driver): 
org.apache.spark.SparkException: Task failed while writing rows.
        at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:291)
        at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:205)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:127)
        at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:444)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:447)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.ArithmeticException: long overflow
        at java.lang.Math.multiplyExact(Math.java:892)
        at 
org.apache.spark.sql.catalyst.util.DateTimeUtils$.fromMillis(DateTimeUtils.scala:185)
        at 
org.apache.spark.sql.avro.AvroSerializer.$anonfun$newConverter$17(AvroSerializer.scala:155)
        at 
org.apache.spark.sql.avro.AvroSerializer.$anonfun$newConverter$17$adapted(AvroSerializer.scala:152)
        at 
org.apache.spark.sql.avro.AvroSerializer.$anonfun$newStructConverter$2(AvroSerializer.scala:245)
        at 
org.apache.spark.sql.avro.AvroSerializer.serialize(AvroSerializer.scala:51)
        at 
org.apache.spark.sql.avro.AvroOutputWriter.write(AvroOutputWriter.scala:64)
        at 
org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.write(FileFormatDataWriter.scala:140)
        at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$executeTask$1(FileFormatWriter.scala:273)
        at 
org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1411)
        at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:281)
{code}

> Incompatible Avro dates/timestamps with Spark 2.4
> -------------------------------------------------
>
>                 Key: SPARK-31183
>                 URL: https://issues.apache.org/jira/browse/SPARK-31183
>             Project: Spark
>          Issue Type: Sub-task
>          Components: SQL
>    Affects Versions: 3.0.0
>            Reporter: Maxim Gekk
>            Assignee: Maxim Gekk
>            Priority: Major
>             Fix For: 3.0.0
>
>
> Write dates/timestamps to Avro file in Spark 2.4.5:
> {code}
> $ export TZ="America/Los_Angeles"
> $ bin/spark-shell --packages org.apache.spark:spark-avro_2.11:2.4.5
> {code}
> {code:scala}
> scala> 
> df.write.format("avro").save("/Users/maxim/tmp/before_1582/2_4_5_ts_avro")
> scala> 
> spark.read.format("avro").load("/Users/maxim/tmp/before_1582/2_4_5_ts_avro").show(false)
> +----------+
> |date      |
> +----------+
> |1001-01-01|
> +----------+
> scala> 
> df2.write.format("avro").save("/Users/maxim/tmp/before_1582/2_4_5_ts_avro")
> scala> 
> spark.read.format("avro").load("/Users/maxim/tmp/before_1582/2_4_5_ts_avro").show(false)
> +--------------------------+
> |ts                        |
> +--------------------------+
> |1001-01-01 01:02:03.123456|
> +--------------------------+
> {code}
> Spark 3.0.0-preview2 ( and 3.1.0-SNAPSHOT) outputs different values from 
> Spark 2.4.5:
> {code}
> $ export TZ="America/Los_Angeles"
> $ /bin/spark-shell --packages org.apache.spark:spark-avro_2.12:2.4.5
> {code}
> {code:scala}
> scala> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
> scala> 
> spark.read.format("avro").load("/Users/maxim/tmp/before_1582/2_4_5_date_avro").show(false)
> +----------+
> |date      |
> +----------+
> |1001-01-07|
> +----------+
> scala> 
> spark.read.format("avro").load("/Users/maxim/tmp/before_1582/2_4_5_ts_avro").show(false)
> +--------------------------+
> |ts                        |
> +--------------------------+
> |1001-01-07 01:09:05.123456|
> +--------------------------+
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to