-
Notifications
You must be signed in to change notification settings - Fork 264
Open
Labels
bugSomething isn't workingSomething isn't working
Description
Describe the bug
An error (data with no columns should have been filtered out already) occurred when join an empty table reading from the csv file on the GPU engines.
Steps/Code to reproduce bug
import os
os.environ['JAVA_HOME'] = "/usr/lib/jvm/java-17-openjdk-amd64"
os.environ['SPARK_HOME'] = "./spark-4.0.1-bin-hadoop3"
os.environ['PYSPARK_SUBMIT_ARGS'] = "--jars ./rapids-4-spark_2.13-25.10.0.jar,./cudf-25.10.0-cuda12.jar --master local[*] pyspark-shell"
import findspark
findspark.init()
from pyspark.sql import SparkSession
from pyspark.conf import SparkConf
from pyspark.sql import functions as F
spark = SparkSession.builder.appName('SparkRAPIDS').config('spark.plugins','com.nvidia.spark.SQLPlugin').config("spark.executor.memory", "8g").config("spark.driver.memory", "8g").getOrCreate()
spark.sparkContext.addPyFile('./rapids-4-spark_2.13-25.10.0.jar')
spark.sparkContext.addPyFile('./cudf-25.10.0-cuda12.jar')
spark.conf.set('spark.rapids.sql.incompatibleOps.enabled', 'true')
spark.conf.set('spark.rapids.sql.format.csv.read.enabled', 'true')
spark.conf.set('spark.rapids.sql.format.csv.enabled', 'true')
spark.conf.set("spark.executor.resource.gpu.amount", "1")
spark.conf.set("spark.task.resource.gpu.amount", "1")
spark.conf.set("spark.rapids.sql.concurrentGpuTasks", "1")
spark.conf.set("spark.rapids.sql.exec.CollectLimitExec", "true")
spark.conf.set('spark.rapids.sql.enabled', 'false')
df = spark.read.csv("t1.csv", header=True, inferSchema=True)
df.createOrReplaceTempView("t1")
df = spark.read.csv("t2.csv", header=True, inferSchema=True)
df.createOrReplaceTempView("t2")
print("CPU:")
cpu_result = spark.table("t1").join(spark.table("t2"), F.lit(True), "right_outer").select(F.col("t2.c0"))
cpu_result.show(truncate=False)
spark.conf.set('spark.rapids.sql.enabled', 'true')
print("GPU:")
gpu_result = spark.table("t1").join(spark.table("t2"), F.lit(True), "right_outer").select(F.col("t2.c0"))
gpu_result.show(truncate=False)t1.csv:
c0
0
t2.csv:
c0
CPU:
+---+
|c0 |
+---+
+---+
GPU:
25/11/24 04:05:43 ERROR Executor: Exception in task 0.0 in stage 7.0 (TID 7)
java.lang.AssertionError: assertion failed: data with no columns should have been filtered out already
at scala.Predef$.assert(Predef.scala:279)
at com.nvidia.spark.rapids.JoinGathererImpl.<init>(JoinGatherer.scala:541)
at com.nvidia.spark.rapids.JoinGatherer$.apply(JoinGatherer.scala:148)
at org.apache.spark.sql.rapids.execution.CrossJoinIterator.$anonfun$setupNextGatherer$1(GpuBroadcastNestedLoopJoinExecBase.scala:178)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:359)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:347)
at org.apache.spark.sql.rapids.execution.CrossJoinIterator.setupNextGatherer(GpuBroadcastNestedLoopJoinExecBase.scala:160)
at com.nvidia.spark.rapids.AbstractGpuJoinIterator.hasNext(AbstractGpuJoinIterator.scala:102)
at scala.collection.Iterator$$anon$9.hasNext(Iterator.scala:583)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at com.nvidia.spark.rapids.AbstractProjectSplitIterator.hasNext(basicPhysicalOperators.scala:256)
at scala.collection.Iterator$$anon$9.hasNext(Iterator.scala:583)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at com.nvidia.spark.rapids.GpuBaseLimitIterator.hasNext(limit.scala:50)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.partNextBatch(GpuShuffleExchangeExecBase.scala:411)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.hasNext(GpuShuffleExchangeExecBase.scala:434)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:143)
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:57)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:111)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:54)
at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:171)
at org.apache.spark.scheduler.Task.run(Task.scala:147)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$5(Executor.scala:647)
at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:80)
at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:77)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:99)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:650)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:840)
25/11/24 04:05:43 WARN TaskSetManager: Lost task 0.0 in stage 7.0 (TID 7) (70e244428b2c executor driver): java.lang.AssertionError: assertion failed: data with no columns should have been filtered out already
at scala.Predef$.assert(Predef.scala:279)
at com.nvidia.spark.rapids.JoinGathererImpl.<init>(JoinGatherer.scala:541)
at com.nvidia.spark.rapids.JoinGatherer$.apply(JoinGatherer.scala:148)
at org.apache.spark.sql.rapids.execution.CrossJoinIterator.$anonfun$setupNextGatherer$1(GpuBroadcastNestedLoopJoinExecBase.scala:178)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:359)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:347)
at org.apache.spark.sql.rapids.execution.CrossJoinIterator.setupNextGatherer(GpuBroadcastNestedLoopJoinExecBase.scala:160)
at com.nvidia.spark.rapids.AbstractGpuJoinIterator.hasNext(AbstractGpuJoinIterator.scala:102)
at scala.collection.Iterator$$anon$9.hasNext(Iterator.scala:583)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at com.nvidia.spark.rapids.AbstractProjectSplitIterator.hasNext(basicPhysicalOperators.scala:256)
at scala.collection.Iterator$$anon$9.hasNext(Iterator.scala:583)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at com.nvidia.spark.rapids.GpuBaseLimitIterator.hasNext(limit.scala:50)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.partNextBatch(GpuShuffleExchangeExecBase.scala:411)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.hasNext(GpuShuffleExchangeExecBase.scala:434)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:143)
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:57)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:111)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:54)
at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:171)
at org.apache.spark.scheduler.Task.run(Task.scala:147)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$5(Executor.scala:647)
at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:80)
at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:77)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:99)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:650)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:840)
25/11/24 04:05:43 ERROR TaskSetManager: Task 0 in stage 7.0 failed 1 times; aborting job
Traceback (most recent call last):
File "/xxx/bug.py", line 41, in <module>
gpu_result.show(truncate=False)
File "/xxx/../library/spark-4.0.1-bin-hadoop3/python/pyspark/sql/classic/dataframe.py", line 285, in show
print(self._show_string(n, truncate, vertical))
File "/xxx/../library/spark-4.0.1-bin-hadoop3/python/pyspark/sql/classic/dataframe.py", line 316, in _show_string
return self._jdf.showString(n, int_truncate, vertical)
File "../library/spark-4.0.1-bin-hadoop3/python/lib/py4j-0.10.9.9-src.zip/py4j/java_gateway.py", line 1362, in __call__
File "/xxx/../library/spark-4.0.1-bin-hadoop3/python/pyspark/errors/exceptions/captured.py", line 282, in deco
return f(*a, **kw)
File "../library/spark-4.0.1-bin-hadoop3/python/lib/py4j-0.10.9.9-src.zip/py4j/protocol.py", line 327, in get_return_value
py4j.protocol.Py4JJavaError: An error occurred while calling o69.showString.
: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 7.0 failed 1 times, most recent failure: Lost task 0.0 in stage 7.0 (TID 7) (70e244428b2c executor driver): java.lang.AssertionError: assertion failed: data with no columns should have been filtered out already
at scala.Predef$.assert(Predef.scala:279)
at com.nvidia.spark.rapids.JoinGathererImpl.<init>(JoinGatherer.scala:541)
at com.nvidia.spark.rapids.JoinGatherer$.apply(JoinGatherer.scala:148)
at org.apache.spark.sql.rapids.execution.CrossJoinIterator.$anonfun$setupNextGatherer$1(GpuBroadcastNestedLoopJoinExecBase.scala:178)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:359)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:347)
at org.apache.spark.sql.rapids.execution.CrossJoinIterator.setupNextGatherer(GpuBroadcastNestedLoopJoinExecBase.scala:160)
at com.nvidia.spark.rapids.AbstractGpuJoinIterator.hasNext(AbstractGpuJoinIterator.scala:102)
at scala.collection.Iterator$$anon$9.hasNext(Iterator.scala:583)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at com.nvidia.spark.rapids.AbstractProjectSplitIterator.hasNext(basicPhysicalOperators.scala:256)
at scala.collection.Iterator$$anon$9.hasNext(Iterator.scala:583)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at com.nvidia.spark.rapids.GpuBaseLimitIterator.hasNext(limit.scala:50)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.partNextBatch(GpuShuffleExchangeExecBase.scala:411)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.hasNext(GpuShuffleExchangeExecBase.scala:434)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:143)
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:57)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:111)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:54)
at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:171)
at org.apache.spark.scheduler.Task.run(Task.scala:147)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$5(Executor.scala:647)
at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:80)
at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:77)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:99)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:650)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:840)
Driver stacktrace:
at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$3(DAGScheduler.scala:2935)
at scala.Option.getOrElse(Option.scala:201)
at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2935)
at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2927)
at scala.collection.immutable.List.foreach(List.scala:334)
at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2927)
at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:1295)
at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:1295)
at scala.Option.foreach(Option.scala:437)
at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:1295)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:3207)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:3141)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:3130)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:50)
at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:1009)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2484)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2505)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2524)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2549)
at org.apache.spark.rdd.RDD.$anonfun$collect$1(RDD.scala:1057)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
at org.apache.spark.rdd.RDD.withScope(RDD.scala:417)
at org.apache.spark.rdd.RDD.collect(RDD.scala:1056)
at org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:462)
at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$executeCollect$1(AdaptiveSparkPlanExec.scala:402)
at org.apache.spark.sql.execution.adaptive.ResultQueryStageExec.$anonfun$doMaterialize$1(QueryStageExec.scala:325)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$4(SQLExecution.scala:322)
at org.apache.spark.sql.execution.SQLExecution$.withSessionTagsApplied(SQLExecution.scala:272)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$3(SQLExecution.scala:320)
at org.apache.spark.JobArtifactSet$.withActiveJobArtifactState(JobArtifactSet.scala:94)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$2(SQLExecution.scala:316)
at java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:840)
Caused by: java.lang.AssertionError: assertion failed: data with no columns should have been filtered out already
at scala.Predef$.assert(Predef.scala:279)
at com.nvidia.spark.rapids.JoinGathererImpl.<init>(JoinGatherer.scala:541)
at com.nvidia.spark.rapids.JoinGatherer$.apply(JoinGatherer.scala:148)
at org.apache.spark.sql.rapids.execution.CrossJoinIterator.$anonfun$setupNextGatherer$1(GpuBroadcastNestedLoopJoinExecBase.scala:178)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:359)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:347)
at org.apache.spark.sql.rapids.execution.CrossJoinIterator.setupNextGatherer(GpuBroadcastNestedLoopJoinExecBase.scala:160)
at com.nvidia.spark.rapids.AbstractGpuJoinIterator.hasNext(AbstractGpuJoinIterator.scala:102)
at scala.collection.Iterator$$anon$9.hasNext(Iterator.scala:583)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at com.nvidia.spark.rapids.AbstractProjectSplitIterator.hasNext(basicPhysicalOperators.scala:256)
at scala.collection.Iterator$$anon$9.hasNext(Iterator.scala:583)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at com.nvidia.spark.rapids.GpuBaseLimitIterator.hasNext(limit.scala:50)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.partNextBatch(GpuShuffleExchangeExecBase.scala:411)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.hasNext(GpuShuffleExchangeExecBase.scala:434)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$hasNext$1(GpuExec.scala:68)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.scala:17)
at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.hasNext(GpuExec.scala:68)
at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:143)
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:57)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:111)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:54)
at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:171)
at org.apache.spark.scheduler.Task.run(Task.scala:147)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$5(Executor.scala:647)
at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:80)
at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:77)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:99)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:650)
... 3 moreExpected behavior
No error occurs when using the GPU engines.
The result is consistent when using the CPU and GPU engines.
Environment details (please complete the following information)
- Environment location: Standalone
- Spark configuration settings related to the issue:
java-17-openjdk-amd64
spark-4.0.1-bin-hadoop3
rapids-4-spark_2.13-25.10.0
cudf-25.10.0-cuda12
Additional context
Add any other context about the problem here.
Metadata
Metadata
Assignees
Labels
bugSomething isn't workingSomething isn't working