spark-rapids icon indicating copy to clipboard operation
spark-rapids copied to clipboard

[BUG] `cast` mixed numeric and date data reading from csv files brings errors on the GPU engines

Open asddfl opened this issue 1 month ago • 1 comments

Describe the bug cast mixed numeric and date data reading from csv files brings errors 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("t0.csv", header=True, inferSchema=True)
df.createOrReplaceTempView("t0")
df = spark.read.csv("t1.csv", header=True, inferSchema=True)
df.createOrReplaceTempView("t1")

print("CPU:")
cpu_result = spark.table("t0").crossJoin(spark.table("t1")).where((F.col("t1.c0").cast("boolean")) | (F.col("t0.c0").cast("boolean")))
cpu_result.show(truncate=False)

spark.conf.set('spark.rapids.sql.enabled', 'true')
print("GPU:")
gpu_result = spark.table("t0").crossJoin(spark.table("t1")).where((F.col("t1.c0").cast("boolean")) | (F.col("t0.c0").cast("boolean")))
gpu_result.show(truncate=False)

t0.csv:

c0
0.05
1970-01-13

t1.csv:

c0
1.0
CPU:
+----------+---+
|c0        |c0 |
+----------+---+
|0.05      |1.0|
|1970-01-13|1.0|
+----------+---+

GPU:
25/12/02 07:32:25 ERROR Executor: Exception in task 0.0 in stage 7.0 (TID 7)
org.apache.spark.SparkRuntimeException: [CAST_INVALID_INPUT] The value 'in the input column has atleast one invalid value' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. SQLSTATE: 22018
        at org.apache.spark.sql.errors.QueryExecutionErrors$.invalidInputSyntaxForBooleanError(QueryExecutionErrors.scala:130)
        at org.apache.spark.sql.rapids.shims.RapidsErrorUtils340PlusBase.invalidInputSyntaxForBooleanError(RapidsErrorUtils340PlusBase.scala:105)
        at org.apache.spark.sql.rapids.shims.RapidsErrorUtils340PlusBase.invalidInputSyntaxForBooleanError$(RapidsErrorUtils340PlusBase.scala:104)
        at org.apache.spark.sql.rapids.shims.RapidsErrorUtils$.invalidInputSyntaxForBooleanError(RapidsErrorUtils.scala:23)
        at com.nvidia.spark.rapids.GpuCast$.$anonfun$castStringToBool$5(GpuCast.scala:1243)
        at com.nvidia.spark.rapids.GpuCast$.$anonfun$castStringToBool$5$adapted(GpuCast.scala:1240)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuCast$.$anonfun$castStringToBool$4(GpuCast.scala:1240)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuCast$.$anonfun$castStringToBool$3(GpuCast.scala:1237)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuCast$.$anonfun$castStringToBool$1(GpuCast.scala:1236)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuCast$.castStringToBool(GpuCast.scala:1232)
        at com.nvidia.spark.rapids.GpuCast$.$anonfun$doCast$28(GpuCast.scala:560)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuCast$.doCast(GpuCast.scala:560)
        at com.nvidia.spark.rapids.GpuCast.doColumnar(GpuCast.scala:1713)
        at com.nvidia.spark.rapids.GpuUnaryExpression.doItColumnar(GpuExpressions.scala:276)
        at com.nvidia.spark.rapids.GpuUnaryExpression.$anonfun$columnarEval$1(GpuExpressions.scala:287)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuUnaryExpression.columnarEval(GpuExpressions.scala:286)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$ReallyAGpuExpression.columnarEval(implicits.scala:34)
        at com.nvidia.spark.rapids.GpuAlias.columnarEval(namedExpressions.scala:110)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$ReallyAGpuExpression.columnarEval(implicits.scala:34)
        at com.nvidia.spark.rapids.GpuProjectExec$.$anonfun$project$1(basicPhysicalOperators.scala:132)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$IterableMapsSafely.$anonfun$safeMap$1(implicits.scala:166)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$IterableMapsSafely.$anonfun$safeMap$1$adapted(implicits.scala:163)
        at scala.collection.immutable.List.foreach(List.scala:334)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$IterableMapsSafely.safeMap(implicits.scala:163)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$AutoCloseableProducingSeq.safeMap(implicits.scala:277)
        at com.nvidia.spark.rapids.GpuProjectExec$.project(basicPhysicalOperators.scala:132)
        at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$project$2(basicPhysicalOperators.scala:948)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuTieredProject.recurse$2(basicPhysicalOperators.scala:947)
        at com.nvidia.spark.rapids.GpuTieredProject.project(basicPhysicalOperators.scala:960)
        at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$projectWithRetrySingleBatchInternal$5(basicPhysicalOperators.scala:896)
        at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.withRestoreOnRetry(RmmRapidsRetryIterator.scala:275)
        at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$projectWithRetrySingleBatchInternal$4(basicPhysicalOperators.scala:896)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$projectWithRetrySingleBatchInternal$3(basicPhysicalOperators.scala:894)
        at com.nvidia.spark.rapids.RmmRapidsRetryIterator$NoInputSpliterator.next(RmmRapidsRetryIterator.scala:415)
        at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryIterator.next(RmmRapidsRetryIterator.scala:690)
        at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryAutoCloseableIterator.next(RmmRapidsRetryIterator.scala:577)
        at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.drainSingleWithVerification(RmmRapidsRetryIterator.scala:294)
        at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.withRetryNoSplit(RmmRapidsRetryIterator.scala:188)
        at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$projectWithRetrySingleBatchInternal$1(basicPhysicalOperators.scala:894)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:39)
        at com.nvidia.spark.rapids.GpuTieredProject.projectWithRetrySingleBatchInternal(basicPhysicalOperators.scala:891)
        at com.nvidia.spark.rapids.GpuTieredProject.projectAndCloseWithRetrySingleBatch(basicPhysicalOperators.scala:930)
        at com.nvidia.spark.rapids.GpuProjectExec.$anonfun$internalDoExecuteColumnar$3(basicPhysicalOperators.scala:702)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuProjectExec.$anonfun$internalDoExecuteColumnar$2(basicPhysicalOperators.scala:697)
        at scala.collection.Iterator$$anon$9.next(Iterator.scala:584)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$next$1(GpuExec.scala:80)
        at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.next(GpuExec.scala:80)
        at com.nvidia.spark.rapids.CloseableBufferedIterator.next(CloseableBufferedIterator.scala:65)
        at com.nvidia.spark.rapids.CloseableBufferedIterator.head(CloseableBufferedIterator.scala:47)
        at org.apache.spark.sql.rapids.execution.GpuBroadcastHelper$.$anonfun$getBroadcastBuiltBatchAndStreamIter$2(GpuBroadcastHelper.scala:126)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at org.apache.spark.sql.rapids.execution.GpuBroadcastHelper$.$anonfun$getBroadcastBuiltBatchAndStreamIter$1(GpuBroadcastHelper.scala:124)
        at com.nvidia.spark.rapids.Arm$.closeOnExcept(Arm.scala:98)
        at org.apache.spark.sql.rapids.execution.GpuBroadcastHelper$.getBroadcastBuiltBatchAndStreamIter(GpuBroadcastHelper.scala:119)
        at org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase.makeBuiltBatchAndStreamIter(GpuBroadcastNestedLoopJoinExecBase.scala:584)
        at org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase.$anonfun$doConditionalJoin$1(GpuBroadcastNestedLoopJoinExecBase.scala:793)
        at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:866)
        at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:866)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD.compute(GpuExec.scala:61)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD.compute(GpuExec.scala:61)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD.compute(GpuExec.scala:61)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD.compute(GpuExec.scala:61)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD.compute(GpuExec.scala:61)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:107)
        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/12/02 07:32:25 WARN TaskSetManager: Lost task 0.0 in stage 7.0 (TID 7) (70e244428b2c executor driver): org.apache.spark.SparkRuntimeException: [CAST_INVALID_INPUT] The value 'in the input column has atleast one invalid value' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. SQLSTATE: 22018
        at org.apache.spark.sql.errors.QueryExecutionErrors$.invalidInputSyntaxForBooleanError(QueryExecutionErrors.scala:130)
        at org.apache.spark.sql.rapids.shims.RapidsErrorUtils340PlusBase.invalidInputSyntaxForBooleanError(RapidsErrorUtils340PlusBase.scala:105)
        at org.apache.spark.sql.rapids.shims.RapidsErrorUtils340PlusBase.invalidInputSyntaxForBooleanError$(RapidsErrorUtils340PlusBase.scala:104)
        at org.apache.spark.sql.rapids.shims.RapidsErrorUtils$.invalidInputSyntaxForBooleanError(RapidsErrorUtils.scala:23)
        at com.nvidia.spark.rapids.GpuCast$.$anonfun$castStringToBool$5(GpuCast.scala:1243)
        at com.nvidia.spark.rapids.GpuCast$.$anonfun$castStringToBool$5$adapted(GpuCast.scala:1240)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuCast$.$anonfun$castStringToBool$4(GpuCast.scala:1240)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuCast$.$anonfun$castStringToBool$3(GpuCast.scala:1237)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuCast$.$anonfun$castStringToBool$1(GpuCast.scala:1236)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuCast$.castStringToBool(GpuCast.scala:1232)
        at com.nvidia.spark.rapids.GpuCast$.$anonfun$doCast$28(GpuCast.scala:560)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuCast$.doCast(GpuCast.scala:560)
        at com.nvidia.spark.rapids.GpuCast.doColumnar(GpuCast.scala:1713)
        at com.nvidia.spark.rapids.GpuUnaryExpression.doItColumnar(GpuExpressions.scala:276)
        at com.nvidia.spark.rapids.GpuUnaryExpression.$anonfun$columnarEval$1(GpuExpressions.scala:287)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuUnaryExpression.columnarEval(GpuExpressions.scala:286)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$ReallyAGpuExpression.columnarEval(implicits.scala:34)
        at com.nvidia.spark.rapids.GpuAlias.columnarEval(namedExpressions.scala:110)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$ReallyAGpuExpression.columnarEval(implicits.scala:34)
        at com.nvidia.spark.rapids.GpuProjectExec$.$anonfun$project$1(basicPhysicalOperators.scala:132)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$IterableMapsSafely.$anonfun$safeMap$1(implicits.scala:166)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$IterableMapsSafely.$anonfun$safeMap$1$adapted(implicits.scala:163)
        at scala.collection.immutable.List.foreach(List.scala:334)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$IterableMapsSafely.safeMap(implicits.scala:163)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$AutoCloseableProducingSeq.safeMap(implicits.scala:277)
        at com.nvidia.spark.rapids.GpuProjectExec$.project(basicPhysicalOperators.scala:132)
        at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$project$2(basicPhysicalOperators.scala:948)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuTieredProject.recurse$2(basicPhysicalOperators.scala:947)
        at com.nvidia.spark.rapids.GpuTieredProject.project(basicPhysicalOperators.scala:960)
        at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$projectWithRetrySingleBatchInternal$5(basicPhysicalOperators.scala:896)
        at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.withRestoreOnRetry(RmmRapidsRetryIterator.scala:275)
        at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$projectWithRetrySingleBatchInternal$4(basicPhysicalOperators.scala:896)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$projectWithRetrySingleBatchInternal$3(basicPhysicalOperators.scala:894)
        at com.nvidia.spark.rapids.RmmRapidsRetryIterator$NoInputSpliterator.next(RmmRapidsRetryIterator.scala:415)
        at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryIterator.next(RmmRapidsRetryIterator.scala:690)
        at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryAutoCloseableIterator.next(RmmRapidsRetryIterator.scala:577)
        at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.drainSingleWithVerification(RmmRapidsRetryIterator.scala:294)
        at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.withRetryNoSplit(RmmRapidsRetryIterator.scala:188)
        at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$projectWithRetrySingleBatchInternal$1(basicPhysicalOperators.scala:894)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:39)
        at com.nvidia.spark.rapids.GpuTieredProject.projectWithRetrySingleBatchInternal(basicPhysicalOperators.scala:891)
        at com.nvidia.spark.rapids.GpuTieredProject.projectAndCloseWithRetrySingleBatch(basicPhysicalOperators.scala:930)
        at com.nvidia.spark.rapids.GpuProjectExec.$anonfun$internalDoExecuteColumnar$3(basicPhysicalOperators.scala:702)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at com.nvidia.spark.rapids.GpuProjectExec.$anonfun$internalDoExecuteColumnar$2(basicPhysicalOperators.scala:697)
        at scala.collection.Iterator$$anon$9.next(Iterator.scala:584)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.$anonfun$next$1(GpuExec.scala:80)
        at com.nvidia.spark.rapids.GpuMetric.ns(GpuMetrics.scala:354)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD$$anon$1.next(GpuExec.scala:80)
        at com.nvidia.spark.rapids.CloseableBufferedIterator.next(CloseableBufferedIterator.scala:65)
        at com.nvidia.spark.rapids.CloseableBufferedIterator.head(CloseableBufferedIterator.scala:47)
        at org.apache.spark.sql.rapids.execution.GpuBroadcastHelper$.$anonfun$getBroadcastBuiltBatchAndStreamIter$2(GpuBroadcastHelper.scala:126)
        at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
        at org.apache.spark.sql.rapids.execution.GpuBroadcastHelper$.$anonfun$getBroadcastBuiltBatchAndStreamIter$1(GpuBroadcastHelper.scala:124)
        at com.nvidia.spark.rapids.Arm$.closeOnExcept(Arm.scala:98)
        at org.apache.spark.sql.rapids.execution.GpuBroadcastHelper$.getBroadcastBuiltBatchAndStreamIter(GpuBroadcastHelper.scala:119)
        at org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase.makeBuiltBatchAndStreamIter(GpuBroadcastNestedLoopJoinExecBase.scala:584)
        at org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase.$anonfun$doConditionalJoin$1(GpuBroadcastNestedLoopJoinExecBase.scala:793)
        at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:866)
        at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:866)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD.compute(GpuExec.scala:61)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD.compute(GpuExec.scala:61)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD.compute(GpuExec.scala:61)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD.compute(GpuExec.scala:61)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at com.nvidia.spark.rapids.GpuOpTimeTrackingRDD.compute(GpuExec.scala:61)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
        at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:107)
        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/12/02 07:32:25 ERROR TaskSetManager: Task 0 in stage 7.0 failed 1 times; aborting job
Traceback (most recent call last):
  File "/xxx/bug.py", line 52, 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 288, in deco
    raise converted from None
pyspark.errors.exceptions.captured.SparkRuntimeException: [CAST_INVALID_INPUT] The value 'in the input column has atleast one invalid value' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. SQLSTATE: 22018

Expected 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.

asddfl avatar Dec 02 '25 07:12 asddfl

I investigated this issue and below are the details.

The issue is seen on Spark-4.0 as ANSI is enabled by default. The issue is not related to csv file reader. This bug can be reproduced on parquet, raw dataframe etc. The issue is due to the | (OR) condition mentioned in the query.

spark.table("t0").crossJoin(spark.table("t1")).where((F.col("t1.c0").cast("boolean")) | (F.col("t0.c0").cast("boolean")))

t1.csv:

c0
1.0

During CPU run, (F.col("t1.c0").cast("boolean")) is evaluated first. Value 1.0 in t1.c0 is considered as a valid input and evaluated as TRUE. Since the LHS is true, CPU doesn't evaluate the RHS i.e it short circuits the expression evaluation and returns true. So we see the result as above mentioned in the issue description.

In GPU case, we currently don't short circuit the expression evaluation so end up evaluating both LHS and RHS. So the RHS (` (F.col("t0.c0").cast("boolean")) has invalid value i.e date(1970-01-13), number(0.05) cannot be cast to boolean. So we see the error thrown as above in the bug. The error is thrown because ANSI is enabled and RHS has invalid values that cannot be cast to boolean.

This seems to be a edge case issue where we need to short-circuit the AND and OR operator in the above case. We see error being thrown on CPU case when we switch LHS and RHS. i.e error is thrown on CPU run for below query.

cpu_result = spark.table("t0").crossJoin(spark.table("t1")).where((F.col("t0.c0").cast("boolean")) | (F.col("t1.c0").cast("boolean")))

@asddfl - Wanted to check with you if the above issue is impacting you in any of the jobs you are running. In the short term, the workaround is to use try_cast instead of cast or disable ANSI mode(set spark.sql.ansi.enabled -> false).

nartal1 avatar Dec 10 '25 19:12 nartal1