[ https://issues.apache.org/jira/browse/SPARK-27144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16791376#comment-16791376 ]
Yoga edited comment on SPARK-27144 at 3/13/19 6:48 AM: ------------------------------------------------------- I think the problem comes from the handler logic of different struct type of the array elements, after the exploding, that the schema of `test` column is wrong like below, {code:java} root |-- weight: double (nullable = false) |-- animal: string (nullable = true) |-- extra: string (nullable = true) |-- test: struct (nullable = false) | |-- key: string (nullable = false) | |-- value: string (nullable = false) {code} As the `value` in `test` is nullable, but the cell value contain NULL cell indeed. was (Author: yogatamekasa): I think the problem is the handler of diffrent struct type of the array elements, after the exploding, that the schema of `test` column is wrong like below, {code:java} root |-- weight: double (nullable = false) |-- animal: string (nullable = true) |-- extra: string (nullable = true) |-- test: struct (nullable = false) | |-- key: string (nullable = false) | |-- value: string (nullable = false) {code} As the `value` in `test` is nullable, but the cell value contain NULL cell indeed. > Explode with structType may throw NPE > ------------------------------------- > > Key: SPARK-27144 > URL: https://issues.apache.org/jira/browse/SPARK-27144 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 2.3.0 > Environment: Spark 2.3.0, local mode. > Reporter: Yoga > Priority: Major > > Create a dataFrame containing two columns names [weight, animal], the > weight's nullable is false while the animal' nullable is true. > Give null value in the col animal, > then construct a new column with > {code:java} > explode( > array( > struct(lit("weight").alias("key"), > col("weight").cast(StringType).alias("value")), > struct(lit("animal").alias("key"), > col("animal").cast(StringType).alias("value")) > ) > ) > {code} > then select the struct with .*, Spark will throw NPE > {code:java} > 19/03/13 14:39:10 INFO DAGScheduler: ResultStage 3 (show at > SparkTest.scala:74) failed in 0.043 s due to Job aborted due to stage > failure: Task 3 in stage 3.0 failed 1 times, most recent failure: Lost task > 3.0 in stage 3.0 (TID 9, localhost, executor driver): > java.lang.NullPointerException > at > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(UnsafeRowWriter.java:194) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.project_doConsume$(Unknown > Source) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown > Source) > at > org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) > at > org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$10$$anon$1.hasNext(WholeStageCodegenExec.scala:614) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:253) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:247) > at > org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:830) > at > org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:830) > at > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) > at > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) > at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87) > at org.apache.spark.scheduler.Task.run(Task.scala:109) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345) > 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) > {code} > > Codes for reproduce: > {code:java} > val data = Seq( > Row(20.0, "dog","a"), > Row(3.5, "cat","b"), > Row(0.000006, null,"c") > ) > val schema = StructType(List( > StructField("weight", DoubleType, false), > StructField("animal", StringType, true), > StructField("extra", StringType, true) > ) > ) > val col1 = "weight" > val col2 = "animal" > //this should fail in select(test.*) > val df1 = originalDF.withColumn("test", > explode( > array( > struct(lit(col1).alias("key"), > col(col1).cast(StringType).alias("value")), > struct(lit(col2).alias("key"), > col(col2).cast(StringType).alias("value")) > ) > ) > ) > df1.printSchema() > df1.select("test.*").show() > // this should succeed in select(test.*) > val df2 = originalDF.withColumn("test", > explode( > array( > struct(lit(col2).alias("key"), > col(col2).cast(StringType).alias("value")), > struct(lit(col1).alias("key"), > col(col1).cast(StringType).alias("value")) > ) > ) > ) > df2.printSchema() > dfs.select("test.*").show() > {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org