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

Many user provided closures are not actually cleaned

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 1.0.0
    • 1.4.0
    • Spark Core
    • None

    Description

      It appears that many operations throughout Spark actually do not actually clean the closures provided by the user.

      Simple reproduction:

      def test(): Unit = {
        sc.parallelize(1 to 10).mapPartitions { iter => return; iter }.collect()
      }
      

      Clearly, the inner closure is not serializable, but when we serialize it we should expect the ClosureCleaner to fail fast and complain loudly about return statements. Instead, we get a mysterious stack trace:

      java.io.NotSerializableException: java.lang.Object
      Serialization stack:
      	- object not serializable (class: java.lang.Object, value: java.lang.Object@6db4b914)
      	- field (class: $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$anonfun$1, name: nonLocalReturnKey1$1, type: class java.lang.Object)
      	- object (class $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$anonfun$1, <function1>)
      	- field (class: org.apache.spark.rdd.RDD$$anonfun$14, name: f$4, type: interface scala.Function1)
      	- object (class org.apache.spark.rdd.RDD$$anonfun$14, <function3>)
      	at org.apache.spark.serializer.SerializationDebugger$.improveException(SerializationDebugger.scala:40)
      	at org.apache.spark.serializer.JavaSerializationStream.writeObject(JavaSerializer.scala:47)
      	at org.apache.spark.serializer.JavaSerializerInstance.serialize(JavaSerializer.scala:81)
      	at org.apache.spark.util.ClosureCleaner$.ensureSerializable(ClosureCleaner.scala:314)
      

      What might have caused this? If you look at the code for mapPartitions, you'll notice that we never explicitly clean the closure passed in by the user. Instead, we only wrap it in another closure and clean only the outer one:

      def mapPartitions[U: ClassTag](
            f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
          val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(iter)
          new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning)
        }
      

      This is not sufficient, however, because the user provided closure is actually a field of the outer closure, and this inner closure doesn't get cleaned. If we rewrite the above by cleaning the inner closure preemptively, as we have done in other places:

      def mapPartitions[U: ClassTag](
            f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
          val cleanedFunc = clean(f)
          new MapPartitionsRDD(
            this,
            (context: TaskContext, index: Int, iter: Iterator[T]) => cleanedFunc(iter),
            preservesPartitioning)
        }
      

      Then we get the exception that we would expect by running the test() example above:

      org.apache.spark.SparkException: Return statements aren't allowed in Spark closures
      	at org.apache.spark.util.ReturnStatementFinder$$anon$1.visitTypeInsn(ClosureCleaner.scala:357)
      	at com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.ClassReader.accept(Unknown Source)
      	at com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.ClassReader.accept(Unknown Source)
      	at org.apache.spark.util.ClosureCleaner$.org$apache$spark$util$ClosureCleaner$$clean(ClosureCleaner.scala:215)
      	at org.apache.spark.util.ClosureCleaner$.clean(ClosureCleaner.scala:132)
      	at org.apache.spark.SparkContext.clean(SparkContext.scala:1759)
      	at org.apache.spark.rdd.RDD.mapPartitions(RDD.scala:640)
      

      It seems to me that we simply forgot to do this in a few places (e.g. mapPartitions, keyBy, aggregateByKey), because in other similar places we do this correctly (e.g. groupBy, combineByKey, zipPartitions).

      Attachments

        Issue Links

          Activity

            People

              andrewor14 Andrew Or
              andrewor14 Andrew Or
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: