Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-3070

Kryo deserialization without using the custom registrator

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Duplicate
    • 1.0.2
    • None
    • Spark Core
    • None

    Description

      If an RDD partition is cached on executor1 and used by a task on executor2 then the partition needs to be serialized and sent over. For this particular serialization/deserialization usecase, when using kry, it appears that the custom registrator will not be used on the deserialization side. This of course results in some totally misleading kry deserialization errors.

      The cause for this behavior seems to be that the thread running this deserialization has a classloader which does not have the jars specified in the SparkConf on its classpath. So it fails to load the Registrator with a ClassNotFoundException, but it catches the exception and happily continues without a registrator. (A bug on its own right in my opinion.)

      To reproduce, have two rdds partitioned the same way (as in with the same partitioner) but corresponding partitions cached on different machines, then join them. See below a somewhat convoluted way to achieve this. If you run the below program on a spark cluster with two workers, each with one core, you will be able to trigger the bug. Basically it runs two counts in parallel, which ensures that the two RDDs will be computed in parallel, and as a consequence on different executors.

      import com.esotericsoftware.kryo.Kryo
      import org.apache.spark.HashPartitioner
      import org.apache.spark.SparkConf
      import org.apache.spark.SparkContext
      import org.apache.spark.SparkContext._
      import org.apache.spark.rdd.RDD
      import org.apache.spark.serializer.KryoRegistrator
      import scala.actors.Actor
      
      case class MyClass(a: Int)
      
      class MyKryoRegistrator extends KryoRegistrator {
        override def registerClasses(kryo: Kryo) {
          kryo.register(classOf[MyClass])
        }
      }
      
      class CountActor(rdd: RDD[_]) extends Actor {
        def act() {
          println("Start count")
          println(rdd.count)
          println("Stop count")
        }
      }
      
      object KryBugExample  {
        def main(args: Array[String]) {
          val sparkConf = new SparkConf()
            .setMaster(args(0))
            .setAppName("KryBugExample")
            .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
            .set("spark.kryo.registrator", "MyKryoRegistrator")
            .setJars(Seq("target/scala-2.10/krybugexample_2.10-0.1-SNAPSHOT.jar"))
          val sc = new SparkContext(sparkConf)
      
          val partitioner = new HashPartitioner(1)
          val rdd1 = sc
            .parallelize((0 until 100000).map(i => (i, MyClass(i))), 1)
            .partitionBy(partitioner).cache
          val rdd2 = sc
            .parallelize((0 until 100000).map(i => (i, MyClass(i * 2))), 1)
            .partitionBy(partitioner).cache
          new CountActor(rdd1).start
          new CountActor(rdd2).start
          println(rdd1.join(rdd2).count)
          while (true) {}
        }
      }
      

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              xandrew Andras Nemeth
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: