You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Describe the bug
TPCDS tests FAILED On SPARK-3.2.0, com.nvidia.spark.rapids.SerializedTableColumn cannot be cast to com.nvidia.spark.rapids.GpuColumnVector
WARN TaskSetManager: Lost task 0.0 in stage 45.0 (TID 34) (10.233.113.204 executor 0): org.apache.****.SparkException: Task failed while writing rows.
at org.apache.spark.sql.rapids.GpuFileFormatWriter$.executeTask(GpuFileFormatWriter.scala:323)
at org.apache.spark.sql.rapids.GpuFileFormatWriter$.$anonfun$write$13(GpuFileFormatWriter.scala:243)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
at org.apache.spark.scheduler.Task.run(Task.scala:131)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
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.ClassCastException: com.nvidia.spark.rapids.SerializedTableColumn cannot be cast to com.nvidia.spark.rapids.GpuColumnVector
at com.nvidia.spark.rapids.GpuColumnVector.extractColumns(GpuColumnVector.java:995)
at com.nvidia.spark.rapids.GpuParquetWriter.write(GpuParquetFileFormat.scala:293)
at org.apache.spark.sql.rapids.GpuSingleDirectoryDataWriter.write(GpuFileFormatDataWriter.scala:163)
at org.apache.spark.sql.rapids.GpuFileFormatWriter$.$anonfun$executeTask$1(GpuFileFormatWriter.scala:311)
at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1496)
at org.apache.spark.sql.rapids.GpuFileFormatWriter$.executeTask(GpuFileFormatWriter.scala:308)
... 9 more
5:> (0 + 1) / 1]
07:51:18 ERROR TaskSetManager: Task 0 in stage 45.0 failed 4 times; aborting job
21/12/12 07:51:18 ERROR GpuFileFormatWriter: Aborting job 881bd8b9-c73e-4d66-b274-491c6d016d4f.
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 45.0 failed 4 times, most recent failure: Lost task 0.3 in stage 45.0 (TID 37) (10.233.113.204 executor 0): org.apache.spark.SparkException: Task failed while writing rows.
at org.apache.spark.sql.rapids.GpuFileFormatWriter$.executeTask(GpuFileFormatWriter.scala:323)
at org.apache.spark.sql.rapids.GpuFileFormatWriter$.$anonfun$write$13(GpuFileFormatWriter.scala:243)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
at org.apache.spark.scheduler.Task.run(Task.scala:131)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
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.ClassCastException: com.nvidia.spark.rapids.SerializedTableColumn cannot be cast to com.nvidia.spark.rapids.GpuColumnVector
at com.nvidia.spark.rapids.GpuColumnVector.extractColumns(GpuColumnVector.java:995)
at com.nvidia.spark.rapids.GpuParquetWriter.write(GpuParquetFileFormat.scala:293)
at org.apache.spark.sql.rapids.GpuSingleDirectoryDataWriter.write(GpuFileFormatDataWriter.scala:163)
at org.apache.spark.sql.rapids.GpuFileFormatWriter$.$anonfun$executeTask$1(GpuFileFormatWriter.scala:311)
at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1496)
at org.apache.spark.sql.rapids.GpuFileFormatWriter$.executeTask(GpuFileFormatWriter.scala:308)
... 9 more
Driver stacktrace:
at org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2403)
at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2352)
at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2351)
at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2351)
at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:1109)
at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:1109)
at scala.Option.foreach(Option.scala:407)
at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:1109)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2591)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2533)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2522)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:898)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2214)
at org.apache.spark.sql.rapids.GpuFileFormatWriter$.write(GpuFileFormatWriter.scala:233)
at org.apache.spark.sql.rapids.GpuInsertIntoHadoopFsRelationCommand.runColumnar(GpuInsertIntoHadoopFsRelationCommand.scala:166)
at com.nvidia.spark.rapids.GpuDataWritingCommandExec.sideEffectResult$lzycompute(GpuDataWritingCommandExec.scala:97)
at com.nvidia.spark.rapids.GpuDataWritingCommandExec.sideEffectResult(GpuDataWritingCommandExec.scala:96)
at com.nvidia.spark.rapids.GpuDataWritingCommandExec.doExecuteColumnar(GpuDataWritingCommandExec.scala:120)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeColumnar$1(SparkPlan.scala:211)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:222)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:219)
at org.apache.spark.sql.execution.SparkPlan.executeColumnar(SparkPlan.scala:207)
at com.nvidia.spark.rapids.GpuColumnarToRowExecParent.doExecute(GpuColumnarToRowExec.scala:309)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:184)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:222)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:219)
at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:180)
at org.apache.spark.sql.execution.SparkPlan.getByteArrayRdd(SparkPlan.scala:325)
at org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:391)
at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:110)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:110)
at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:106)
at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:481)
at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:82)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:481)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:457)
at org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:106)
at org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:93)
at org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:91)
at org.apache.spark.sql.execution.QueryExecution.assertCommandExecuted(QueryExecution.scala:128)
at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:848)
at org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:382)
at org.apache.spark.sql.DataFrameWriter.saveInternal(DataFrameWriter.scala:355)
at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:239)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
at py4j.Gateway.invoke(Gateway.java:282)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.ClientServerConnection.waitForCommands(ClientServerConnection.java:182)
at py4j.ClientServerConnection.run(ClientServerConnection.java:106)
at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.spark.SparkException: Task failed while writing rows.
at org.apache.spark.sql.rapids.GpuFileFormatWriter$.executeTask(GpuFileFormatWriter.scala:323)
at org.apache.spark.sql.rapids.GpuFileFormatWriter$.$anonfun$write$13(GpuFileFormatWriter.scala:243)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
at org.apache.spark.scheduler.Task.run(Task.scala:131)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
... 1 more
Caused by: java.lang.ClassCastException: com.nvidia.spark.rapids.SerializedTableColumn cannot be cast to com.nvidia.spark.rapids.GpuColumnVector
at com.nvidia.spark.rapids.GpuColumnVector.extractColumns(GpuColumnVector.java:995)
at com.nvidia.spark.rapids.GpuParquetWriter.write(GpuParquetFileFormat.scala:293)
at org.apache.spark.sql.rapids.GpuSingleDirectoryDataWriter.write(GpuFileFormatDataWriter.scala:163)
at org.apache.spark.sql.rapids.GpuFileFormatWriter$.$anonfun$executeTask$1(GpuFileFormatWriter.scala:311)
at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1496)
at org.apache.spark.sql.rapids.GpuFileFormatWriter$.executeTask(GpuFileFormatWriter.scala:308)
... 9 more
Traceback (most recent call last):
Steps/Code to reproduce bug
Run notebook: https://github.com/NVIDIA/spark-rapids-examples/blob/branch-21.12/examples/micro-benchmarks/notebooks/micro-benchmarks-gpu.ipynb
Additional context
Test PASS on spark-3.0.x and spark3.1.x
The text was updated successfully, but these errors were encountered:
Before #3144 this generates a SQL plan with the following sequence:
GpuLocalLimit->GpuColumnarExchange->GpuShuffleCoalesce->GpuGlobalLimit->GpuColumnarToRow->AdaptiveSparkPlan->GpuRowToColumnar->AvoidAdaptiveTransitionToRow->GpuInsertIntoHadoopFsRelationCommand
After #3144 the generated SQL plan looks like the following invalid plan:
GpuLocalLimit->GpuColumnarExchange->GpuGlobalLimit->AdaptiveSparkPlan->GpuInsertIntoHadoopFsRelationCommand
There's a missing GpuShuffleCoalesce after the GpuColumnarExchange that triggers the error. GpuShuffleCoalesce must appear after each GpuColumnarExchange when using Spark's built-in shuffle to convert the SerializedTableColumn into a GpuColumnVector.
Describe the bug
TPCDS tests FAILED On SPARK-3.2.0, com.nvidia.spark.rapids.SerializedTableColumn cannot be cast to com.nvidia.spark.rapids.GpuColumnVector
Steps/Code to reproduce bug
Run notebook:
https://github.com/NVIDIA/spark-rapids-examples/blob/branch-21.12/examples/micro-benchmarks/notebooks/micro-benchmarks-gpu.ipynb
Additional context
Test PASS on
spark-3.0.x
andspark3.1.x
The text was updated successfully, but these errors were encountered: