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.types.{IntegerType, StructType}
scala> val df = Seq("""[]""").toDS()
scala> df.write.mode("OVERWRITE").parquet("TEMP")
24/05/27 10:39:59 WARN GpuOverrides:
*Exec <DataWritingCommandExec> will run on GPU
*Output <InsertIntoHadoopFsRelationCommand> will run on GPU
! <LocalTableScanExec> cannot run on GPU because GPU does not currently support the operator class org.apache.spark.sql.execution.LocalTableScanExec
@Expression <AttributeReference> value#1 could run on GPU
scala> val df2 = spark.read.parquet("TEMP")
scala> val schema = new StructType().add("a", IntegerType)
scala> var parsed = df2.select(from_json($"value", schema))
scala> parsed.show()
24/05/27 10:40:02 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> cast(from_json(StructField(a,IntegerType,true), value#5, Some(UTC)) as string) AS from_json(value)#11 will run on GPU
*Expression <Cast> cast(from_json(StructField(a,IntegerType,true), value#5, Some(UTC)) as string) will run on GPU
*Expression <JsonToStructs> from_json(StructField(a,IntegerType,true), value#5, Some(UTC)) will run on GPU
*Exec <FileSourceScanExec> will run on GPU
24/05/27 10:40:02 ERROR Executor: Exception in task 0.0 in stage 3.0 (TID 3)
java.lang.IllegalStateException: No empty row count provided and the table read has no row count or columns
at ai.rapids.cudf.Table.gatherJSONColumns(Table.java:1204)
at ai.rapids.cudf.Table.readJSON(Table.java:1446)
at ai.rapids.cudf.Table.readJSON(Table.java:1428)
at org.apache.spark.sql.rapids.GpuJsonToStructs.$anonfun$doColumnar$2(GpuJsonToStructs.scala:179)
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
at org.apache.spark.sql.rapids.GpuJsonToStructs.$anonfun$doColumnar$1(GpuJsonToStructs.scala:177)
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
at org.apache.spark.sql.rapids.GpuJsonToStructs.doColumnar(GpuJsonToStructs.scala:175)
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.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)
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
at com.nvidia.spark.rapids.GpuTieredProject.recurse$2(basicPhysicalOperators.scala:618)
at com.nvidia.spark.rapids.GpuTieredProject.project(basicPhysicalOperators.scala:631)
at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$projectWithRetrySingleBatchInternal$5(basicPhysicalOperators.scala:567)
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.withRestoreOnRetry(RmmRapidsRetryIterator.scala:272)
at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$projectWithRetrySingleBatchInternal$4(basicPhysicalOperators.scala:567)
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
at com.nvidia.spark.rapids.GpuTieredProject.$anonfun$projectWithRetrySingleBatchInternal$3(basicPhysicalOperators.scala:565)
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$NoInputSpliterator.next(RmmRapidsRetryIterator.scala:395)
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryIterator.next(RmmRapidsRetryIterator.scala:6
The text was updated successfully, but these errors were encountered:
Feng-Jiang28
changed the title
from_json - input=empty array, schema=struct, output=single row with null
from_json function parses a column containing an empty array, throws an exception.
May 27, 2024
from_json function parses a column containing an empty array, throws an exception.
CPU:
GPU:
The text was updated successfully, but these errors were encountered: