[ https://issues.apache.org/jira/browse/SPARK-18737?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15729830#comment-15729830 ]
Sean Owen commented on SPARK-18737: ----------------------------------- Yeah I mean something a little different. If you (or a library) call registerKryoClasses it will enable Kryo. (The registration is actually irrelevant.) Maybe not a factor here. However I think this behavior is by design in some cases in Spark 2. What problem are you solving that you need to disable Kryo? > Serialization setting "spark.serializer" ignored in Spark 2.x > ------------------------------------------------------------- > > Key: SPARK-18737 > URL: https://issues.apache.org/jira/browse/SPARK-18737 > Project: Spark > Issue Type: Bug > Affects Versions: 2.0.0, 2.0.1 > Reporter: Dr. Michael Menzel > > The following exception occurs although the JavaSerializer has been activated: > 16/11/22 10:49:24 INFO TaskSetManager: Starting task 0.0 in stage 9.0 (TID > 77, ip-10-121-14-147.eu-central-1.compute.internal, partition 1, RACK_LOCAL, > 5621 bytes) > 16/11/22 10:49:24 INFO YarnSchedulerBackend$YarnDriverEndpoint: Launching > task 77 on executor id: 2 hostname: > ip-10-121-14-147.eu-central-1.compute.internal. > 16/11/22 10:49:24 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory > on ip-10-121-14-147.eu-central-1.compute.internal:45059 (size: 879.0 B, free: > 410.4 MB) > 16/11/22 10:49:24 WARN TaskSetManager: Lost task 0.0 in stage 9.0 (TID 77, > ip-10-121-14-147.eu-central-1.compute.internal): > com.esotericsoftware.kryo.KryoException: Encountered unregistered class ID: > 13994 > at > com.esotericsoftware.kryo.util.DefaultClassResolver.readClass(DefaultClassResolver.java:137) > at com.esotericsoftware.kryo.Kryo.readClass(Kryo.java:670) > at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:781) > at > org.apache.spark.serializer.KryoDeserializationStream.readObject(KryoSerializer.scala:229) > at > org.apache.spark.serializer.DeserializationStream$$anon$1.getNext(Serializer.scala:169) > at org.apache.spark.util.NextIterator.hasNext(NextIterator.scala:73) > at scala.collection.Iterator$class.foreach(Iterator.scala:893) > at org.apache.spark.util.NextIterator.foreach(NextIterator.scala:21) > at > scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59) > at > scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104) > at > scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48) > at > scala.collection.TraversableOnce$class.to(TraversableOnce.scala:310) > at org.apache.spark.util.NextIterator.to(NextIterator.scala:21) > at > scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:302) > at org.apache.spark.util.NextIterator.toBuffer(NextIterator.scala:21) > at > scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:289) > at org.apache.spark.util.NextIterator.toArray(NextIterator.scala:21) > at > org.apache.spark.rdd.RDD$$anonfun$toLocalIterator$1$$anonfun$org$apache$spark$rdd$RDD$$anonfun$$collectPartition$1$1.apply(RDD.scala:927) > at > org.apache.spark.rdd.RDD$$anonfun$toLocalIterator$1$$anonfun$org$apache$spark$rdd$RDD$$anonfun$$collectPartition$1$1.apply(RDD.scala:927) > at > org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1916) > at > org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1916) > at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70) > at org.apache.spark.scheduler.Task.run(Task.scala:86) > at > org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > The code runs perfectly with Spark 1.6.0. Since we moved to 2.0.0 and now > 2.0.1, we see the Kyro deserialization exception and over time the Spark > streaming job stops processing since too many tasks failed. > Our action was to use conf.set("spark.serializer", > "org.apache.spark.serializer.JavaSerializer") and to disable Kryo class > registration with conf.set("spark.kryo.registrationRequired", false). We hope > to identify the root cause of the exception. > However, setting the serializer to JavaSerializer is oviously ignored by the > Spark-internals. Despite the setting we still see the exception printed in > the log and tasks fail. The occurence seems to be non-deterministic, but to > become more frequent over time. > Several questions we could not answer during our troubleshooting: > 1. How can the debug log for Kryo be enabled? -- We tried following the > minilog documentation, but no output can be found. > 2. Is the serializer setting effective for Spark internal serializations? How > can the JavaSerialize be forced on internal serializations for worker to > driver communication? -- 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