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
scala> import org.apache.spark.sql.{SparkSession, Row}
import org.apache.spark.sql.{SparkSession, Row}
scala> import org.apache.spark.sql.functions._
import org.apache.spark.sql.functions._
scala> import org.apache.spark.sql.types._
import org.apache.spark.sql.types._
scala> val spark = SparkSession.builder.appName("Test").getOrCreate()
24/05/28 08:28:57 WARN RapidsPluginUtils: RAPIDS Accelerator is enabled, to disable GPU support set `spark.rapids.sql.enabled` to false.
24/05/28 08:28:57 WARN RapidsPluginUtils: spark.rapids.sql.explain is set to `ALL`. Set it to 'NONE' to suppress the diagnostics logging about the query placement on the GPU.
spark: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession@290a21ea
scala> import spark.implicits._
import spark.implicits._
scala> val structType = StructType(StructField("a", IntegerType) :: Nil)
structType: org.apache.spark.sql.types.StructType = StructType(StructField(a,IntegerType,true))
scala> val inputSchema = ArrayType(structType)
inputSchema: org.apache.spark.sql.types.ArrayType = ArrayType(StructType(StructField(a,IntegerType,true)),true)
scala> val data = Seq(
| Row(Seq(Row(null)))
| )
data: Seq[org.apache.spark.sql.Row] = List([List([null])])
scala> val df = spark.createDataFrame(
| spark.sparkContext.parallelize(data),
| StructType(Seq(StructField("value", inputSchema)))
| )
df: org.apache.spark.sql.DataFrame = [value: array<struct<a:int>>]
scala> val jsonDF = df.select(to_json(col("value")).alias("json_string"))
jsonDF: org.apache.spark.sql.DataFrame = [json_string: string]
scala> jsonDF.show(false)
24/05/28 08:29:00 WARN GpuOverrides:
!Exec <CollectLimitExec> cannot run on GPU because the Exec CollectLimitExec has been disabled, and is disabled by default because Collect Limit replacement can be slower on the GPU, if huge number of rows in a batch it could help by limiting the number of rows transferred from GPU to CPU. Set spark.rapids.sql.exec.CollectLimitExec to true if you wish to enable it
@Partitioning <SinglePartition$> could run on GPU
*Exec <ProjectExec> will run on GPU
*Expression <Alias> to_json(value#31, Some(UTC)) AS json_string#36 will run on GPU
*Expression <StructsToJson> to_json(value#31, Some(UTC)) will run on GPU
! <RDDScanExec> cannot run on GPU because GPU does not currently support the operator class org.apache.spark.sql.execution.RDDScanExec
@Expression <AttributeReference> value#31 could run on GPU
24/05/28 08:29:00 ERROR Executor: Exception in task 6.0 in stage 2.0 (TID 11)
java.lang.ClassCastException: org.apache.spark.sql.types.ArrayType cannot be cast to org.apache.spark.sql.types.StructType
at org.apache.spark.sql.rapids.GpuStructsToJson.doColumnar(GpuStructsToJson.scala:86)
at com.nvidia.spark.rapids.GpuUnaryExpression.doItColumnar(GpuExpressions.scala:250)
at com.nvidia.spark.rapids.GpuUnaryExpression.$anonfun$columnarEval$1(GpuExpressions.scala:261)
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
at com.nvidia.spark.rapids.GpuUnaryExpression.columnarEval(GpuExpressions.scala:260)
at com.nvidia.spark.rapids.RapidsPluginImplicits$ReallyAGpuExpression.columnarEval(implicits.scala:35)
at com.nvidia.spark.rapids.GpuAlias.columnarEval(namedExpressions.scala:110)
at com.nvidia.spark.rapids.RapidsPluginImplicits$ReallyAGpuExpression.columnarEval(implicits.scala:35)
at com.nvidia.spark.rapids.GpuProjectExec$.$anonfun$project$1(basicPhysicalOperators.scala:110)
at com.nvidia.spark.rapids.RapidsPluginImplicits$MapsSafely.$anonfun$safeMap$1(implicits.scala:221)
at com.nvidia.spark.rapids.RapidsPluginImplicits$MapsSafely.$anonfun$safeMap$1$adapted(implicits.scala:218)
at scala.collection.immutable.List.foreach(List.scala:431)
at com.nvidia.spark.rapids.RapidsPluginImplicits$MapsSafely.safeMap(implicits.scala:218)
at com.nvidia.spark.rapids.RapidsPluginImplicits$AutoCloseableProducingSeq.safeMap(implicits.scala:253)
at com.nvidia.spark.rapids.GpuProjectExec$.project(basicPhysicalOperators.scala:110)
at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$project$2(basicPhysicalOperators.scala:619)
The text was updated successfully, but these errors were encountered:
Feng-Jiang28
changed the title
to_json - empty array
to_json function to convert the null array of structs column to a JSON string throws an exception.
May 28, 2024
Feng-Jiang28
changed the title
to_json function to convert the null array of structs column to a JSON string throws an exception.
to_json function to convert the empty array of structs column to a JSON string throws an exception.
May 28, 2024
Feng-Jiang28
changed the title
to_json function to convert the empty array of structs column to a JSON string throws an exception.
to_json function to convert the array with a single empty row to a JSON string throws an exception.
May 28, 2024
to_json function to convert the array with a single empty row to a JSON string throws an exception.
Reproduce:
CPU:
GPU:
The text was updated successfully, but these errors were encountered: