[ https://issues.apache.org/jira/browse/SPARK-3447?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14138873#comment-14138873 ]
mohan gaddam commented on SPARK-3447: ------------------------------------- I am also facing the same issue with spark streaming API, kryo serializer and Avro Objects. I have observed this behavior with output operations like print, collect etc. also observed that if the avro object is simple, no problem. but if the avro objects are complex with unions/Arrays, gives the exception. find the stack trace below: ERROR scheduler.JobScheduler: Error running job streaming job 1411043845000 ms.0 org.apache.spark.SparkException: Job aborted due to stage failure: Exception while getting task result: com.esotericsoftware.kryo.KryoException: java.lang.NullPointerException Serialization trace: value (com.globallogic.goliath.model.Datum) data (com.globallogic.goliath.model.ResourceMessage) at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1185) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1174) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1173) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1173) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:688) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:688) at scala.Option.foreach(Option.scala:236) at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:688) at org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2.applyOrElse(DAGScheduler.scala:1391) at akka.actor.ActorCell.receiveMessage(ActorCell.scala:498) at akka.actor.ActorCell.invoke(ActorCell.scala:456) at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:237) at akka.dispatch.Mailbox.run(Mailbox.scala:219) at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386) at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) below is the avro message. {"version": "01", "sequence": "00001", "resource": "sensor-001", "controller": "002", "controllerTimestamp": "1411038710358", "data": {"value": [{"name": "Temperature", "value": "30"}, {"name": "Speed", "value": "60"}, {"name": "Location", "value": ["+401213.1", "-0750015.1"]}, {"name": "Timestamp", "value": "2014-09-09T08:15:25-05:00"}]}} message is been successfully decoded in decoder, but throws exception for output operation. > Kryo NPE when serializing JListWrapper > -------------------------------------- > > Key: SPARK-3447 > URL: https://issues.apache.org/jira/browse/SPARK-3447 > Project: Spark > Issue Type: Bug > Components: SQL > Reporter: Michael Armbrust > Assignee: Michael Armbrust > Fix For: 1.2.0 > > > Repro (provided by [~davies]): > {code} > from pyspark.sql import SQLContext; > SQLContext(sc).inferSchema(sc.parallelize([{"a": > [3]}]))._jschema_rdd.collect() > {code} > {code} > 14/09/05 21:59:47 ERROR TaskResultGetter: Exception while getting task result > com.esotericsoftware.kryo.KryoException: java.lang.NullPointerException > Serialization trace: > underlying (scala.collection.convert.Wrappers$JListWrapper) > values (org.apache.spark.sql.catalyst.expressions.GenericRow) > at > com.esotericsoftware.kryo.serializers.FieldSerializer$ObjectField.read(FieldSerializer.java:626) > at > com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:221) > at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:729) > at > com.esotericsoftware.kryo.serializers.DefaultArraySerializers$ObjectArraySerializer.read(DefaultArraySerializers.java:338) > at > com.esotericsoftware.kryo.serializers.DefaultArraySerializers$ObjectArraySerializer.read(DefaultArraySerializers.java:293) > at com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:648) > at > com.esotericsoftware.kryo.serializers.FieldSerializer$ObjectField.read(FieldSerializer.java:605) > at > com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:221) > at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:729) > at > com.esotericsoftware.kryo.serializers.DefaultArraySerializers$ObjectArraySerializer.read(DefaultArraySerializers.java:338) > at > com.esotericsoftware.kryo.serializers.DefaultArraySerializers$ObjectArraySerializer.read(DefaultArraySerializers.java:293) > at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:729) > at > org.apache.spark.serializer.KryoSerializerInstance.deserialize(KryoSerializer.scala:162) > at org.apache.spark.scheduler.DirectTaskResult.value(TaskResult.scala:79) > at > org.apache.spark.scheduler.TaskSetManager.handleSuccessfulTask(TaskSetManager.scala:514) > at > org.apache.spark.scheduler.TaskSchedulerImpl.handleSuccessfulTask(TaskSchedulerImpl.scala:355) > at > org.apache.spark.scheduler.TaskResultGetter$$anon$2$$anonfun$run$1.apply$mcV$sp(TaskResultGetter.scala:68) > at > org.apache.spark.scheduler.TaskResultGetter$$anon$2$$anonfun$run$1.apply(TaskResultGetter.scala:47) > at > org.apache.spark.scheduler.TaskResultGetter$$anon$2$$anonfun$run$1.apply(TaskResultGetter.scala:47) > at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1276) > at > org.apache.spark.scheduler.TaskResultGetter$$anon$2.run(TaskResultGetter.scala:46) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1146) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:701) > Caused by: java.lang.NullPointerException > at > scala.collection.convert.Wrappers$MutableBufferWrapper.add(Wrappers.scala:80) > at > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:109) > at > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:18) > at com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:648) > at > com.esotericsoftware.kryo.serializers.FieldSerializer$ObjectField.read(FieldSerializer.java:605) > ... 23 more > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org