From 6c997629e4d3bf9bccfbe9c3fa65aa1afa4bfca0 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 30 Oct 2014 11:02:04 -0400 Subject: [PATCH 01/22] Created class to traverse dependency graph of RDD --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 ++ core/src/main/scala/org/apache/spark/util/RDDWalker.scala | 8 ++++++++ 2 files changed, 10 insertions(+) create mode 100644 core/src/main/scala/org/apache/spark/util/RDDWalker.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b2774dfc4755..2c7d673327ad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -828,6 +828,8 @@ class DAGScheduler( val taskBinaryBytes: Array[Byte] = if (stage.isShuffleMap) { closureSerializer.serialize((stage.rdd, stage.shuffleDep.get) : AnyRef).array() + + stage.rdd. } else { closureSerializer.serialize((stage.rdd, stage.resultOfJob.get.func) : AnyRef).array() } diff --git a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala new file mode 100644 index 000000000000..19b8f52a162b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala @@ -0,0 +1,8 @@ +package org.apache.spark.util + +/** + * Created by zjb238 on 10/30/14. + */ +class RDDWalker { + +} From 47ccc227e5bdf14a1db20edfcf1b8f9c77b3b64a Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 30 Oct 2014 18:06:04 -0400 Subject: [PATCH 02/22] Started walker code --- .../org/apache/spark/util/RDDWalker.scala | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala index 19b8f52a162b..218dec9aa81c 100644 --- a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala +++ b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala @@ -1,8 +1,23 @@ package org.apache.spark.util +import org.apache.spark.rdd.RDD + +import scala.reflect.ClassTag + /** - * Created by zjb238 on 10/30/14. + * This class allows execution of a function on an RDD and all of its dependencies. This is accomplished by + * walking the object graph linking these RDDs. This is useful for debugging internal RDD references. + * @param rddToWalk - The RDD to traverse along with its dependencies + * @tparam T - The type of the RDD */ -class RDDWalker { +class RDDWalker[T : ClassTag] (rddToWalk : RDD[T]){ + val rdd = rddToWalk + + /** + * Execute the passed function on the underlying RDD + * @param func - The function to execute on + */ + def walk(func : (RDD[T])=>Unit): Unit ={ + } } From a8d5332a71fbad4cca0aa1a7ca73db8e1386e15f Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 6 Nov 2014 13:40:38 -0500 Subject: [PATCH 03/22] RDD WAlker updates --- .../apache/spark/scheduler/DAGScheduler.scala | 13 ++++++- .../spark/scheduler/TaskSetManager.scala | 23 ++++++++++++ .../org/apache/spark/util/RDDWalker.scala | 37 ++++++++++++++++--- .../org/apache/spark/CheckpointSuite.scala | 4 +- 4 files changed, 69 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 2c7d673327ad..cea45d436440 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -761,6 +761,10 @@ class DAGScheduler( submitWaitingStages() } + private def shouldPrintDebug(): Boolean ={ + return false + } + /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { val jobId = activeJobForStage(stage) @@ -828,11 +832,16 @@ class DAGScheduler( val taskBinaryBytes: Array[Byte] = if (stage.isShuffleMap) { closureSerializer.serialize((stage.rdd, stage.shuffleDep.get) : AnyRef).array() - - stage.rdd. } else { closureSerializer.serialize((stage.rdd, stage.resultOfJob.get.func) : AnyRef).array() } + + //Before serialization check whether a debug flag is enabled and if so, print out the RDD and its references + //prior to serialization + if(shouldPrintDebug()){ + logDebug(stage.rdd.toDebugString) + } + taskBinary = sc.broadcast(taskBinaryBytes) } catch { // In the case of a failure during serialization, abort the stage. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index d9d53faf843f..7a865eef4b5c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -435,6 +435,8 @@ private[spark] class TaskSetManager( val startTime = clock.getTime() // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here // we assume the task can be serialized without exceptions. + logDebug(taskDebugString(task, sched.sc.addedFiles, sched.sc.addedJars)) + val serializedTask = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) if (serializedTask.limit > TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024 && @@ -462,6 +464,27 @@ private[spark] class TaskSetManager( None } + /** + * Provide a string representation of the task and its dependencies (in terms of added files and jars that must + * be shipped with the task) for debugging purposes. + * @param task - The task to serialize + * @param addedFiles - The file dependencies + * @param addedJars - The JAR dependencies + * @return String - The task and dependencies as a string + */ + private def taskDebugString(task : Task[_], addedFiles : HashMap[String,Long], addedJars : HashMap[String,Long]): String ={ + val taskStr = "[" + task.toString + "] \n" + val strPrefix = s"-- " + val nl = s"\n" + val fileTitle = s"File dependencies:$nl" + val jarTitle = s"Jar dependencies:$nl" + + val fileStr = addedFiles.keys.map(file => s"$strPrefix $file").reduce(_ + nl + _) + nl + val jarStr = addedJars.keys.map(jar => s"$strPrefix $jar").reduce(_ + nl + _) + nl + + s"$taskStr $nl $fileTitle $fileStr $jarTitle $jarStr" + } + private def maybeFinishTaskSet() { if (isZombie && runningTasks == 0) { sched.taskSetFinished(this) diff --git a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala index 218dec9aa81c..5777cbea4f34 100644 --- a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala +++ b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala @@ -1,5 +1,7 @@ package org.apache.spark.util +import java.util + import org.apache.spark.rdd.RDD import scala.reflect.ClassTag @@ -7,17 +9,42 @@ import scala.reflect.ClassTag /** * This class allows execution of a function on an RDD and all of its dependencies. This is accomplished by * walking the object graph linking these RDDs. This is useful for debugging internal RDD references. - * @param rddToWalk - The RDD to traverse along with its dependencies - * @tparam T - The type of the RDD */ -class RDDWalker[T : ClassTag] (rddToWalk : RDD[T]){ - val rdd = rddToWalk +object RDDWalker { + + val walkQueue = new util.ArrayDeque[RDD[_]] + var visited = new util.HashSet[RDD[_]] + /** + * * Execute the passed function on the underlying RDD + * * @param rddToWalk - The RDD to traverse along with its dependencies * @param func - The function to execute on + * TODO Can there be cycles in RDD dependencies? */ - def walk(func : (RDD[T])=>Unit): Unit ={ + def walk(rddToWalk : RDD[_], func : (RDD[_])=>Unit): Unit ={ + + //Implement as a queue to perform a BFS + walkQueue.addFirst(rddToWalk) + + while(!walkQueue.isEmpty){ + //Pop from the queue + val rddToProcess : RDD[_] = walkQueue.pollFirst() + if(!visited.contains(rddToProcess)){ + rddToProcess.equals() + rddToProcess.dependencies.foreach(s => walkQueue.addFirst(s.rdd)) + func(rddToProcess) + visited.add(rddToProcess) + } + } + } + + def getRddString(rddToWalk : RDD[_]): Unit ={ + rddToWalk.toDebugString + } + + def toString(rddToWalk : RDD[_]) : Unit = { } } diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index a41914a1a9d0..e96f904aee23 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -112,7 +112,9 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { test("CartesianRDD") { def otherRDD = sc.makeRDD(1 to 10, 1) testRDD(new CartesianRDD(sc, _, otherRDD)) - testRDDPartitions(new CartesianRDD(sc, _, otherRDD)) + testRDDPartitions(new CartesianRDD(sc, _, oth + + // Test that the CartesianRDD updates parent partitions (CartesianRDD.s1/s2) after // the parent RDD has been checkpointed and parent partitions have been changed. From a63652f8240e0c370100ab05a11c95beaf47faa5 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 6 Nov 2014 13:42:48 -0500 Subject: [PATCH 04/22] Added debug output to task serialization. Added debug output to RDD serialization. --- .../apache/spark/scheduler/DAGScheduler.scala | 11 +--- .../org/apache/spark/util/RDDWalker.scala | 50 ------------------- 2 files changed, 2 insertions(+), 59 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/util/RDDWalker.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index cea45d436440..b8b51871842a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -761,10 +761,6 @@ class DAGScheduler( submitWaitingStages() } - private def shouldPrintDebug(): Boolean ={ - return false - } - /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { val jobId = activeJobForStage(stage) @@ -836,11 +832,8 @@ class DAGScheduler( closureSerializer.serialize((stage.rdd, stage.resultOfJob.get.func) : AnyRef).array() } - //Before serialization check whether a debug flag is enabled and if so, print out the RDD and its references - //prior to serialization - if(shouldPrintDebug()){ - logDebug(stage.rdd.toDebugString) - } + //Before serialization print out the RDD and its references + logDebug(stage.rdd.toDebugString) taskBinary = sc.broadcast(taskBinaryBytes) } catch { diff --git a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala deleted file mode 100644 index 5777cbea4f34..000000000000 --- a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala +++ /dev/null @@ -1,50 +0,0 @@ -package org.apache.spark.util - -import java.util - -import org.apache.spark.rdd.RDD - -import scala.reflect.ClassTag - -/** - * This class allows execution of a function on an RDD and all of its dependencies. This is accomplished by - * walking the object graph linking these RDDs. This is useful for debugging internal RDD references. - */ -object RDDWalker { - - val walkQueue = new util.ArrayDeque[RDD[_]] - var visited = new util.HashSet[RDD[_]] - - - /** - * - * Execute the passed function on the underlying RDD - * * @param rddToWalk - The RDD to traverse along with its dependencies - * @param func - The function to execute on - * TODO Can there be cycles in RDD dependencies? - */ - def walk(rddToWalk : RDD[_], func : (RDD[_])=>Unit): Unit ={ - - //Implement as a queue to perform a BFS - walkQueue.addFirst(rddToWalk) - - while(!walkQueue.isEmpty){ - //Pop from the queue - val rddToProcess : RDD[_] = walkQueue.pollFirst() - if(!visited.contains(rddToProcess)){ - rddToProcess.equals() - rddToProcess.dependencies.foreach(s => walkQueue.addFirst(s.rdd)) - func(rddToProcess) - visited.add(rddToProcess) - } - } - } - - def getRddString(rddToWalk : RDD[_]): Unit ={ - rddToWalk.toDebugString - } - - def toString(rddToWalk : RDD[_]) : Unit = { - - } -} From 05f2cc0665af3ca297936c8c4c5f6128be5a1ddc Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 6 Nov 2014 13:51:50 -0500 Subject: [PATCH 05/22] Rebase --- core/src/test/scala/org/apache/spark/CheckpointSuite.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index e96f904aee23..a41914a1a9d0 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -112,9 +112,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { test("CartesianRDD") { def otherRDD = sc.makeRDD(1 to 10, 1) testRDD(new CartesianRDD(sc, _, otherRDD)) - testRDDPartitions(new CartesianRDD(sc, _, oth - - + testRDDPartitions(new CartesianRDD(sc, _, otherRDD)) // Test that the CartesianRDD updates parent partitions (CartesianRDD.s1/s2) after // the parent RDD has been checkpointed and parent partitions have been changed. From cbb1d771f4576c6ba981252cd8b7490722317ddf Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 14 Nov 2014 14:03:25 -0500 Subject: [PATCH 06/22] Style errors --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b8b51871842a..090d13bac61c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -832,7 +832,7 @@ class DAGScheduler( closureSerializer.serialize((stage.rdd, stage.resultOfJob.get.func) : AnyRef).array() } - //Before serialization print out the RDD and its references + // Before serialization print out the RDD and its references logDebug(stage.rdd.toDebugString) taskBinary = sc.broadcast(taskBinaryBytes) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 7a865eef4b5c..db8430456ec3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -465,14 +465,16 @@ private[spark] class TaskSetManager( } /** - * Provide a string representation of the task and its dependencies (in terms of added files and jars that must + * Provide a string representation of the task and its dependencies (in terms of added files and + * jars that must * be shipped with the task) for debugging purposes. * @param task - The task to serialize * @param addedFiles - The file dependencies * @param addedJars - The JAR dependencies * @return String - The task and dependencies as a string */ - private def taskDebugString(task : Task[_], addedFiles : HashMap[String,Long], addedJars : HashMap[String,Long]): String ={ + private def taskDebugString(task : Task[_], addedFiles : HashMap[String,Long], + addedJars : HashMap[String,Long]): String ={ val taskStr = "[" + task.toString + "] \n" val strPrefix = s"-- " val nl = s"\n" From 916a31c57d89bc6fb83b33fdf70dfc1b94192cc5 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Sat, 29 Nov 2014 18:52:00 -0500 Subject: [PATCH 07/22] Manual merge of updates --- .../apache/spark/scheduler/DAGScheduler.scala | 93 ++++++++++++- .../spark/scheduler/TaskSetManager.scala | 40 ++---- .../spark/scheduler/DAGSchedulerSuite.scala | 131 +++++++++++++++++- 3 files changed, 230 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 3a232a7808ff..39c5b769c196 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -17,10 +17,15 @@ package org.apache.spark.scheduler + import java.io.NotSerializableException +import java.util import java.util.Properties import java.util.concurrent.atomic.AtomicInteger +import org.apache.spark.serializer.SerializerInstance + +import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} import scala.concurrent.Await import scala.concurrent.duration._ @@ -39,7 +44,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage._ -import org.apache.spark.util.{CallSite, SystemClock, Clock, Utils} +import org.apache.spark.util.{CallSite, SystemClock, Clock, Utils, RDDWalker, SerializationHelper} import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat /** @@ -788,6 +793,63 @@ class DAGScheduler( } } + /** + * Helper function to check whether an RDD is serializable. + * + * Note: This function is defined seperately from the SerializationHelper.isSerializable() + * since DAGScheduler.isSerialiazble() is passed as a parameter to the RDDWalker class's graph + * traversal, which would otherwise require knowledge of the closureSerializer + * (which was undesirable). + * + * @param rdd - Rdd to attempt to serialize + * @return - An output string qualifying success or failure. + */ + private def isSerializable(rdd: RDD[_]): String = { + SerializationHelper.isSerializable(closureSerializer,rdd) + } + + /** + * Use the RDDWalker class to execute a graph traversal of an RDD and its dependencies to help + * identify which RDDs are not serializable. In short, attempt to serialize the RDD and catch + * any Exceptions thrown (this is the same mechanism used within submitMissingTasks() to deal with + * serialization failures). + * + * Note: This is defined here since it uses the isSerializale function which in turn uses + * the closure serializer. Although the better place for the serializer would be in the + * SerializationHelper, the Helper is not guaranteed to run in a single thread unlike the + * DAGScheduler. + + * + * @param rdd - The rdd for which to print the serialization trace to identify unserializable + * components + * @return - String - The serialization trace + * + */ + def getSerializationTrace(rdd : RDD[_]): String = { + // Next, if there are dependencies, attempt to serialize those + val results: util.ArrayList[String] = RDDWalker.walk(rdd, isSerializable) + + var trace = "Serialization trace:\n" + + val it = results.iterator() + while(it.hasNext){ + trace += it.next() + "\n" + + } + + trace + } + + /** + * Use the RDD toDebugString function to print a formatted dependency trace for an RDD + * @param rdd - The RDD for which to print the dependency graph + * @return + */ + def getDependencyTrace(rdd: RDD[_]): String ={ + val debugString = rdd.toDebugString + "RDD Dependencies:\n" + debugString + "\n" + } + /** Called when stage's parents are available and we can now do its task. */ private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") @@ -826,9 +888,23 @@ class DAGScheduler( // might modify state of objects referenced in their closures. This is necessary in Hadoop // where the JobConf/Configuration object is not thread-safe. var taskBinary: Broadcast[Array[Byte]] = null + + // Check if RDD serialization debugging is enabled + // TODO After acceptance documentation for this option should be added to ScalaDoc + val debugSerialization: Boolean = sc.getConf.getOption("spark.serializer.debug") + .getOrElse("false").equals("true") + try { - // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). - // For ResultTask, serialize and broadcast (rdd, func). + // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). + + // Before serialization print out the RDD and its references. + if(debugSerialization) + { + logDebug(getDependencyTrace(stage.rdd)) + logDebug(getSerializationTrace(stage.rdd)) + } + + // For ResultTask, serialize and broadcast (rdd, func). val taskBinaryBytes: Array[Byte] = if (stage.isShuffleMap) { closureSerializer.serialize((stage.rdd, stage.shuffleDep.get) : AnyRef).array() @@ -836,12 +912,9 @@ class DAGScheduler( closureSerializer.serialize((stage.rdd, stage.resultOfJob.get.func) : AnyRef).array() } - // Before serialization print out the RDD and its references - logDebug(stage.rdd.toDebugString) - taskBinary = sc.broadcast(taskBinaryBytes) } catch { - // In the case of a failure during serialization, abort the stage. + // In the case of a failure during serialization, abort the stage. case e: NotSerializableException => abortStage(stage, "Task not serializable: " + e.toString) runningStages -= stage @@ -849,6 +922,7 @@ class DAGScheduler( case NonFatal(e) => abortStage(stage, s"Task serialization failed: $e\n${e.getStackTraceString}") runningStages -= stage + return } @@ -877,6 +951,11 @@ class DAGScheduler( // We've already serialized RDDs and closures in taskBinary, but here we check for all other // objects such as Partition. try { + if(debugSerialization) + { + logDebug(SerializationHelper.taskDebugString(tasks.head, sc.addedFiles, sc.addedJars)) + } + closureSerializer.serialize(tasks.head) } catch { case e: NotSerializableException => diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 1d9f6f094b02..118615bc6ee2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -27,8 +27,9 @@ import scala.math.{min, max} import org.apache.spark._ import org.apache.spark.executor.TaskMetrics + import org.apache.spark.TaskState.TaskState -import org.apache.spark.util.{Clock, SystemClock, Utils} +import org.apache.spark.util.{Clock, SystemClock, Utils, SerializationHelper} /** * Schedules the tasks within a single TaskSet in the TaskSchedulerImpl. This class keeps track of @@ -458,7 +459,19 @@ private[spark] class TaskSetManager( val startTime = clock.getTime() // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here // we assume the task can be serialized without exceptions. - logDebug(taskDebugString(task, sched.sc.addedFiles, sched.sc.addedJars)) + + // Check if serialization debugging is enabled + // TODO After acceptance, documentation for this option should be added to ScalaDoc + val printRdd : Boolean = sched.sc.getConf.getOption("spark.serializer.debug") + .getOrElse("false").equals("true") + + // If enabled, print out the added JARs and files (as part of the context) to help + // identify unserializable components + if(printRdd) + { + logDebug(SerializationHelper.taskDebugString(task, sched.sc.addedFiles, + sched.sc.addedJars)) + } val serializedTask = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) @@ -487,29 +500,6 @@ private[spark] class TaskSetManager( None } - /** - * Provide a string representation of the task and its dependencies (in terms of added files and - * jars that must - * be shipped with the task) for debugging purposes. - * @param task - The task to serialize - * @param addedFiles - The file dependencies - * @param addedJars - The JAR dependencies - * @return String - The task and dependencies as a string - */ - private def taskDebugString(task : Task[_], addedFiles : HashMap[String,Long], - addedJars : HashMap[String,Long]): String ={ - val taskStr = "[" + task.toString + "] \n" - val strPrefix = s"-- " - val nl = s"\n" - val fileTitle = s"File dependencies:$nl" - val jarTitle = s"Jar dependencies:$nl" - - val fileStr = addedFiles.keys.map(file => s"$strPrefix $file").reduce(_ + nl + _) + nl - val jarStr = addedJars.keys.map(jar => s"$strPrefix $jar").reduce(_ + nl + _) + nl - - s"$taskStr $nl $fileTitle $fileStr $jarTitle $jarStr" - } - private def maybeFinishTaskSet() { if (isZombie && runningTasks == 0) { sched.taskSetFinished(this) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index bdd721dc7eaf..bdca2503cd10 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} -import org.apache.spark.util.CallSite +import org.apache.spark.util.{SerializationHelper, CallSite} import org.apache.spark.executor.TaskMetrics class BuggyDAGEventProcessActor extends Actor { @@ -244,6 +244,133 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F private def cancel(jobId: Int) { runEvent(JobCancelled(jobId)) } + + test("Serialization trace for unserializable task") { + val unserializableRdd = new MyRDD(sc, 1, Nil) { + class UnserializableClass + val unserializable = new UnserializableClass + } + + val trace = scheduler.getSerializationTrace(unserializableRdd) + + val splitS = trace.split(":") + val depth = splitS(1).trim() + val status = splitS(2).trim() + val rddName = splitS(3).trim() + + assert(rddName.equals("DAGSchedulerSuiteRDD 0")) + assert(status.equals(SerializationHelper.Failed)) + } + + test("Serialization trace for unserializable task with serializable dependencies") { + // The trace should show which nested dependency is unserializable + + val baseRdd = new MyRDD(sc, 1, Nil) + val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) + val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))){ + class UnserializableClass + val unserializable = new UnserializableClass + } + + val result = Array(SerializationHelper.Failed, + SerializationHelper.Serialized, + SerializationHelper.Serialized) + + val trace = scheduler.getSerializationTrace(finalRdd) + val splitRdds = trace.split("\n") + + var x = 0 + for(x <- 1 until splitRdds.length){ + val splitS = splitRdds(x).split(":") + val status = splitS(1).trim() + + assert(status.equals(result(x-1))) + + } + + } + + test("Serialization trace for serializable task and nested unserializable dependency") { + // The trace should show which nested dependency is unserializable + + val baseRdd = new MyRDD(sc, 1, Nil){ + class UnserializableClass + val unserializable = new UnserializableClass + } + + val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) + val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) + val result = Array(SerializationHelper.Serialized, + SerializationHelper.FailedDeps, + SerializationHelper.Failed) + + val trace = scheduler.getSerializationTrace(finalRdd) + val splitRdds = trace.split("\n") + + var x = 0 + + for(x <- 1 until splitRdds.length){ + val splitS = splitRdds(x).split(":") + val status = splitS(1).trim() + + assert(status.equals(result(x-1))) + + } + + } + + test("Serialization trace for serializable task with sandwiched unserializable dependency") { + // The trace should show which nested dependency is unserializable + + val baseRdd = new MyRDD(sc, 1, Nil) + val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))){ + class UnserializableClass + val unserializable = new UnserializableClass + } + val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) + val result = Array(SerializationHelper.FailedDeps, + SerializationHelper.Failed, + SerializationHelper.Serialized) + + val trace = scheduler.getSerializationTrace(finalRdd) + val splitRdds = trace.split("\n") + + var x = 0 + for(x <- 1 until splitRdds.length){ + val splitS = splitRdds(x).split(":") + val status = splitS(1).trim() + assert(status.equals(result(x-1))) + + } + + } + + test("Serialization trace for serializable task and nested dependencies") { + // Because serialization also attempts to serialize dependencies, attempting to + // serialize the serializable "finalRdd" should fail and the trace should show all its + // dependencies as being unserializable. + + val baseRdd = new MyRDD(sc, 1, Nil) + val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) + val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) + + val trace = scheduler.getSerializationTrace(finalRdd) + val splitRdds = trace.split("\n") + + var x = 0 + for(x <- 1 until splitRdds.length){ + val splitS = splitRdds(x).split(":") + val status = splitS(1).trim() + + if(!status.equals(SerializationHelper.Serialized)) + throw new Exception(trace + "\n" +">"+status+":"+SerializationHelper.Serialized+"<") + + assert(status.equals(SerializationHelper.Serialized)) + + } + + } + test("[SPARK-3353] parent stage should have lower stage id") { sparkListener.stageByOrderOfExecution.clear() @@ -539,7 +666,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } - + /** * Makes sure that failures of stage used by multiple jobs are correctly handled. * From bfb723de65e60aabb9cccc3b45ccc4638f12583d Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Sat, 29 Nov 2014 18:55:40 -0500 Subject: [PATCH 08/22] Added helper files --- .../org/apache/spark/util/RDDWalker.scala | 66 +++++++++ .../spark/util/SerializationHelper.scala | 127 ++++++++++++++++++ 2 files changed, 193 insertions(+) create mode 100644 core/src/main/scala/org/apache/spark/util/RDDWalker.scala create mode 100644 core/src/main/scala/org/apache/spark/util/SerializationHelper.scala diff --git a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala new file mode 100644 index 000000000000..91068f8616b7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// Created by Ilya Ganelin + +package org.apache.spark.util + +import java.util +import org.apache.spark.rdd.RDD +import scala.language.existentials + +/** + * This class allows execution of a function on an RDD and all of its dependencies. This is + * accomplished by walking the object graph linking these RDDs. This is useful for debugging + * internal RDD references. See SPARK-3694. + */ +object RDDWalker { + + // Keep track of both the RDD and its depth in the traversal graph. + val walkQueue = new util.ArrayDeque[(RDD[_], Int)] + var visited = new util.HashSet[RDD[_]] + + + /** + * + * Execute the passed function on the underlying RDD + * @param rddToWalk - The RDD to traverse along with its dependencies + * @param func - The function to execute on each node. Returns a string + * @return Array[String] - An array of results generated by the traversal function + * TODO Can there be cycles in RDD dependencies? + */ + def walk(rddToWalk : RDD[_], func : (RDD[_])=>String): util.ArrayList[String] ={ + + val results = new util.ArrayList[String] + // Implement as a queue to perform a BFS + walkQueue.addFirst(rddToWalk,0) + + while(!walkQueue.isEmpty){ + // Pop from the queue + val (rddToProcess : RDD[_], depth:Int) = walkQueue.pollFirst() + if(!visited.contains(rddToProcess)){ + visited.add(rddToProcess) + rddToProcess.dependencies.foreach(s => walkQueue.addFirst(s.rdd, depth + 1)) + results.add("Depth " + depth + ": " + func(rddToProcess)) + } + } + + results + } + + +} diff --git a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala new file mode 100644 index 000000000000..38676ef407f4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// Created by Ilya Ganelin +package org.apache.spark.util + +import java.io.NotSerializableException + +import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.Task +import org.apache.spark.serializer.{SerializerInstance, Serializer} + +import scala.collection.mutable.HashMap +import scala.util.control.NonFatal + +/** + * This class is designed to encapsulate some utilities to facilitate debugging serialization + * problems in the DAGScheduler and the TaskSetManager. See SPARK-3694. + */ +object SerializationHelper { + // Define vars to standardize debugging output + var Failed = "Failed to serialize" + var FailedDeps = "Failed to serialize dependencies" + var Serialized = "Serialized" + + /** + * Helper function to check whether an RDD is serializable. + * + * If any dependency of an RDD is un-serializable, a NotSerializableException will be thrown + * and the entire RDD will be deemed un-serializable if done with a single try-catch. + * + * Therefore, split the evaluation into two stages, in the first stage attempt to serialize + * the rdd. If it fails, attempt to serialize its dependencies in the failure handler and see + * if those also fail. + * + * This approach will show if any of the dependencies are un-serializable and will not + * incorrectly identify the parent RDD as being serializable. + * + * @param closureSerializer - An instance of a serializer (single-threaded) that will be used + * @param rdd - Rdd to attempt to serialize + * @return - An output string qualifying success or failure. + */ + def isSerializable(closureSerializer : SerializerInstance, rdd : RDD[_]) : String = { + try { + closureSerializer.serialize(rdd: AnyRef) + Serialized + ": " + rdd.toString + } + catch { + case e: NotSerializableException => + handleFailure(closureSerializer, rdd) + + case NonFatal(e) => + handleFailure(closureSerializer, rdd) + } + } + + /** + * Helper function to seperate an un-serialiable parent rdd from un-serializable dependencies + * @param closureSerializer - An instance of a serializer (single-threaded) that will be used + * @param rdd - Rdd to attempt to serialize + * @return - An output string qualifying success or failure. + */ + def handleFailure(closureSerializer : SerializerInstance, + rdd: RDD[_]): String ={ + if(rdd.dependencies.length > 0){ + try{ + rdd.dependencies.foreach(dep => closureSerializer.serialize(dep : AnyRef)) + + // By default, return a failure since we still failed to serialize the parent RDD + // Now, however, we know that the dependencies are serializable + Failed + ": " + rdd.toString + } + catch { + // If instead, however, the dependencies ALSO fail to serialize then the subsequent stage + // of evaluation will help identify which of the dependencies has failed + case e: NotSerializableException => + FailedDeps + ": " + rdd.toString + + case NonFatal(e) => + FailedDeps + ": " + rdd.toString + } + + } + else{ + Failed + ": " + rdd.toString + } + + } + + /** + * Provide a string representation of the task and its dependencies (in terms of added files + * and jars that must be shipped with the task) for debugging purposes. + * @param task - The task to serialize + * @param addedFiles - The file dependencies + * @param addedJars - The JAR dependencies + * @return String - The task and dependencies as a string + */ + def taskDebugString(task : Task[_], + addedFiles : HashMap[String,Long], + addedJars : HashMap[String,Long]): String ={ + val taskStr = "[" + task.toString + "] \n" + val strPrefix = s"-- " + val nl = s"\n" + val fileTitle = s"File dependencies:$nl" + val jarTitle = s"Jar dependencies:$nl" + + val fileStr = addedFiles.keys.map(file => s"$strPrefix $file").reduce(_ + nl + _) + nl + val jarStr = addedJars.keys.map(jar => s"$strPrefix $jar").reduce(_ + nl + _) + nl + + s"$taskStr $nl $fileTitle $fileStr $jarTitle $jarStr" + } +} + From e0a81537d5962f8bc79b8b9193a30b46827246ed Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Sat, 29 Nov 2014 19:45:52 -0500 Subject: [PATCH 09/22] Fixed whitespace errors --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 9 +++------ .../org/apache/spark/scheduler/TaskSetManager.scala | 1 - .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 1 - 3 files changed, 3 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 39c5b769c196..a435d9a3318c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -896,6 +896,7 @@ class DAGScheduler( try { // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). + // For ResultTask, serialize and broadcast (rdd, func). // Before serialization print out the RDD and its references. if(debugSerialization) @@ -903,18 +904,16 @@ class DAGScheduler( logDebug(getDependencyTrace(stage.rdd)) logDebug(getSerializationTrace(stage.rdd)) } - - // For ResultTask, serialize and broadcast (rdd, func). + val taskBinaryBytes: Array[Byte] = if (stage.isShuffleMap) { closureSerializer.serialize((stage.rdd, stage.shuffleDep.get) : AnyRef).array() } else { closureSerializer.serialize((stage.rdd, stage.resultOfJob.get.func) : AnyRef).array() } - taskBinary = sc.broadcast(taskBinaryBytes) } catch { - // In the case of a failure during serialization, abort the stage. + // In the case of a failure during serialization, abort the stage. case e: NotSerializableException => abortStage(stage, "Task not serializable: " + e.toString) runningStages -= stage @@ -922,7 +921,6 @@ class DAGScheduler( case NonFatal(e) => abortStage(stage, s"Task serialization failed: $e\n${e.getStackTraceString}") runningStages -= stage - return } @@ -955,7 +953,6 @@ class DAGScheduler( { logDebug(SerializationHelper.taskDebugString(tasks.head, sc.addedFiles, sc.addedJars)) } - closureSerializer.serialize(tasks.head) } catch { case e: NotSerializableException => diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 118615bc6ee2..003d7f546483 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -27,7 +27,6 @@ import scala.math.{min, max} import org.apache.spark._ import org.apache.spark.executor.TaskMetrics - import org.apache.spark.TaskState.TaskState import org.apache.spark.util.{Clock, SystemClock, Utils, SerializationHelper} diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index bdca2503cd10..75b96548f373 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -666,7 +666,6 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } - /** * Makes sure that failures of stage used by multiple jobs are correctly handled. * From cb6ebb1fdfb9447a7e04e756fa40431b359311b5 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Sat, 29 Nov 2014 19:58:54 -0500 Subject: [PATCH 10/22] Updated documentation to add debug parameter for rdd serialization --- docs/configuration.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index 0b77f5ab645c..fca852850b26 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -517,6 +517,14 @@ Apart from these, the following properties are also available, and may be useful + spark.serializer.debug + false + + To view the dependency graph for an RDD or the file dependencies for task set this option + to true. Doing so will display a text based graph of these dependencies along with a + serialization trace that identifies which components of an RDD failed to serialize. + + spark.kryo.referenceTracking true From 95fa69b59f72105bc6b8b5101ea2bf3eedf1039f Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 1 Dec 2014 14:56:30 -0500 Subject: [PATCH 11/22] Incorporated feedback from PR. Cleaned up code and refactored interface to RDD traversal so that we now return a structured object instead of a formatted string --- .../apache/spark/scheduler/DAGScheduler.scala | 64 +++++----- .../spark/scheduler/TaskSetManager.scala | 12 +- .../org/apache/spark/util/RDDWalker.scala | 29 ++--- .../spark/util/SerializationHelper.scala | 84 +++++++------ .../spark/scheduler/DAGSchedulerSuite.scala | 119 ++++++++---------- 5 files changed, 139 insertions(+), 169 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index a435d9a3318c..b6743281d03b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -19,11 +19,12 @@ package org.apache.spark.scheduler import java.io.NotSerializableException +import java.nio.ByteBuffer import java.util import java.util.Properties import java.util.concurrent.atomic.AtomicInteger -import org.apache.spark.serializer.SerializerInstance + import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} @@ -44,7 +45,9 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage._ -import org.apache.spark.util.{CallSite, SystemClock, Clock, Utils, RDDWalker, SerializationHelper} +import org.apache.spark.util.{CallSite, Clock, RDDWalker, SerializationHelper, + SerializationState, SystemClock, Utils} +import org.apache.spark.util.SerializationHelper.SerializedRdd import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat /** @@ -796,16 +799,23 @@ class DAGScheduler( /** * Helper function to check whether an RDD is serializable. * - * Note: This function is defined seperately from the SerializationHelper.isSerializable() - * since DAGScheduler.isSerialiazble() is passed as a parameter to the RDDWalker class's graph + * Note: This function is defined separately from the SerializationHelper.isSerializable() + * since DAGScheduler.isSerializable() is passed as a parameter to the RDDWalker class's graph * traversal, which would otherwise require knowledge of the closureSerializer * (which was undesirable). * * @param rdd - Rdd to attempt to serialize - * @return - An output string qualifying success or failure. + * @return Array[SerializedRdd] - + * Return an array of Either objects indicating if serialization is successful: + * Success: ByteBuffer - The serialized RDD + * Failure: String - The reason for the failure. + * */ - private def isSerializable(rdd: RDD[_]): String = { - SerializationHelper.isSerializable(closureSerializer,rdd) + def tryToSerialize(rdd: RDD[_]): Array[SerializedRdd] = { + val traversal : Array[(RDD[_], Int)] = RDDWalker.walk(rdd) + traversal.map({ + case (curRdd, depth) => SerializationHelper.tryToSerialize(closureSerializer, curRdd) + }) } /** @@ -814,42 +824,29 @@ class DAGScheduler( * any Exceptions thrown (this is the same mechanism used within submitMissingTasks() to deal with * serialization failures). * - * Note: This is defined here since it uses the isSerializale function which in turn uses + * Note: This is defined here since it uses the tryToSerialize function which in turn uses * the closure serializer. Although the better place for the serializer would be in the * SerializationHelper, the Helper is not guaranteed to run in a single thread unlike the - * DAGScheduler. - + * DAGScheduler. * - * @param rdd - The rdd for which to print the serialization trace to identify unserializable + * @param rdd - The rdd for which to print the serialization trace to identify un-serializable * components * @return - String - The serialization trace * */ - def getSerializationTrace(rdd : RDD[_]): String = { - // Next, if there are dependencies, attempt to serialize those - val results: util.ArrayList[String] = RDDWalker.walk(rdd, isSerializable) + def getSerializationAsString(rdd : RDD[_]): String = { + // Next, if there are dependencies, attempt to serialize those + val results : Array[SerializedRdd] = tryToSerialize(rdd) var trace = "Serialization trace:\n" - val it = results.iterator() + val it = results.iterator while(it.hasNext){ - trace += it.next() + "\n" - + trace += it.next().fold(l => l, r=> SerializationState.Success) + "\n" } - trace } - /** - * Use the RDD toDebugString function to print a formatted dependency trace for an RDD - * @param rdd - The RDD for which to print the dependency graph - * @return - */ - def getDependencyTrace(rdd: RDD[_]): String ={ - val debugString = rdd.toDebugString - "RDD Dependencies:\n" + debugString + "\n" - } - /** Called when stage's parents are available and we can now do its task. */ private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") @@ -890,19 +887,16 @@ class DAGScheduler( var taskBinary: Broadcast[Array[Byte]] = null // Check if RDD serialization debugging is enabled - // TODO After acceptance documentation for this option should be added to ScalaDoc - val debugSerialization: Boolean = sc.getConf.getOption("spark.serializer.debug") - .getOrElse("false").equals("true") + val debugSerialization: Boolean = sc.getConf.getBoolean("spark.serializer.debug", false) try { // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). // For ResultTask, serialize and broadcast (rdd, func). // Before serialization print out the RDD and its references. - if(debugSerialization) - { - logDebug(getDependencyTrace(stage.rdd)) - logDebug(getSerializationTrace(stage.rdd)) + if(debugSerialization) { + logDebug("RDD Dependencies:\n" + stage.rdd.toDebugString + "\n") + logDebug(getSerializationAsString(stage.rdd)) } val taskBinaryBytes: Array[Byte] = diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 003d7f546483..d79dc9f432b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -28,7 +28,7 @@ import scala.math.{min, max} import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.TaskState.TaskState -import org.apache.spark.util.{Clock, SystemClock, Utils, SerializationHelper} +import org.apache.spark.util.{Clock, SerializationHelper, SystemClock, Utils} /** * Schedules the tasks within a single TaskSet in the TaskSchedulerImpl. This class keeps track of @@ -460,14 +460,10 @@ private[spark] class TaskSetManager( // we assume the task can be serialized without exceptions. // Check if serialization debugging is enabled - // TODO After acceptance, documentation for this option should be added to ScalaDoc - val printRdd : Boolean = sched.sc.getConf.getOption("spark.serializer.debug") - .getOrElse("false").equals("true") + val debugSerialization: Boolean = sched.sc.getConf.getBoolean("spark.serializer.debug", false) - // If enabled, print out the added JARs and files (as part of the context) to help - // identify unserializable components - if(printRdd) - { + // If enabled, print out the added JARs and files (as part of the context) + if(debugSerialization) { logDebug(SerializationHelper.taskDebugString(task, sched.sc.addedFiles, sched.sc.addedJars)) } diff --git a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala index 91068f8616b7..cd92d01301cd 100644 --- a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala +++ b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala @@ -15,16 +15,17 @@ * limitations under the License. */ -// Created by Ilya Ganelin - package org.apache.spark.util import java.util -import org.apache.spark.rdd.RDD + +import scala.collection.mutable.ArrayBuffer import scala.language.existentials +import org.apache.spark.rdd.RDD + /** - * This class allows execution of a function on an RDD and all of its dependencies. This is + * This class allows trarseal of an RDD's dependency graph. This is * accomplished by walking the object graph linking these RDDs. This is useful for debugging * internal RDD references. See SPARK-3694. */ @@ -34,18 +35,16 @@ object RDDWalker { val walkQueue = new util.ArrayDeque[(RDD[_], Int)] var visited = new util.HashSet[RDD[_]] - /** - * - * Execute the passed function on the underlying RDD + * Traverse the dependencies of the RDD and store them within an Array along with their depths. + * Return this data structure and subsequently process it. + * * @param rddToWalk - The RDD to traverse along with its dependencies - * @param func - The function to execute on each node. Returns a string - * @return Array[String] - An array of results generated by the traversal function - * TODO Can there be cycles in RDD dependencies? + * @return Array[(RDD[_], depth : Int] - An array of results generated by the traversal function */ - def walk(rddToWalk : RDD[_], func : (RDD[_])=>String): util.ArrayList[String] ={ + def walk(rddToWalk : RDD[_]): Array[(RDD[_], Int)] ={ - val results = new util.ArrayList[String] + val results = new ArrayBuffer[(RDD[_], Int)] // Implement as a queue to perform a BFS walkQueue.addFirst(rddToWalk,0) @@ -55,12 +54,10 @@ object RDDWalker { if(!visited.contains(rddToProcess)){ visited.add(rddToProcess) rddToProcess.dependencies.foreach(s => walkQueue.addFirst(s.rdd, depth + 1)) - results.add("Depth " + depth + ": " + func(rddToProcess)) + results.append((rddToProcess, depth)) } } - results + results.toArray } - - } diff --git a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala index 38676ef407f4..bf81954b2833 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala @@ -15,28 +15,33 @@ * limitations under the License. */ -// Created by Ilya Ganelin package org.apache.spark.util import java.io.NotSerializableException +import java.nio.ByteBuffer + +import scala.collection.mutable.HashMap +import scala.util.control.NonFatal import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.Task -import org.apache.spark.serializer.{SerializerInstance, Serializer} +import org.apache.spark.serializer.SerializerInstance -import scala.collection.mutable.HashMap -import scala.util.control.NonFatal +object SerializationState extends Enumeration { + // Define vars to standardize debugging output + type SerializationState = String + var Failed = "Failed to serialize parent." + var FailedDeps = "Failed to serialize dependencies." + val Success = "Success" +} /** * This class is designed to encapsulate some utilities to facilitate debugging serialization * problems in the DAGScheduler and the TaskSetManager. See SPARK-3694. */ object SerializationHelper { - // Define vars to standardize debugging output - var Failed = "Failed to serialize" - var FailedDeps = "Failed to serialize dependencies" - var Serialized = "Serialized" - + type SerializedRdd = Either[String,ByteBuffer] + /** * Helper function to check whether an RDD is serializable. * @@ -52,53 +57,50 @@ object SerializationHelper { * * @param closureSerializer - An instance of a serializer (single-threaded) that will be used * @param rdd - Rdd to attempt to serialize - * @return - An output string qualifying success or failure. + * @return SerializedRdd - If serialization is successful, return the serialized bytes, else + * return a String, which clarifies why things failed. + * + * */ - def isSerializable(closureSerializer : SerializerInstance, rdd : RDD[_]) : String = { - try { - closureSerializer.serialize(rdd: AnyRef) - Serialized + ": " + rdd.toString - } - catch { + def tryToSerialize(closureSerializer : SerializerInstance, + rdd : RDD[_]) : SerializedRdd = { + val result: SerializedRdd = try { + Right(closureSerializer.serialize(rdd: AnyRef)) + } catch { case e: NotSerializableException => - handleFailure(closureSerializer, rdd) + Left(handleFailure(closureSerializer, rdd)) case NonFatal(e) => - handleFailure(closureSerializer, rdd) + Left(handleFailure(closureSerializer, rdd)) } + + result } /** - * Helper function to seperate an un-serialiable parent rdd from un-serializable dependencies + * Helper function to separate an un-serializable parent rdd from un-serializable dependencies * @param closureSerializer - An instance of a serializer (single-threaded) that will be used * @param rdd - Rdd to attempt to serialize - * @return - An output string qualifying success or failure. + * @return String - Return a String (SerializationFailure), which clarifies why the serialization + * failed. */ def handleFailure(closureSerializer : SerializerInstance, - rdd: RDD[_]): String ={ - if(rdd.dependencies.length > 0){ - try{ - rdd.dependencies.foreach(dep => closureSerializer.serialize(dep : AnyRef)) - - // By default, return a failure since we still failed to serialize the parent RDD - // Now, however, we know that the dependencies are serializable - Failed + ": " + rdd.toString - } - catch { - // If instead, however, the dependencies ALSO fail to serialize then the subsequent stage - // of evaluation will help identify which of the dependencies has failed - case e: NotSerializableException => - FailedDeps + ": " + rdd.toString + rdd: RDD[_]): String ={ + try { + rdd.dependencies.foreach(dep => closureSerializer.serialize(dep : AnyRef)) - case NonFatal(e) => - FailedDeps + ": " + rdd.toString - } + // By default, return a failure since we still failed to serialize the parent RDD + // Now, however, we know that the dependencies are serializable + SerializationState.Failed + } catch { + // If instead, however, the dependencies ALSO fail to serialize then the subsequent stage + // of evaluation will help identify which of the dependencies has failed + case e: NotSerializableException => + SerializationState.FailedDeps + case NonFatal(e) => + SerializationState.FailedDeps } - else{ - Failed + ": " + rdd.toString - } - } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 75b96548f373..80d8f13a4b7b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -31,7 +31,9 @@ import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} -import org.apache.spark.util.{SerializationHelper, CallSite} +import org.apache.spark.util.{SerializationState, SerializationHelper, CallSite} +import org.apache.spark.util.SerializationHelper.SerializedRdd + import org.apache.spark.executor.TaskMetrics class BuggyDAGEventProcessActor extends Actor { @@ -251,15 +253,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val unserializable = new UnserializableClass } - val trace = scheduler.getSerializationTrace(unserializableRdd) + val trace : Array[SerializedRdd] = scheduler.tryToSerialize(unserializableRdd) - val splitS = trace.split(":") - val depth = splitS(1).trim() - val status = splitS(2).trim() - val rddName = splitS(3).trim() - - assert(rddName.equals("DAGSchedulerSuiteRDD 0")) - assert(status.equals(SerializationHelper.Failed)) + assert(trace.length == 1) + assert(trace(0).isLeft) //Failed to serialize } test("Serialization trace for unserializable task with serializable dependencies") { @@ -271,23 +268,22 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F class UnserializableClass val unserializable = new UnserializableClass } + + val trace : Array[SerializedRdd] = scheduler.tryToSerialize(finalRdd) - val result = Array(SerializationHelper.Failed, - SerializationHelper.Serialized, - SerializationHelper.Serialized) + val results = Array((false, SerializationState.Failed), + (true, SerializationState.Success), + (true, SerializationState.Success)) - val trace = scheduler.getSerializationTrace(finalRdd) - val splitRdds = trace.split("\n") - - var x = 0 - for(x <- 1 until splitRdds.length){ - val splitS = splitRdds(x).split(":") - val status = splitS(1).trim() - - assert(status.equals(result(x-1))) - - } - + val zipped : Array[(SerializedRdd,Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + zipped.map({ + case (serializationState : SerializedRdd, idx : Int) => { + serializationState match { + case Right(r) => assert(results(idx)._1) //Success + case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings + } + } + }) } test("Serialization trace for serializable task and nested unserializable dependency") { @@ -300,22 +296,20 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) - val result = Array(SerializationHelper.Serialized, - SerializationHelper.FailedDeps, - SerializationHelper.Failed) - val trace = scheduler.getSerializationTrace(finalRdd) - val splitRdds = trace.split("\n") - - var x = 0 - - for(x <- 1 until splitRdds.length){ - val splitS = splitRdds(x).split(":") - val status = splitS(1).trim() - - assert(status.equals(result(x-1))) - - } + val results = Array((false, SerializationState.Success), + (true, SerializationState.FailedDeps), + (true, SerializationState.Failed)) + + val zipped : Array[(SerializedRdd,Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + zipped.map({ + case (serializationState : SerializedRdd, idx : Int) => { + serializationState match { + case Right(r) => assert(results(idx)._1) //Success + case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings + } + } + }) } @@ -328,21 +322,19 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val unserializable = new UnserializableClass } val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) - val result = Array(SerializationHelper.FailedDeps, - SerializationHelper.Failed, - SerializationHelper.Serialized) - - val trace = scheduler.getSerializationTrace(finalRdd) - val splitRdds = trace.split("\n") - - var x = 0 - for(x <- 1 until splitRdds.length){ - val splitS = splitRdds(x).split(":") - val status = splitS(1).trim() - assert(status.equals(result(x-1))) - - } - + val results = Array((false, SerializationState.FailedDeps), + (true, SerializationState.Failed), + (true, SerializationState.Success)) + + val zipped : Array[(SerializedRdd,Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + zipped.map({ + case (serializationState : SerializedRdd, idx : Int) => { + serializationState match { + case Right(r) => assert(results(idx)._1) //Success + case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings + } + } + }) } test("Serialization trace for serializable task and nested dependencies") { @@ -354,21 +346,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) - val trace = scheduler.getSerializationTrace(finalRdd) - val splitRdds = trace.split("\n") - - var x = 0 - for(x <- 1 until splitRdds.length){ - val splitS = splitRdds(x).split(":") - val status = splitS(1).trim() - - if(!status.equals(SerializationHelper.Serialized)) - throw new Exception(trace + "\n" +">"+status+":"+SerializationHelper.Serialized+"<") - - assert(status.equals(SerializationHelper.Serialized)) - - } - + val zipped : Array[(SerializedRdd,Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + zipped.map({ + case (serializationState : SerializedRdd, idx : Int) => assert(serializationState.isRight) + }) } From d2abbb55f9f6c1042edd4218c94abe68bef8a6a8 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 1 Dec 2014 15:21:25 -0500 Subject: [PATCH 12/22] Fixed incorrect boolean in test script --- .../spark/util/SerializationHelper.scala | 63 +++++++++---------- .../spark/scheduler/DAGSchedulerSuite.scala | 31 ++++----- 2 files changed, 47 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala index bf81954b2833..cdabb8486c77 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala @@ -40,8 +40,8 @@ object SerializationState extends Enumeration { * problems in the DAGScheduler and the TaskSetManager. See SPARK-3694. */ object SerializationHelper { - type SerializedRdd = Either[String,ByteBuffer] - + type SerializedRdd = Either[String, ByteBuffer] + /** * Helper function to check whether an RDD is serializable. * @@ -59,47 +59,46 @@ object SerializationHelper { * @param rdd - Rdd to attempt to serialize * @return SerializedRdd - If serialization is successful, return the serialized bytes, else * return a String, which clarifies why things failed. - * - * + * + * */ - def tryToSerialize(closureSerializer : SerializerInstance, - rdd : RDD[_]) : SerializedRdd = { + def tryToSerialize(closureSerializer: SerializerInstance, + rdd: RDD[_]): SerializedRdd = { val result: SerializedRdd = try { Right(closureSerializer.serialize(rdd: AnyRef)) } catch { - case e: NotSerializableException => - Left(handleFailure(closureSerializer, rdd)) + case e: NotSerializableException => Left(handleFailure(closureSerializer, rdd)) + + case NonFatal(e) => Left(handleFailure(closureSerializer, rdd)) + } - case NonFatal(e) => - Left(handleFailure(closureSerializer, rdd)) - } - result } - + /** * Helper function to separate an un-serializable parent rdd from un-serializable dependencies * @param closureSerializer - An instance of a serializer (single-threaded) that will be used * @param rdd - Rdd to attempt to serialize * @return String - Return a String (SerializationFailure), which clarifies why the serialization - * failed. + * failed. */ - def handleFailure(closureSerializer : SerializerInstance, - rdd: RDD[_]): String ={ - try { - rdd.dependencies.foreach(dep => closureSerializer.serialize(dep : AnyRef)) - - // By default, return a failure since we still failed to serialize the parent RDD - // Now, however, we know that the dependencies are serializable + def handleFailure(closureSerializer: SerializerInstance, + rdd: RDD[_]): String = { + if (rdd.dependencies.nonEmpty) { + try { + rdd.dependencies.foreach(dep => closureSerializer.serialize(dep: AnyRef)) + + //By default return a parent failure since we know that the parent already failed + SerializationState.Failed + } catch { + // If instead, however, the dependencies ALSO fail to serialize then the subsequent stage + // of evaluation will help identify which of the dependencies has failed + case e: NotSerializableException => SerializationState.FailedDeps + case NonFatal(e) => SerializationState.FailedDeps + } + } + else { SerializationState.Failed - } catch { - // If instead, however, the dependencies ALSO fail to serialize then the subsequent stage - // of evaluation will help identify which of the dependencies has failed - case e: NotSerializableException => - SerializationState.FailedDeps - - case NonFatal(e) => - SerializationState.FailedDeps } } @@ -111,9 +110,9 @@ object SerializationHelper { * @param addedJars - The JAR dependencies * @return String - The task and dependencies as a string */ - def taskDebugString(task : Task[_], - addedFiles : HashMap[String,Long], - addedJars : HashMap[String,Long]): String ={ + def taskDebugString(task: Task[_], + addedFiles: HashMap[String, Long], + addedJars: HashMap[String, Long]): String = { val taskStr = "[" + task.toString + "] \n" val strPrefix = s"-- " val nl = s"\n" diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 80d8f13a4b7b..1d4bc92eb7f1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -247,7 +247,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F runEvent(JobCancelled(jobId)) } - test("Serialization trace for unserializable task") { + test("Serialization trace for un-serializable task") { val unserializableRdd = new MyRDD(sc, 1, Nil) { class UnserializableClass val unserializable = new UnserializableClass @@ -259,7 +259,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assert(trace(0).isLeft) //Failed to serialize } - test("Serialization trace for unserializable task with serializable dependencies") { + test("Serialization trace for un-serializable task with serializable dependencies") { // The trace should show which nested dependency is unserializable val baseRdd = new MyRDD(sc, 1, Nil) @@ -271,18 +271,18 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val trace : Array[SerializedRdd] = scheduler.tryToSerialize(finalRdd) + // Generate results array as (Success/Failure (Boolean) , ResultString (String)) val results = Array((false, SerializationState.Failed), (true, SerializationState.Success), (true, SerializationState.Success)) - val zipped : Array[(SerializedRdd,Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + val zipped : Array[(SerializedRdd, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex zipped.map({ - case (serializationState : SerializedRdd, idx : Int) => { + case (serializationState : SerializedRdd, idx : Int) => serializationState match { case Right(r) => assert(results(idx)._1) //Success case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings } - } }) } @@ -297,18 +297,18 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) - val results = Array((false, SerializationState.Success), - (true, SerializationState.FailedDeps), - (true, SerializationState.Failed)) + // Generate results array as (Success/Failure (Boolean) , ResultString (String)) + val results = Array((false, SerializationState.FailedDeps), + (false, SerializationState.FailedDeps), + (false, SerializationState.Failed)) - val zipped : Array[(SerializedRdd,Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + val zipped : Array[(SerializedRdd, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex zipped.map({ - case (serializationState : SerializedRdd, idx : Int) => { + case (serializationState : SerializedRdd, idx : Int) => serializationState match { case Right(r) => assert(results(idx)._1) //Success case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings } - } }) } @@ -322,18 +322,19 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val unserializable = new UnserializableClass } val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) + + // Generate results array as (Success/Failure (Boolean) , ResultString (String)) val results = Array((false, SerializationState.FailedDeps), - (true, SerializationState.Failed), + (false, SerializationState.Failed), (true, SerializationState.Success)) - val zipped : Array[(SerializedRdd,Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + val zipped : Array[(SerializedRdd, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex zipped.map({ - case (serializationState : SerializedRdd, idx : Int) => { + case (serializationState : SerializedRdd, idx : Int) => serializationState match { case Right(r) => assert(results(idx)._1) //Success case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings } - } }) } From ef3dd39109aca93e899affef8716655aa7669ce0 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 1 Dec 2014 17:08:59 -0500 Subject: [PATCH 13/22] Minor comment fixes --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- .../main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 3 ++- .../main/scala/org/apache/spark/util/SerializationHelper.scala | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b6743281d03b..919b6b87f1a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -842,7 +842,7 @@ class DAGScheduler( val it = results.iterator while(it.hasNext){ - trace += it.next().fold(l => l, r=> SerializationState.Success) + "\n" + trace += rdd.name + ": " + it.next().fold(l => l, r=> SerializationState.Success) + "\n" } trace } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index d79dc9f432b8..cc1ae0a5e658 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -460,7 +460,8 @@ private[spark] class TaskSetManager( // we assume the task can be serialized without exceptions. // Check if serialization debugging is enabled - val debugSerialization: Boolean = sched.sc.getConf.getBoolean("spark.serializer.debug", false) + val debugSerialization: Boolean = sched.sc.getConf. + getBoolean("spark.serializer.debug", false) // If enabled, print out the added JARs and files (as part of the context) if(debugSerialization) { diff --git a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala index cdabb8486c77..61f3b932167e 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala @@ -88,7 +88,7 @@ object SerializationHelper { try { rdd.dependencies.foreach(dep => closureSerializer.serialize(dep: AnyRef)) - //By default return a parent failure since we know that the parent already failed + // By default return a parent failure since we know that the parent already failed SerializationState.Failed } catch { // If instead, however, the dependencies ALSO fail to serialize then the subsequent stage From aff02e9c2e349888a9fbe295cd1d917f6f1bd3b9 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 15 Dec 2014 16:29:26 -0800 Subject: [PATCH 14/22] Added ObjectWalker class to traverse the references of a generic object and return a set of items that make it possible to trace the path to any particular reference --- .../apache/spark/scheduler/DAGScheduler.scala | 15 +-- .../spark/scheduler/TaskSetManager.scala | 6 - .../org/apache/spark/util/ObjectWalker.scala | 106 ++++++++++++++++++ .../org/apache/spark/util/RDDWalker.scala | 15 ++- .../spark/util/SerializationHelper.scala | 4 +- .../spark/scheduler/DAGSchedulerSuite.scala | 22 ++-- 6 files changed, 130 insertions(+), 38 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/ObjectWalker.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 919b6b87f1a6..9b58a2543f63 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -17,15 +17,12 @@ package org.apache.spark.scheduler - import java.io.NotSerializableException import java.nio.ByteBuffer import java.util import java.util.Properties import java.util.concurrent.atomic.AtomicInteger - - import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} import scala.concurrent.Await @@ -813,9 +810,9 @@ class DAGScheduler( */ def tryToSerialize(rdd: RDD[_]): Array[SerializedRdd] = { val traversal : Array[(RDD[_], Int)] = RDDWalker.walk(rdd) - traversal.map({ + traversal.map { case (curRdd, depth) => SerializationHelper.tryToSerialize(closureSerializer, curRdd) - }) + } } /** @@ -841,7 +838,7 @@ class DAGScheduler( var trace = "Serialization trace:\n" val it = results.iterator - while(it.hasNext){ + while (it.hasNext) { trace += rdd.name + ": " + it.next().fold(l => l, r=> SerializationState.Success) + "\n" } trace @@ -894,7 +891,7 @@ class DAGScheduler( // For ResultTask, serialize and broadcast (rdd, func). // Before serialization print out the RDD and its references. - if(debugSerialization) { + if (debugSerialization) { logDebug("RDD Dependencies:\n" + stage.rdd.toDebugString + "\n") logDebug(getSerializationAsString(stage.rdd)) } @@ -943,10 +940,6 @@ class DAGScheduler( // We've already serialized RDDs and closures in taskBinary, but here we check for all other // objects such as Partition. try { - if(debugSerialization) - { - logDebug(SerializationHelper.taskDebugString(tasks.head, sc.addedFiles, sc.addedJars)) - } closureSerializer.serialize(tasks.head) } catch { case e: NotSerializableException => diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index cc1ae0a5e658..bae776050367 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -463,12 +463,6 @@ private[spark] class TaskSetManager( val debugSerialization: Boolean = sched.sc.getConf. getBoolean("spark.serializer.debug", false) - // If enabled, print out the added JARs and files (as part of the context) - if(debugSerialization) { - logDebug(SerializationHelper.taskDebugString(task, sched.sc.addedFiles, - sched.sc.addedJars)) - } - val serializedTask = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) if (serializedTask.limit > TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024 && diff --git a/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala b/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala new file mode 100644 index 000000000000..c97e1eeeae49 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.util + +import java.lang.reflect.{Modifier, Field} + +import com.google.common.collect.{Queues, Sets} + +import scala.collection.mutable + +/** + * This class permits traversing a generic Object's reference graph. This is useful for debugging + * serialization errors. See SPARK-3694. + * + * This code is based on code written by Josh Rosen found here: + * https://gist.github.com/JoshRosen/d6a8972c99992e97d040 + */ +object ObjectWalker { + case class EdgeRef(self : AnyRef, parent : AnyRef) + case class Edge(from: AnyRef, field: Field, to: AnyRef) + + def isTransient(field: Field): Boolean = Modifier.isTransient(field.getModifiers) + def isStatic(field: Field): Boolean = Modifier.isStatic(field.getModifiers) + def isPrimitive(field: Field): Boolean = field.getType.isPrimitive + + /** + * Traverse the graph representing all references between the provided root object, its + * members, and their references in turn. + * + * What we want to be able to do is readily identify un-serializable components AND the path + * to those components. To do this, store the traversal of the graph as a 2-tuple - the actual + * reference visited and its parent. Then, to get the path to the un-serializable reference + * we can simply follow the parent links. + * + * @param rootObj - The root object for which to generate the reference graph + * @return a new java.util.Set containing the 2-tuple of references from the BFS traversal of the + * reference graph along with their parent references. (self, parent) + */ + def buildRefGraph(rootObj: AnyRef): java.util.Set[(AnyRef, AnyRef)] = { + val visited = Sets.newIdentityHashSet[(AnyRef, AnyRef)]() + val toVisit = Queues.newArrayDeque[AnyRef]() + + toVisit.add(rootObj) + while (!toVisit.isEmpty) { + val obj : AnyRef = toVisit.pollFirst() + + // Store the last parent reference to enable quick retrieval of the path to a broken node + var lastParent : AnyRef = null + + if (!visited.contains(obj)) { + visited.add((obj, lastParent)) + lastParent = obj + + // Extract all the fields from the object that would be serialized. Transient and + // static references are not serialized and primitive variables will always be serializable + // and will not contain further references. + for (field <- getAllFields(obj.getClass) + .filterNot(isStatic) + .filterNot(isTransient) + .filterNot(isPrimitive)) { + // Extract the field object and pass to the visitor + val originalAccessibility = field.isAccessible + field.setAccessible(true) + val fieldObj = field.get(obj) + field.setAccessible(originalAccessibility) + + if (fieldObj != null) { + toVisit.add(fieldObj) + } + } + } + } + visited + } + + /** + * Get all fields (including private ones) from this class and its superclasses. + * @param cls - The class from which to retrieve fields + * @return a new mutable.Set representing the fields of the reference + */ + private def getAllFields(cls: Class[_]): Set[Field] = { + val fields = mutable.Set[Field]() + var _cls: Class[_] = cls + while (_cls != null) { + fields ++= _cls.getDeclaredFields + fields ++= _cls.getFields + _cls = _cls.getSuperclass + } + println(fields.map(_.getName).toSeq) + fields.toSet + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala index cd92d01301cd..f82995ab9ef6 100644 --- a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala +++ b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import java.util +import com.google.common.collect.{Queues, Sets} import scala.collection.mutable.ArrayBuffer import scala.language.existentials @@ -25,15 +25,15 @@ import scala.language.existentials import org.apache.spark.rdd.RDD /** - * This class allows trarseal of an RDD's dependency graph. This is + * This class permits traversing the RDD's dependency graph. This is * accomplished by walking the object graph linking these RDDs. This is useful for debugging * internal RDD references. See SPARK-3694. */ object RDDWalker { // Keep track of both the RDD and its depth in the traversal graph. - val walkQueue = new util.ArrayDeque[(RDD[_], Int)] - var visited = new util.HashSet[RDD[_]] + val walkQueue = Queues.newArrayDeque[(RDD[_], Int)] + var visited = Sets.newIdentityHashSet[RDD[_]] /** * Traverse the dependencies of the RDD and store them within an Array along with their depths. @@ -42,16 +42,15 @@ object RDDWalker { * @param rddToWalk - The RDD to traverse along with its dependencies * @return Array[(RDD[_], depth : Int] - An array of results generated by the traversal function */ - def walk(rddToWalk : RDD[_]): Array[(RDD[_], Int)] ={ - + def walk(rddToWalk : RDD[_]): Array[(RDD[_], Int)] = { val results = new ArrayBuffer[(RDD[_], Int)] // Implement as a queue to perform a BFS walkQueue.addFirst(rddToWalk,0) - while(!walkQueue.isEmpty){ + while (!walkQueue.isEmpty) { // Pop from the queue val (rddToProcess : RDD[_], depth:Int) = walkQueue.pollFirst() - if(!visited.contains(rddToProcess)){ + if (!visited.contains(rddToProcess)) { visited.add(rddToProcess) rddToProcess.dependencies.foreach(s => walkQueue.addFirst(s.rdd, depth + 1)) results.append((rddToProcess, depth)) diff --git a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala index 61f3b932167e..804b619aca64 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala @@ -30,8 +30,8 @@ import org.apache.spark.serializer.SerializerInstance object SerializationState extends Enumeration { // Define vars to standardize debugging output type SerializationState = String - var Failed = "Failed to serialize parent." - var FailedDeps = "Failed to serialize dependencies." + val Failed = "Failed to serialize parent." + val FailedDeps = "Failed to serialize dependencies." val Success = "Success" } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 1d4bc92eb7f1..2b1afc551522 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -264,7 +264,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val baseRdd = new MyRDD(sc, 1, Nil) val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) - val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))){ + val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) { class UnserializableClass val unserializable = new UnserializableClass } @@ -277,19 +277,19 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F (true, SerializationState.Success)) val zipped : Array[(SerializedRdd, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex - zipped.map({ + zipped.map { case (serializationState : SerializedRdd, idx : Int) => serializationState match { case Right(r) => assert(results(idx)._1) //Success case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings } - }) + } } test("Serialization trace for serializable task and nested unserializable dependency") { // The trace should show which nested dependency is unserializable - val baseRdd = new MyRDD(sc, 1, Nil){ + val baseRdd = new MyRDD(sc, 1, Nil) { class UnserializableClass val unserializable = new UnserializableClass } @@ -303,13 +303,13 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F (false, SerializationState.Failed)) val zipped : Array[(SerializedRdd, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex - zipped.map({ + zipped.map { case (serializationState : SerializedRdd, idx : Int) => serializationState match { case Right(r) => assert(results(idx)._1) //Success case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings } - }) + } } @@ -317,7 +317,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F // The trace should show which nested dependency is unserializable val baseRdd = new MyRDD(sc, 1, Nil) - val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))){ + val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) { class UnserializableClass val unserializable = new UnserializableClass } @@ -329,13 +329,13 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F (true, SerializationState.Success)) val zipped : Array[(SerializedRdd, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex - zipped.map({ + zipped.map { case (serializationState : SerializedRdd, idx : Int) => serializationState match { case Right(r) => assert(results(idx)._1) //Success case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings } - }) + } } test("Serialization trace for serializable task and nested dependencies") { @@ -348,9 +348,9 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) val zipped : Array[(SerializedRdd,Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex - zipped.map({ + zipped.map { case (serializationState : SerializedRdd, idx : Int) => assert(serializationState.isRight) - }) + } } From 47b027ace9d6b823d991954208ccbf9d26990499 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 15 Dec 2014 17:09:25 -0800 Subject: [PATCH 15/22] Started on updating SerializationHelper to add generic ObjectWalker --- .../org/apache/spark/util/ObjectWalker.scala | 2 +- .../spark/util/SerializationHelper.scala | 28 ++++++++++++++++++- 2 files changed, 28 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala b/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala index c97e1eeeae49..f0cec58b5caa 100644 --- a/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala +++ b/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala @@ -30,7 +30,7 @@ import scala.collection.mutable * https://gist.github.com/JoshRosen/d6a8972c99992e97d040 */ object ObjectWalker { - case class EdgeRef(self : AnyRef, parent : AnyRef) + case class EdgeRef(cur : AnyRef, parent : AnyRef) case class Edge(from: AnyRef, field: Field, to: AnyRef) def isTransient(field: Field): Boolean = Modifier.isTransient(field.getModifiers) diff --git a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala index 804b619aca64..79df31cb513b 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala @@ -20,7 +20,9 @@ package org.apache.spark.util import java.io.NotSerializableException import java.nio.ByteBuffer -import scala.collection.mutable.HashMap +import com.google.common.collect.Sets + +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal import org.apache.spark.rdd.RDD @@ -102,6 +104,30 @@ object SerializationHelper { } } + /** + * When an RDD is identified as un-serializable, use the generic ObjectWalker class to debug + * the references of that RDD + * @param closureSerializer - An instance of a serializer (single-threaded) that will be used + * @param rdd - The RDD that is known to be un-serializable + * @return a Set of ArrayBuffers representing the paths to un-serializable references + */ + def getPathsToBrokenRefs(closureSerializer: SerializerInstance, + rdd: RDD[_]) : java.util.Set[ArrayBuffer[_]]= { + val refGraph : java.util.Set[(AnyRef, AnyRef)] = ObjectWalker.buildRefGraph(rdd) + val brokenRefs = Sets.newIdentityHashSet[ArrayBuffer[_]] + refGraph.forEach{ + case (cur : AnyRef, parent : AnyRef) => + try { + closureSerializer.serialize(cur) + } catch { + // If instead, however, the dependencies ALSO fail to serialize then the subsequent stage + // of evaluation will help identify which of the dependencies has failed + case e: NotSerializableException => SerializationState.FailedDeps + case NonFatal(e) => SerializationState.FailedDeps + } + } + } + /** * Provide a string representation of the task and its dependencies (in terms of added files * and jars that must be shipped with the task) for debugging purposes. From 25d578015e2fc1d045b0f3ae4dc0af569eec4cc7 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 17 Dec 2014 10:12:34 -0800 Subject: [PATCH 16/22] Code needs work, need to properly store references --- .../apache/spark/scheduler/DAGScheduler.scala | 11 +-- .../org/apache/spark/util/ObjectWalker.scala | 60 +++++++++--- .../org/apache/spark/util/RDDWalker.scala | 9 +- .../spark/util/SerializationHelper.scala | 92 ++++++++++++++++--- .../spark/scheduler/DAGSchedulerSuite.scala | 4 +- 5 files changed, 133 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 9b58a2543f63..bd25b7a206d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -832,16 +832,7 @@ class DAGScheduler( * */ def getSerializationAsString(rdd : RDD[_]): String = { - // Next, if there are dependencies, attempt to serialize those - val results : Array[SerializedRdd] = tryToSerialize(rdd) - - var trace = "Serialization trace:\n" - - val it = results.iterator - while (it.hasNext) { - trace += rdd.name + ": " + it.next().fold(l => l, r=> SerializationState.Success) + "\n" - } - trace + SerializationHelper.getSerializationTrace(closureSerializer, rdd, tryToSerialize(rdd)) } /** Called when stage's parents are available and we can now do its task. */ diff --git a/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala b/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala index f0cec58b5caa..b6a8e91a9cb8 100644 --- a/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala +++ b/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala @@ -18,9 +18,10 @@ package org.apache.spark.util import java.lang.reflect.{Modifier, Field} -import com.google.common.collect.{Queues, Sets} +import com.google.common.collect.Queues import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer /** * This class permits traversing a generic Object's reference graph. This is useful for debugging @@ -30,8 +31,11 @@ import scala.collection.mutable * https://gist.github.com/JoshRosen/d6a8972c99992e97d040 */ object ObjectWalker { - case class EdgeRef(cur : AnyRef, parent : AnyRef) - case class Edge(from: AnyRef, field: Field, to: AnyRef) + case class EdgeRef(cur : AnyRef, parent : EdgeRef) { + def equals(other : EdgeRef) : Boolean = { + cur.equals(other.cur) + } + } def isTransient(field: Field): Boolean = Modifier.isTransient(field.getModifiers) def isStatic(field: Field): Boolean = Modifier.isStatic(field.getModifiers) @@ -47,52 +51,76 @@ object ObjectWalker { * we can simply follow the parent links. * * @param rootObj - The root object for which to generate the reference graph - * @return a new java.util.Set containing the 2-tuple of references from the BFS traversal of the + * @return a new Set containing the 2-tuple of references from the traversal of the * reference graph along with their parent references. (self, parent) */ - def buildRefGraph(rootObj: AnyRef): java.util.Set[(AnyRef, AnyRef)] = { - val visited = Sets.newIdentityHashSet[(AnyRef, AnyRef)]() + def buildRefGraph(rootObj: AnyRef): mutable.DoubleLinkedList[AnyRef] = { + val visitedRefs = mutable.Set[AnyRef]() val toVisit = Queues.newArrayDeque[AnyRef]() + val results = mutable.DoubleLinkedList[AnyRef] toVisit.add(rootObj) + while (!toVisit.isEmpty) { val obj : AnyRef = toVisit.pollFirst() - + println("Visting " + obj) // Store the last parent reference to enable quick retrieval of the path to a broken node var lastParent : AnyRef = null - if (!visited.contains(obj)) { - visited.add((obj, lastParent)) + if (!visitedRefs.contains(obj)) { + visitedRefs.add(obj) + results += (obj) lastParent = obj - + println("Length of visited = " + visited.size) // Extract all the fields from the object that would be serialized. Transient and // static references are not serialized and primitive variables will always be serializable // and will not contain further references. - for (field <- getAllFields(obj.getClass) + + for (field <- getAllFields(obj.cur.getClass) .filterNot(isStatic) .filterNot(isTransient) .filterNot(isPrimitive)) { // Extract the field object and pass to the visitor val originalAccessibility = field.isAccessible field.setAccessible(true) - val fieldObj = field.get(obj) + val fieldObj = field.get(obj.cur) field.setAccessible(originalAccessibility) if (fieldObj != null) { - toVisit.add(fieldObj) + println("Adding " + fieldObj + " to visit.") + toVisit.add(EdgeRef(fieldObj, lastParent)) } } } + } visited } + + /** + * Traverse the links from each reference to its parent and return this traversal to generate + * the path of references to the unserializable reference + * @param brokenRef - The EdgeRef object which identifies the broken reference + * @return a new ArrayBuffer containing the path to the broken reference + */ + def pathToBrokenRef(brokenRef : EdgeRef) : ArrayBuffer[AnyRef] = { + val path = new ArrayBuffer[AnyRef]() + var ref = brokenRef + + while (ref != null) { + path.prepend(ref.cur) + ref = ref.parent + } + + path + } /** * Get all fields (including private ones) from this class and its superclasses. * @param cls - The class from which to retrieve fields * @return a new mutable.Set representing the fields of the reference */ - private def getAllFields(cls: Class[_]): Set[Field] = { + private def getAllFields(cls: Class[_]): mutable.Set[Field] = { val fields = mutable.Set[Field]() var _cls: Class[_] = cls while (_cls != null) { @@ -100,7 +128,9 @@ object ObjectWalker { fields ++= _cls.getFields _cls = _cls.getSuperclass } + + println("Length of fields = " + fields.size) println(fields.map(_.getName).toSeq) - fields.toSet + fields } } \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala index f82995ab9ef6..4935aafbdca4 100644 --- a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala +++ b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala @@ -17,8 +17,9 @@ package org.apache.spark.util -import com.google.common.collect.{Queues, Sets} +import com.google.common.collect.Queues +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.language.existentials @@ -32,8 +33,8 @@ import org.apache.spark.rdd.RDD object RDDWalker { // Keep track of both the RDD and its depth in the traversal graph. - val walkQueue = Queues.newArrayDeque[(RDD[_], Int)] - var visited = Sets.newIdentityHashSet[RDD[_]] + val walkQueue = Queues.newArrayDeque[(RDD[_], Int)]() + var visited = mutable.Set[RDD[_]]() /** * Traverse the dependencies of the RDD and store them within an Array along with their depths. @@ -43,7 +44,7 @@ object RDDWalker { * @return Array[(RDD[_], depth : Int] - An array of results generated by the traversal function */ def walk(rddToWalk : RDD[_]): Array[(RDD[_], Int)] = { - val results = new ArrayBuffer[(RDD[_], Int)] + val results = new ArrayBuffer[(RDD[_], Int)]() // Implement as a queue to perform a BFS walkQueue.addFirst(rddToWalk,0) diff --git a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala index 79df31cb513b..06fcacc557f7 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala @@ -20,8 +20,7 @@ package org.apache.spark.util import java.io.NotSerializableException import java.nio.ByteBuffer -import com.google.common.collect.Sets - +import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal @@ -77,6 +76,37 @@ object SerializationHelper { result } + /** + * Returns an array of SerializedRdd objects (which are either the successfully serialized RDD and + * its dependencies or a failure description), as nicely formatted text. + * + * @param closureSerializer - An instance of a serializer (single-threaded) that will be used + * @param rdd - The top-level rdd that we are attempting to serialize + * @param results - The result of attempting to serialize that rdd + + * @return + */ + def getSerializationTrace(closureSerializer: SerializerInstance, + rdd : RDD[_], + results : Array[SerializedRdd]) = { + var trace = "RDD serialization trace:\n" + val it = results.iterator + + while (it.hasNext) { + trace += rdd.name + ": " + it.next().fold(l => { + // When we know that this RDD was explicitly the one that failed, print out the + // path to the broken refs + if (l.equals(SerializationState.Failed)) { + l + "\n" + brokenRefsToString(getPathsToBrokenRefs(closureSerializer, rdd)) + } else { + l + } + + }, r=> SerializationState.Success) + "\n" + } + trace + } + /** * Helper function to separate an un-serializable parent rdd from un-serializable dependencies * @param closureSerializer - An instance of a serializer (single-threaded) that will be used @@ -90,7 +120,7 @@ object SerializationHelper { try { rdd.dependencies.foreach(dep => closureSerializer.serialize(dep: AnyRef)) - // By default return a parent failure since we know that the parent already failed + // By default return a parent failure since we now know that the dependency is serializable SerializationState.Failed } catch { // If instead, however, the dependencies ALSO fail to serialize then the subsequent stage @@ -106,26 +136,62 @@ object SerializationHelper { /** * When an RDD is identified as un-serializable, use the generic ObjectWalker class to debug - * the references of that RDD + * the references of that RDD and generate a set of paths to broken references * @param closureSerializer - An instance of a serializer (single-threaded) that will be used * @param rdd - The RDD that is known to be un-serializable - * @return a Set of ArrayBuffers representing the paths to un-serializable references + * @return a Set of (AnyRef, ArrayBuffer) - a tuple of the unserialiazble reference and the + * path to that reference */ def getPathsToBrokenRefs(closureSerializer: SerializerInstance, - rdd: RDD[_]) : java.util.Set[ArrayBuffer[_]]= { - val refGraph : java.util.Set[(AnyRef, AnyRef)] = ObjectWalker.buildRefGraph(rdd) - val brokenRefs = Sets.newIdentityHashSet[ArrayBuffer[_]] - refGraph.forEach{ - case (cur : AnyRef, parent : AnyRef) => + rdd: RDD[_]) : mutable.Set[(AnyRef, ArrayBuffer[AnyRef])]= { + val refGraph : mutable.Set[ObjectWalker.EdgeRef] = ObjectWalker.buildRefGraph(rdd) + val brokenRefs = mutable.Set[(AnyRef, ArrayBuffer[AnyRef])]() + + refGraph.foreach { + case ref : ObjectWalker.EdgeRef => try { - closureSerializer.serialize(cur) + closureSerializer.serialize(ref.cur) } catch { // If instead, however, the dependencies ALSO fail to serialize then the subsequent stage // of evaluation will help identify which of the dependencies has failed - case e: NotSerializableException => SerializationState.FailedDeps - case NonFatal(e) => SerializationState.FailedDeps + case e: NotSerializableException => brokenRefs.add(ref, ObjectWalker.pathToBrokenRef(ref)) + case NonFatal(e) => brokenRefs.add(ref, ObjectWalker.pathToBrokenRef(ref)) } } + brokenRefs + } + + def refString(ref : AnyRef) : String= { + val refCode = System.identityHashCode(ref) + "Ref (" + ref.getClass + ", " + refCode + ")" + } + + /** + * Given a set of reference and the paths to those references (as a dependency tree), return + * a cleanly formatted string showing these path. + * @param brokenRefPath - a tuple of the un-serialiazble reference and the path to that reference + */ + def brokenRefsToString(brokenRefPath : mutable.Set[(AnyRef, ArrayBuffer[AnyRef])]) : String = { + var trace = "**********************" + brokenRefPath.foreach(trace += brokenRefToString(_) + "**********************\n") + trace + } + + /** + * Given a reference and a path to that reference (as a dependency tree), return a cleanly + * formatted string showing this path. + * @param brokenRefPath - a tuple of the un-serialiazble reference and the path to that reference + */ + def brokenRefToString(brokenRefPath : (AnyRef, ArrayBuffer[AnyRef])) : String = { + val ref = brokenRefPath._1 + val path = brokenRefPath._2 + + var trace = "Un-serializable reference trace for " + ref + " :\n" + path.foreach(s => { + trace += "--- " + refString(s) + "\n" + }) + + trace } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 10c688752206..64278bfba76f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -252,8 +252,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val unserializable = new UnserializableClass } + throw new Exception(scheduler.getSerializationAsString(unserializableRdd)) + val trace : Array[SerializedRdd] = scheduler.tryToSerialize(unserializableRdd) - + assert(trace.length == 1) assert(trace(0).isLeft) //Failed to serialize } From 7a1954713f606268f505c12263468cb82a7a3a2b Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 17 Dec 2014 22:36:23 -0800 Subject: [PATCH 17/22] Updated SerializationHelper to be agnostic to the data type being traversed. --- .../apache/spark/scheduler/DAGScheduler.scala | 70 ++++++---- .../org/apache/spark/util/ObjectWalker.scala | 46 ++----- .../spark/util/SerializationHelper.scala | 121 ++++++------------ .../spark/scheduler/DAGSchedulerSuite.scala | 109 ++++++++++------ 4 files changed, 168 insertions(+), 178 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index bd25b7a206d0..ca92bbf87754 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -44,7 +44,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage._ import org.apache.spark.util.{CallSite, Clock, RDDWalker, SerializationHelper, SerializationState, SystemClock, Utils} -import org.apache.spark.util.SerializationHelper.SerializedRdd +import org.apache.spark.util.SerializationHelper.{BrokenRef, SerializedRef} import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat /** @@ -794,7 +794,7 @@ class DAGScheduler( } /** - * Helper function to check whether an RDD is serializable. + * Helper function to check whether an RDD and its dependencies are serializable. * * Note: This function is defined separately from the SerializationHelper.isSerializable() * since DAGScheduler.isSerializable() is passed as a parameter to the RDDWalker class's graph @@ -803,39 +803,53 @@ class DAGScheduler( * * @param rdd - Rdd to attempt to serialize * @return Array[SerializedRdd] - - * Return an array of Either objects indicating if serialization is successful: + * Return an array of Either objects indicating if serialization is successful. + * Each object represents the RDD or a dependency of the RDD * Success: ByteBuffer - The serialized RDD * Failure: String - The reason for the failure. * */ - def tryToSerialize(rdd: RDD[_]): Array[SerializedRdd] = { + def tryToSerialize(rdd: RDD[_]): Array[SerializedRef] = { + // Walk the RDD so that we can display a trace on a per-dependency basis val traversal : Array[(RDD[_], Int)] = RDDWalker.walk(rdd) + traversal.map { case (curRdd, depth) => SerializationHelper.tryToSerialize(closureSerializer, curRdd) } } /** - * Use the RDDWalker class to execute a graph traversal of an RDD and its dependencies to help - * identify which RDDs are not serializable. In short, attempt to serialize the RDD and catch - * any Exceptions thrown (this is the same mechanism used within submitMissingTasks() to deal with - * serialization failures). - * - * Note: This is defined here since it uses the tryToSerialize function which in turn uses - * the closure serializer. Although the better place for the serializer would be in the - * SerializationHelper, the Helper is not guaranteed to run in a single thread unlike the - * DAGScheduler. - * - * @param rdd - The rdd for which to print the serialization trace to identify un-serializable - * components - * @return - String - The serialization trace - * + * Returns nicely formatted text representing the trace of the failed serialization + * + * Note: This is defined here since it uses the closure serializer. Although the better place for + * the serializer would be in the SerializationHelper, the Helper is not guaranteed to run in a + * single thread unlike the DAGScheduler. + * + * @param ref - The top-level reference that we are attempting to serialize + * @return */ - def getSerializationAsString(rdd : RDD[_]): String = { - SerializationHelper.getSerializationTrace(closureSerializer, rdd, tryToSerialize(rdd)) + def traceBrokenRef(ref: AnyRef): String = { + SerializationHelper.getSerializationTrace(closureSerializer, ref) } - /** Called when stage's parents are available and we can now do its task. */ + /** + * Use the SerializationHelper to execute a graph traversal of a broken reference to identify + * failures. + * + * Note: This is defined here since it uses the closure serializer. Although the better place for + * the serializer would be in the SerializationHelper, the Helper is not guaranteed to run in a + * single thread unlike the DAGScheduler. + * + * @param ref - The broken ref for which to generate a trace + * @return a Set of BrokenRef - a tuple of the un-serialiazble reference and the + * path to that reference + * + */ + def getBrokenRefs(ref : AnyRef): mutable.Set[BrokenRef] = { + SerializationHelper.getPathsToBrokenRefs(closureSerializer, ref) + } + + /** Called when stage's parents are available and we can now do its task. */ private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry @@ -884,7 +898,14 @@ class DAGScheduler( // Before serialization print out the RDD and its references. if (debugSerialization) { logDebug("RDD Dependencies:\n" + stage.rdd.toDebugString + "\n") - logDebug(getSerializationAsString(stage.rdd)) + + val serialization = tryToSerialize(stage.rdd) + + // If we failed to serialize the RDD or any of its dependencies then print the serialization + // trace which will identify these failures + if (serialization.filter(s=>s.isLeft).length > 0) { + logDebug(traceBrokenRef(stage.rdd)) + } } val taskBinaryBytes: Array[Byte] = @@ -931,6 +952,11 @@ class DAGScheduler( // We've already serialized RDDs and closures in taskBinary, but here we check for all other // objects such as Partition. try { + if (debugSerialization) { + logDebug("RDD Dependencies:\n" + stage.rdd.toDebugString + "\n") + logDebug(traceBrokenRef(stage.rdd)) + } + closureSerializer.serialize(tasks.head) } catch { case e: NotSerializableException => diff --git a/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala b/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala index b6a8e91a9cb8..3312778d36c6 100644 --- a/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala +++ b/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala @@ -21,7 +21,7 @@ import java.lang.reflect.{Modifier, Field} import com.google.common.collect.Queues import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer + /** * This class permits traversing a generic Object's reference graph. This is useful for debugging @@ -54,67 +54,45 @@ object ObjectWalker { * @return a new Set containing the 2-tuple of references from the traversal of the * reference graph along with their parent references. (self, parent) */ - def buildRefGraph(rootObj: AnyRef): mutable.DoubleLinkedList[AnyRef] = { + def buildRefGraph(rootObj: AnyRef): mutable.LinkedList[AnyRef] = { val visitedRefs = mutable.Set[AnyRef]() val toVisit = Queues.newArrayDeque[AnyRef]() - val results = mutable.DoubleLinkedList[AnyRef] + var results = mutable.LinkedList[AnyRef]() toVisit.add(rootObj) while (!toVisit.isEmpty) { val obj : AnyRef = toVisit.pollFirst() - println("Visting " + obj) // Store the last parent reference to enable quick retrieval of the path to a broken node - var lastParent : AnyRef = null if (!visitedRefs.contains(obj)) { + results = mutable.LinkedList(obj).append(results) visitedRefs.add(obj) - results += (obj) - lastParent = obj - println("Length of visited = " + visited.size) + // Extract all the fields from the object that would be serialized. Transient and // static references are not serialized and primitive variables will always be serializable // and will not contain further references. - for (field <- getAllFields(obj.cur.getClass) + for (field <- getAllFields(obj.getClass) .filterNot(isStatic) .filterNot(isTransient) .filterNot(isPrimitive)) { // Extract the field object and pass to the visitor val originalAccessibility = field.isAccessible field.setAccessible(true) - val fieldObj = field.get(obj.cur) + val fieldObj = field.get(obj) field.setAccessible(originalAccessibility) if (fieldObj != null) { - println("Adding " + fieldObj + " to visit.") - toVisit.add(EdgeRef(fieldObj, lastParent)) + toVisit.add(fieldObj) } } } } - visited + results } - /** - * Traverse the links from each reference to its parent and return this traversal to generate - * the path of references to the unserializable reference - * @param brokenRef - The EdgeRef object which identifies the broken reference - * @return a new ArrayBuffer containing the path to the broken reference - */ - def pathToBrokenRef(brokenRef : EdgeRef) : ArrayBuffer[AnyRef] = { - val path = new ArrayBuffer[AnyRef]() - var ref = brokenRef - - while (ref != null) { - path.prepend(ref.cur) - ref = ref.parent - } - - path - } - /** * Get all fields (including private ones) from this class and its superclasses. * @param cls - The class from which to retrieve fields @@ -128,9 +106,7 @@ object ObjectWalker { fields ++= _cls.getFields _cls = _cls.getSuperclass } - - println("Length of fields = " + fields.size) - println(fields.map(_.getName).toSeq) + fields } -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala index 06fcacc557f7..dea8cc4c2b3b 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala @@ -21,7 +21,7 @@ import java.io.NotSerializableException import java.nio.ByteBuffer import scala.collection.mutable -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.HashMap import scala.util.control.NonFatal import org.apache.spark.rdd.RDD @@ -41,138 +41,89 @@ object SerializationState extends Enumeration { * problems in the DAGScheduler and the TaskSetManager. See SPARK-3694. */ object SerializationHelper { - type SerializedRdd = Either[String, ByteBuffer] + type PathToRef = mutable.LinkedList[AnyRef] + type BrokenRef = (AnyRef, PathToRef) + type SerializedRef = Either[String, ByteBuffer] /** - * Helper function to check whether an RDD is serializable. + * Helper function to check whether a reference is serializable. * - * If any dependency of an RDD is un-serializable, a NotSerializableException will be thrown - * and the entire RDD will be deemed un-serializable if done with a single try-catch. - * - * Therefore, split the evaluation into two stages, in the first stage attempt to serialize - * the rdd. If it fails, attempt to serialize its dependencies in the failure handler and see - * if those also fail. - * - * This approach will show if any of the dependencies are un-serializable and will not - * incorrectly identify the parent RDD as being serializable. + * If any dependency of an a reference is un-serializable, a NotSerializableException will be + * thrown and then we can execute a serialization trace to identify the problem reference. * * @param closureSerializer - An instance of a serializer (single-threaded) that will be used - * @param rdd - Rdd to attempt to serialize + * @param ref - The top-level reference that we are attempting to serialize * @return SerializedRdd - If serialization is successful, return the serialized bytes, else * return a String, which clarifies why things failed. * * */ def tryToSerialize(closureSerializer: SerializerInstance, - rdd: RDD[_]): SerializedRdd = { - val result: SerializedRdd = try { - Right(closureSerializer.serialize(rdd: AnyRef)) + ref: AnyRef): SerializedRef = { + val result: SerializedRef = try { + Right(closureSerializer.serialize(ref)) } catch { - case e: NotSerializableException => Left(handleFailure(closureSerializer, rdd)) + case e: NotSerializableException => Left(getSerializationTrace(closureSerializer, ref)) - case NonFatal(e) => Left(handleFailure(closureSerializer, rdd)) + case NonFatal(e) => Left(getSerializationTrace(closureSerializer, ref)) } result } /** - * Returns an array of SerializedRdd objects (which are either the successfully serialized RDD and - * its dependencies or a failure description), as nicely formatted text. - * + * Returns nicely formatted text representing the trace of the failed serialization + * * @param closureSerializer - An instance of a serializer (single-threaded) that will be used - * @param rdd - The top-level rdd that we are attempting to serialize - * @param results - The result of attempting to serialize that rdd - + * @param ref - The top-level reference that we are attempting to serialize * @return */ def getSerializationTrace(closureSerializer: SerializerInstance, - rdd : RDD[_], - results : Array[SerializedRdd]) = { - var trace = "RDD serialization trace:\n" - val it = results.iterator - - while (it.hasNext) { - trace += rdd.name + ": " + it.next().fold(l => { - // When we know that this RDD was explicitly the one that failed, print out the - // path to the broken refs - if (l.equals(SerializationState.Failed)) { - l + "\n" + brokenRefsToString(getPathsToBrokenRefs(closureSerializer, rdd)) - } else { - l - } - - }, r=> SerializationState.Success) + "\n" - } + ref : AnyRef) : String = { + var trace = "Reference serialization trace for " + ref.toString + ":\n" + trace += brokenRefsToString(getPathsToBrokenRefs(closureSerializer, ref)) trace } - /** - * Helper function to separate an un-serializable parent rdd from un-serializable dependencies - * @param closureSerializer - An instance of a serializer (single-threaded) that will be used - * @param rdd - Rdd to attempt to serialize - * @return String - Return a String (SerializationFailure), which clarifies why the serialization - * failed. - */ - def handleFailure(closureSerializer: SerializerInstance, - rdd: RDD[_]): String = { - if (rdd.dependencies.nonEmpty) { - try { - rdd.dependencies.foreach(dep => closureSerializer.serialize(dep: AnyRef)) - - // By default return a parent failure since we now know that the dependency is serializable - SerializationState.Failed - } catch { - // If instead, however, the dependencies ALSO fail to serialize then the subsequent stage - // of evaluation will help identify which of the dependencies has failed - case e: NotSerializableException => SerializationState.FailedDeps - case NonFatal(e) => SerializationState.FailedDeps - } - } - else { - SerializationState.Failed - } + def refString(ref : AnyRef) : String = { + val refCode = System.identityHashCode(ref) + "Ref (" + ref.toString + ", Hash: " + refCode + ")" } /** * When an RDD is identified as un-serializable, use the generic ObjectWalker class to debug * the references of that RDD and generate a set of paths to broken references * @param closureSerializer - An instance of a serializer (single-threaded) that will be used - * @param rdd - The RDD that is known to be un-serializable - * @return a Set of (AnyRef, ArrayBuffer) - a tuple of the unserialiazble reference and the + * @param ref - The reference known to be un-serializable + * @return a Set of (AnyRef, LinkedList) - a tuple of the un-serialiazble reference and the * path to that reference */ def getPathsToBrokenRefs(closureSerializer: SerializerInstance, - rdd: RDD[_]) : mutable.Set[(AnyRef, ArrayBuffer[AnyRef])]= { - val refGraph : mutable.Set[ObjectWalker.EdgeRef] = ObjectWalker.buildRefGraph(rdd) - val brokenRefs = mutable.Set[(AnyRef, ArrayBuffer[AnyRef])]() - + ref: AnyRef) : mutable.Set[BrokenRef] = { + val refGraph : mutable.LinkedList[AnyRef] = ObjectWalker.buildRefGraph(ref) + val brokenRefs = mutable.Set[BrokenRef]() + refGraph.foreach { - case ref : ObjectWalker.EdgeRef => + case ref : AnyRef => try { - closureSerializer.serialize(ref.cur) + closureSerializer.serialize(ref) } catch { // If instead, however, the dependencies ALSO fail to serialize then the subsequent stage // of evaluation will help identify which of the dependencies has failed - case e: NotSerializableException => brokenRefs.add(ref, ObjectWalker.pathToBrokenRef(ref)) - case NonFatal(e) => brokenRefs.add(ref, ObjectWalker.pathToBrokenRef(ref)) + case e: NotSerializableException => brokenRefs.add(ref, ObjectWalker.buildRefGraph(ref)) + case NonFatal(e) => brokenRefs.add(ref, ObjectWalker.buildRefGraph(ref)) } } brokenRefs } - def refString(ref : AnyRef) : String= { - val refCode = System.identityHashCode(ref) - "Ref (" + ref.getClass + ", " + refCode + ")" - } - /** * Given a set of reference and the paths to those references (as a dependency tree), return * a cleanly formatted string showing these path. * @param brokenRefPath - a tuple of the un-serialiazble reference and the path to that reference */ - def brokenRefsToString(brokenRefPath : mutable.Set[(AnyRef, ArrayBuffer[AnyRef])]) : String = { - var trace = "**********************" + def brokenRefsToString(brokenRefPath : mutable.Set[BrokenRef]) : String = { + var trace = "**********************\n" brokenRefPath.foreach(trace += brokenRefToString(_) + "**********************\n") trace } @@ -182,11 +133,11 @@ object SerializationHelper { * formatted string showing this path. * @param brokenRefPath - a tuple of the un-serialiazble reference and the path to that reference */ - def brokenRefToString(brokenRefPath : (AnyRef, ArrayBuffer[AnyRef])) : String = { + def brokenRefToString(brokenRefPath : (AnyRef, mutable.LinkedList[AnyRef])) : String = { val ref = brokenRefPath._1 val path = brokenRefPath._2 - var trace = "Un-serializable reference trace for " + ref + " :\n" + var trace = "Un-serializable reference trace for " + ref + ":\n" path.foreach(s => { trace += "--- " + refString(s) + "\n" }) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 64278bfba76f..ba3802566077 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} import org.apache.spark.util.{SerializationState, SerializationHelper, CallSite} -import org.apache.spark.util.SerializationHelper.SerializedRdd +import org.apache.spark.util.SerializationHelper.SerializedRef import org.apache.spark.executor.TaskMetrics @@ -247,21 +247,75 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("Serialization trace for un-serializable task") { + /* Example trace: + RDD serialization trace: + Depth 0: Failed to serialize parent. + ********************** + Un-serializable reference trace for DAGSchedulerSuiteRDD 0 : + --- Ref (class scala.Tuple2, Hash: 198922458) + --- Ref (org.apache.spark.scheduler + .DAGSchedulerSuite$$anonfun$2$$anon$1$UnserializableClass@5d62af13, Hash: 1566748435) + --- Ref (StorageLevel(false, false, false, false, 1), Hash: 1271007740) + --- Ref (None, Hash: 1958058975) + --- Ref (List(), Hash: 450267655) + --- Ref (scala.Tuple2, Hash: 918450821) + --- Ref (DAGSchedulerSuiteRDD 0, Hash: 73759359) + ********************** + Un-serializable reference trace for org.apache.spark.scheduler + .DAGSchedulerSuite$$anonfun$2$$anon$1$UnserializableClass@5d62af13 : + --- Ref (class scala.Tuple2, Hash: 198922458) + --- Ref (StorageLevel(false, false, false, false, 1), Hash: 1271007740) + --- Ref (None, Hash: 1958058975) + --- Ref (List(), Hash: 450267655) + --- Ref (scala.Tuple2, Hash: 918450821) + --- Ref (DAGSchedulerSuiteRDD 0, Hash: 73759359) + --- Ref (org.apache.spark.scheduler + .DAGSchedulerSuite$$anonfun$2$$anon$1$UnserializableClass@5d62af13, Hash: 1566748435) + ********************** + */ val unserializableRdd = new MyRDD(sc, 1, Nil) { class UnserializableClass val unserializable = new UnserializableClass } + print(scheduler.traceBrokenRef(unserializableRdd)) + // First generate the RDD specific trace (which identifies whether an RDD or its dependencies + // are un-serializable + val trace : Array[SerializedRef] = scheduler.tryToSerialize(unserializableRdd) - throw new Exception(scheduler.getSerializationAsString(unserializableRdd)) - - val trace : Array[SerializedRdd] = scheduler.tryToSerialize(unserializableRdd) + // Next generate the general reference graph trace which identifies which specific reference + // is unserializable + val refTrace = scheduler.getBrokenRefs(unserializableRdd) assert(trace.length == 1) assert(trace(0).isLeft) //Failed to serialize } test("Serialization trace for un-serializable task with serializable dependencies") { - // The trace should show which nested dependency is unserializable + /* Example trace: + Reference serialization trace for DAGSchedulerSuiteRDD 2: + ********************** + Un-serializable reference trace for DAGSchedulerSuiteRDD 2: + --- Ref (class scala.Tuple2, Hash: 1296910502) + --- Ref (StorageLevel(false, false, false, false, 1), Hash: 2063786038) + --- Ref (None, Hash: 1105090408) + --- Ref (org.apache.spark.schedule + .DAGSchedulerSuite$$anonfun$3$$anon$2$UnserializableClass@3c2772d1, Hash: 1009218257) + --- Ref (List(), Hash: 1060935276) + --- Ref (scala.Tuple2, Hash: 571481216) + --- Ref (DAGSchedulerSuiteRDD 2, Hash: 2062667107) + ********************** + Un-serializable reference trace for org.apache.spark.scheduler + .DAGSchedulerSuite$$anonfun$3$$anon$2$UnserializableClass@3c2772d1: + --- Ref (class scala.Tuple2, Hash: 1296910502) + --- Ref (StorageLevel(false, false, false, false, 1), Hash: 2063786038) + --- Ref (None, Hash: 1105090408) + --- Ref (List(), Hash: 1060935276) + --- Ref (scala.Tuple2, Hash: 571481216) + --- Ref (DAGSchedulerSuiteRDD 2, Hash: 2062667107) + --- Ref (org.apache.spark.scheduler + .DAGSchedulerSuite$$anonfun$3$$anon$2$UnserializableClass@3c2772d1, Hash: 1009218257) + ********************** + */ val baseRdd = new MyRDD(sc, 1, Nil) val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) @@ -269,21 +323,16 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F class UnserializableClass val unserializable = new UnserializableClass } - - val trace : Array[SerializedRdd] = scheduler.tryToSerialize(finalRdd) + + print(scheduler.traceBrokenRef(finalRdd)) + val trace : Array[SerializedRef] = scheduler.tryToSerialize(finalRdd) // Generate results array as (Success/Failure (Boolean) , ResultString (String)) - val results = Array((false, SerializationState.Failed), - (true, SerializationState.Success), - (true, SerializationState.Success)) + val results = Array(false, true, true) - val zipped : Array[(SerializedRdd, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + val zipped : Array[(SerializedRef, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex zipped.map { - case (serializationState : SerializedRdd, idx : Int) => - serializationState match { - case Right(r) => assert(results(idx)._1) //Success - case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings - } + case (serializationState : SerializedRef, idx : Int) => assert(results(idx)) } } @@ -299,17 +348,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) // Generate results array as (Success/Failure (Boolean) , ResultString (String)) - val results = Array((false, SerializationState.FailedDeps), - (false, SerializationState.FailedDeps), - (false, SerializationState.Failed)) + val results = Array(false,false,false) - val zipped : Array[(SerializedRdd, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + val zipped : Array[(SerializedRef, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex zipped.map { - case (serializationState : SerializedRdd, idx : Int) => - serializationState match { - case Right(r) => assert(results(idx)._1) //Success - case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings - } + case (serializationState : SerializedRef, idx : Int) => assert(results(idx)) } } @@ -325,17 +368,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) // Generate results array as (Success/Failure (Boolean) , ResultString (String)) - val results = Array((false, SerializationState.FailedDeps), - (false, SerializationState.Failed), - (true, SerializationState.Success)) + val results = Array(false,false,true) - val zipped : Array[(SerializedRdd, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + val zipped : Array[(SerializedRef, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex zipped.map { - case (serializationState : SerializedRdd, idx : Int) => - serializationState match { - case Right(r) => assert(results(idx)._1) //Success - case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings - } + case (serializationState : SerializedRef, idx : Int) => assert(results(idx)) } } @@ -348,9 +385,9 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) - val zipped : Array[(SerializedRdd,Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + val zipped : Array[(SerializedRef,Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex zipped.map { - case (serializationState : SerializedRdd, idx : Int) => assert(serializationState.isRight) + case (serializationState : SerializedRef, idx : Int) => assert(serializationState.isRight) } } From bb5f700363dc577b84414e25caedafeb7c247de6 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 18 Dec 2014 00:28:58 -0800 Subject: [PATCH 18/22] Updated DAGScheduler and TaskSetManager classes to add debug printouts for unserializable references in RDDs and tasks. For RDDs, this is a two step process, we first identify which dependency of the RDD is unserializable, and next print out the unserializable references for that RDD. Otherwise, there are too many refernces to easily make sense of --- .../apache/spark/scheduler/DAGScheduler.scala | 51 +++++--- .../spark/scheduler/TaskSetManager.scala | 8 ++ .../spark/util/SerializationHelper.scala | 94 ++++++++++---- .../spark/scheduler/DAGSchedulerSuite.scala | 122 ++++++------------ 4 files changed, 155 insertions(+), 120 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ca92bbf87754..5f91262255ad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -809,13 +809,39 @@ class DAGScheduler( * Failure: String - The reason for the failure. * */ - def tryToSerialize(rdd: RDD[_]): Array[SerializedRef] = { + def tryToSerializeRdd(rdd: RDD[_]): Array[SerializedRef] = { // Walk the RDD so that we can display a trace on a per-dependency basis val traversal : Array[(RDD[_], Int)] = RDDWalker.walk(rdd) - traversal.map { - case (curRdd, depth) => SerializationHelper.tryToSerialize(closureSerializer, curRdd) + // Attempt to serialize each dependency of the RDD (track depth information to facilitate + // debugging) + val serialized = traversal.map { + case (curRdd, depth) => + (curRdd, depth, SerializationHelper.tryToSerialize(closureSerializer, curRdd)) } + + // If serialization was unsuccessful print failures + val anyFailed = serialized.filter(_._3.isLeft).length > 0 + + if (anyFailed) { + // For convenience, first output a trace by depth of whether each dependency was serializable + serialized.map { + case (curRdd, depth, result) => + val out = "Depth " + depth + ": " + curRdd.toString + " - " + result.fold(l => l, + r => SerializationState.Success) + logDebug(out) + } + + // Next, print a specific reference trace for the unserializable RDD + serialized.map { + case (curRdd, depth, result) => + result.fold (l => logDebug(traceBrokenRef(curRdd)), r => {}) + } + + } + + // Lastly return only the results of the attempted serialization for testing purposes + serialized.map(_._3) } /** @@ -897,15 +923,7 @@ class DAGScheduler( // Before serialization print out the RDD and its references. if (debugSerialization) { - logDebug("RDD Dependencies:\n" + stage.rdd.toDebugString + "\n") - - val serialization = tryToSerialize(stage.rdd) - - // If we failed to serialize the RDD or any of its dependencies then print the serialization - // trace which will identify these failures - if (serialization.filter(s=>s.isLeft).length > 0) { - logDebug(traceBrokenRef(stage.rdd)) - } + tryToSerializeRdd(stage.rdd) } val taskBinaryBytes: Array[Byte] = @@ -944,7 +962,7 @@ class DAGScheduler( } if (tasks.size > 0) { - // Preemptively serialize a task to make sure it can be serialized. We are catching this + // Pre-emptively serialize a task to make sure it can be serialized. We are catching this // exception here because it would be fairly hard to catch the non-serializable exception // down the road, where we have several different implementations for local scheduler and // cluster schedulers. @@ -953,8 +971,11 @@ class DAGScheduler( // objects such as Partition. try { if (debugSerialization) { - logDebug("RDD Dependencies:\n" + stage.rdd.toDebugString + "\n") - logDebug(traceBrokenRef(stage.rdd)) + SerializationHelper.tryToSerialize(closureSerializer, tasks.head).fold ( + l => logDebug("Un-serializable reference trace for " + + tasks.head.toString + ":\n" + l), + r => {} + ) } closureSerializer.serialize(tasks.head) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 410cb414055b..6500dbc61201 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -462,6 +462,14 @@ private[spark] class TaskSetManager( // Check if serialization debugging is enabled val debugSerialization: Boolean = sched.sc.getConf. getBoolean("spark.serializer.debug", false) + + if (debugSerialization) { + SerializationHelper.tryToSerialize(ser, task).fold ( + l => logDebug("Un-serializable reference trace for " + + task.toString + ":\n" + l), + r => {} + ) + } val serializedTask = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) diff --git a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala index dea8cc4c2b3b..51fb14cd570a 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala @@ -53,46 +53,72 @@ object SerializationHelper { * * @param closureSerializer - An instance of a serializer (single-threaded) that will be used * @param ref - The top-level reference that we are attempting to serialize - * @return SerializedRdd - If serialization is successful, return the serialized bytes, else - * return a String, which clarifies why things failed. - * - * + * @return SerializedRef - If serialization is successful, return success, else + * return a String, which clarifies why things failed. */ def tryToSerialize(closureSerializer: SerializerInstance, ref: AnyRef): SerializedRef = { val result: SerializedRef = try { Right(closureSerializer.serialize(ref)) } catch { - case e: NotSerializableException => Left(getSerializationTrace(closureSerializer, ref)) - - case NonFatal(e) => Left(getSerializationTrace(closureSerializer, ref)) + case e: NotSerializableException => Left(handleFailure(closureSerializer, ref)) + case NonFatal(e) => Left(handleFailure(closureSerializer, ref)) } result } /** - * Returns nicely formatted text representing the trace of the failed serialization + * Handle failures differently whether the passed reference is an RDD or something else since for + * RDDs we want to explicitly identify failures in dependencies * * @param closureSerializer - An instance of a serializer (single-threaded) that will be used - * @param ref - The top-level reference that we are attempting to serialize - * @return + * @param ref - The top-level reference that we are attempting to serialize + * @return String - Return a String which clarifies why the serialization failed. Either a + * SerializationState for an RDD or the reference trace for all other references */ - def getSerializationTrace(closureSerializer: SerializerInstance, - ref : AnyRef) : String = { - var trace = "Reference serialization trace for " + ref.toString + ":\n" - trace += brokenRefsToString(getPathsToBrokenRefs(closureSerializer, ref)) - trace + def handleFailure(closureSerializer: SerializerInstance, + ref: AnyRef) : String = { + ref match { + case rdd: RDD[_] => + handleFailedRdd(closureSerializer, rdd) + case _ => + getSerializationTrace(closureSerializer,ref) + } } - def refString(ref : AnyRef) : String = { - val refCode = System.identityHashCode(ref) - "Ref (" + ref.toString + ", Hash: " + refCode + ")" + /** + * Helper function to separate an un-serializable parent rdd from un-serializable dependencies + * + * @param closureSerializer - An instance of a serializer (single-threaded) that will be used + * @param rdd - Rdd to attempt to serialize + * @return String - Return a String (SerializationFailure), which clarifies why the serialization + * failed. + */ + def handleFailedRdd(closureSerializer: SerializerInstance, + rdd: RDD[_]): String = { + if (rdd.dependencies.nonEmpty) { + try { + rdd.dependencies.foreach(dep => closureSerializer.serialize(dep: AnyRef)) + + // By default return a parent failure since we know that the parent already failed + SerializationState.Failed + } catch { + // If instead, however, the dependencies ALSO fail to serialize then the subsequent stage + // of evaluation will help identify which of the dependencies has failed + case e: NotSerializableException => SerializationState.FailedDeps + case NonFatal(e) => SerializationState.FailedDeps + } + } + else { + SerializationState.Failed + } } - + /** * When an RDD is identified as un-serializable, use the generic ObjectWalker class to debug * the references of that RDD and generate a set of paths to broken references + * * @param closureSerializer - An instance of a serializer (single-threaded) that will be used * @param ref - The reference known to be un-serializable * @return a Set of (AnyRef, LinkedList) - a tuple of the un-serialiazble reference and the @@ -108,23 +134,43 @@ object SerializationHelper { try { closureSerializer.serialize(ref) } catch { - // If instead, however, the dependencies ALSO fail to serialize then the subsequent stage - // of evaluation will help identify which of the dependencies has failed case e: NotSerializableException => brokenRefs.add(ref, ObjectWalker.buildRefGraph(ref)) case NonFatal(e) => brokenRefs.add(ref, ObjectWalker.buildRefGraph(ref)) } } + brokenRefs } + + /** + * Returns nicely formatted text representing the trace of the failed serialization + * + * @param closureSerializer - An instance of a serializer (single-threaded) that will be used + * @param ref - The top-level reference that we are attempting to serialize + * @return + */ + def getSerializationTrace(closureSerializer: SerializerInstance, + ref : AnyRef) : String = { + var trace = "Un-serializable reference trace for " + ref.toString + ":\n" + trace += brokenRefsToString(getPathsToBrokenRefs(closureSerializer, ref)) + trace + } + + def refString(ref : AnyRef) : String = { + val refCode = System.identityHashCode(ref) + "Ref (" + ref.toString + ", Hash: " + refCode + ")" + } /** * Given a set of reference and the paths to those references (as a dependency tree), return - * a cleanly formatted string showing these path. + * a cleanly formatted string showing these paths. + * * @param brokenRefPath - a tuple of the un-serialiazble reference and the path to that reference */ def brokenRefsToString(brokenRefPath : mutable.Set[BrokenRef]) : String = { var trace = "**********************\n" - brokenRefPath.foreach(trace += brokenRefToString(_) + "**********************\n") + + brokenRefPath.foreach(s => trace += brokenRefToString(s) + "**********************\n") trace } @@ -137,7 +183,7 @@ object SerializationHelper { val ref = brokenRefPath._1 val path = brokenRefPath._2 - var trace = "Un-serializable reference trace for " + ref + ":\n" + var trace = ref + ":\n" path.foreach(s => { trace += "--- " + refString(s) + "\n" }) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index ba3802566077..1607b3a45ed4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -245,77 +245,21 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F private def cancel(jobId: Int) { runEvent(JobCancelled(jobId)) } - + test("Serialization trace for un-serializable task") { - /* Example trace: - RDD serialization trace: - Depth 0: Failed to serialize parent. - ********************** - Un-serializable reference trace for DAGSchedulerSuiteRDD 0 : - --- Ref (class scala.Tuple2, Hash: 198922458) - --- Ref (org.apache.spark.scheduler - .DAGSchedulerSuite$$anonfun$2$$anon$1$UnserializableClass@5d62af13, Hash: 1566748435) - --- Ref (StorageLevel(false, false, false, false, 1), Hash: 1271007740) - --- Ref (None, Hash: 1958058975) - --- Ref (List(), Hash: 450267655) - --- Ref (scala.Tuple2, Hash: 918450821) - --- Ref (DAGSchedulerSuiteRDD 0, Hash: 73759359) - ********************** - Un-serializable reference trace for org.apache.spark.scheduler - .DAGSchedulerSuite$$anonfun$2$$anon$1$UnserializableClass@5d62af13 : - --- Ref (class scala.Tuple2, Hash: 198922458) - --- Ref (StorageLevel(false, false, false, false, 1), Hash: 1271007740) - --- Ref (None, Hash: 1958058975) - --- Ref (List(), Hash: 450267655) - --- Ref (scala.Tuple2, Hash: 918450821) - --- Ref (DAGSchedulerSuiteRDD 0, Hash: 73759359) - --- Ref (org.apache.spark.scheduler - .DAGSchedulerSuite$$anonfun$2$$anon$1$UnserializableClass@5d62af13, Hash: 1566748435) - ********************** - */ val unserializableRdd = new MyRDD(sc, 1, Nil) { class UnserializableClass val unserializable = new UnserializableClass } - print(scheduler.traceBrokenRef(unserializableRdd)) - // First generate the RDD specific trace (which identifies whether an RDD or its dependencies - // are un-serializable - val trace : Array[SerializedRef] = scheduler.tryToSerialize(unserializableRdd) - - // Next generate the general reference graph trace which identifies which specific reference - // is unserializable - val refTrace = scheduler.getBrokenRefs(unserializableRdd) + val trace : Array[SerializedRef] = scheduler.tryToSerializeRdd(unserializableRdd) + assert(trace.length == 1) assert(trace(0).isLeft) //Failed to serialize } test("Serialization trace for un-serializable task with serializable dependencies") { - /* Example trace: - Reference serialization trace for DAGSchedulerSuiteRDD 2: - ********************** - Un-serializable reference trace for DAGSchedulerSuiteRDD 2: - --- Ref (class scala.Tuple2, Hash: 1296910502) - --- Ref (StorageLevel(false, false, false, false, 1), Hash: 2063786038) - --- Ref (None, Hash: 1105090408) - --- Ref (org.apache.spark.schedule - .DAGSchedulerSuite$$anonfun$3$$anon$2$UnserializableClass@3c2772d1, Hash: 1009218257) - --- Ref (List(), Hash: 1060935276) - --- Ref (scala.Tuple2, Hash: 571481216) - --- Ref (DAGSchedulerSuiteRDD 2, Hash: 2062667107) - ********************** - Un-serializable reference trace for org.apache.spark.scheduler - .DAGSchedulerSuite$$anonfun$3$$anon$2$UnserializableClass@3c2772d1: - --- Ref (class scala.Tuple2, Hash: 1296910502) - --- Ref (StorageLevel(false, false, false, false, 1), Hash: 2063786038) - --- Ref (None, Hash: 1105090408) - --- Ref (List(), Hash: 1060935276) - --- Ref (scala.Tuple2, Hash: 571481216) - --- Ref (DAGSchedulerSuiteRDD 2, Hash: 2062667107) - --- Ref (org.apache.spark.scheduler - .DAGSchedulerSuite$$anonfun$3$$anon$2$UnserializableClass@3c2772d1, Hash: 1009218257) - ********************** - */ + // The trace should show which nested dependency is unserializable val baseRdd = new MyRDD(sc, 1, Nil) val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) @@ -323,40 +267,51 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F class UnserializableClass val unserializable = new UnserializableClass } - - print(scheduler.traceBrokenRef(finalRdd)) - val trace : Array[SerializedRef] = scheduler.tryToSerialize(finalRdd) - + + val trace : Array[SerializedRef] = scheduler.tryToSerializeRdd(finalRdd) + // Generate results array as (Success/Failure (Boolean) , ResultString (String)) - val results = Array(false, true, true) - - val zipped : Array[(SerializedRef, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + val results = Array((false, SerializationState.Failed), + (true, SerializationState.Success), + (true, SerializationState.Success)) + + val zipped : Array[(SerializedRef, Int)] = scheduler.tryToSerializeRdd(finalRdd).zipWithIndex zipped.map { - case (serializationState : SerializedRef, idx : Int) => assert(results(idx)) + case (serializationState : SerializedRef, idx : Int) => + serializationState match { + case Right(r) => assert(results(idx)._1) //Success + case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings + } } } test("Serialization trace for serializable task and nested unserializable dependency") { // The trace should show which nested dependency is unserializable - + val baseRdd = new MyRDD(sc, 1, Nil) { class UnserializableClass val unserializable = new UnserializableClass } - + val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) // Generate results array as (Success/Failure (Boolean) , ResultString (String)) - val results = Array(false,false,false) + val results = Array((false, SerializationState.FailedDeps), + (false, SerializationState.FailedDeps), + (false, SerializationState.Failed)) - val zipped : Array[(SerializedRef, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + val zipped : Array[(SerializedRef, Int)] = scheduler.tryToSerializeRdd(finalRdd).zipWithIndex zipped.map { - case (serializationState : SerializedRef, idx : Int) => assert(results(idx)) + case (serializationState : SerializedRef, idx : Int) => + serializationState match { + case Right(r) => assert(results(idx)._1) //Success + case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings + } } - + } - + test("Serialization trace for serializable task with sandwiched unserializable dependency") { // The trace should show which nested dependency is unserializable @@ -368,11 +323,17 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) // Generate results array as (Success/Failure (Boolean) , ResultString (String)) - val results = Array(false,false,true) + val results = Array((false, SerializationState.FailedDeps), + (false, SerializationState.Failed), + (true, SerializationState.Success)) - val zipped : Array[(SerializedRef, Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + val zipped : Array[(SerializedRef, Int)] = scheduler.tryToSerializeRdd(finalRdd).zipWithIndex zipped.map { - case (serializationState : SerializedRef, idx : Int) => assert(results(idx)) + case (serializationState : SerializedRef, idx : Int) => + serializationState match { + case Right(r) => assert(results(idx)._1) //Success + case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings + } } } @@ -385,13 +346,12 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) - val zipped : Array[(SerializedRef,Int)] = scheduler.tryToSerialize(finalRdd).zipWithIndex + val zipped : Array[(SerializedRef,Int)] = scheduler.tryToSerializeRdd(finalRdd).zipWithIndex zipped.map { case (serializationState : SerializedRef, idx : Int) => assert(serializationState.isRight) } } - - + test("[SPARK-3353] parent stage should have lower stage id") { sparkListener.stageByOrderOfExecution.clear() sc.parallelize(1 to 10).map(x => (x, x)).reduceByKey(_ + _, 4).count() From 07142ce80aaf7c5fb8c68782b34253772278380d Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 18 Dec 2014 12:06:38 -0800 Subject: [PATCH 19/22] Moved code out of DAGScheduler into SerializationHelper class to minimize impact on exising classes. Also fixed a bug where unserializable RDD dependencies were not being appropriately printed --- .../apache/spark/scheduler/DAGScheduler.scala | 72 ++------ .../org/apache/spark/util/RDDWalker.scala | 10 +- .../spark/util/SerializationHelper.scala | 173 +++++++++++++----- .../spark/scheduler/DAGSchedulerSuite.scala | 40 ++-- 4 files changed, 170 insertions(+), 125 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5f91262255ad..460266520b53 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -42,9 +42,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage._ -import org.apache.spark.util.{CallSite, Clock, RDDWalker, SerializationHelper, - SerializationState, SystemClock, Utils} -import org.apache.spark.util.SerializationHelper.{BrokenRef, SerializedRef} +import org.apache.spark.util.{CallSite, Clock, RDDTrace, SerializationHelper, SystemClock, Utils} import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat /** @@ -796,6 +794,8 @@ class DAGScheduler( /** * Helper function to check whether an RDD and its dependencies are serializable. * + * This hook is exposed here primarily for testing purposes. + * * Note: This function is defined separately from the SerializationHelper.isSerializable() * since DAGScheduler.isSerializable() is passed as a parameter to the RDDWalker class's graph * traversal, which would otherwise require knowledge of the closureSerializer @@ -809,41 +809,11 @@ class DAGScheduler( * Failure: String - The reason for the failure. * */ - def tryToSerializeRdd(rdd: RDD[_]): Array[SerializedRef] = { - // Walk the RDD so that we can display a trace on a per-dependency basis - val traversal : Array[(RDD[_], Int)] = RDDWalker.walk(rdd) - - // Attempt to serialize each dependency of the RDD (track depth information to facilitate - // debugging) - val serialized = traversal.map { - case (curRdd, depth) => - (curRdd, depth, SerializationHelper.tryToSerialize(closureSerializer, curRdd)) - } - - // If serialization was unsuccessful print failures - val anyFailed = serialized.filter(_._3.isLeft).length > 0 - - if (anyFailed) { - // For convenience, first output a trace by depth of whether each dependency was serializable - serialized.map { - case (curRdd, depth, result) => - val out = "Depth " + depth + ": " + curRdd.toString + " - " + result.fold(l => l, - r => SerializationState.Success) - logDebug(out) - } - - // Next, print a specific reference trace for the unserializable RDD - serialized.map { - case (curRdd, depth, result) => - result.fold (l => logDebug(traceBrokenRef(curRdd)), r => {}) - } - - } - - // Lastly return only the results of the attempted serialization for testing purposes - serialized.map(_._3) + def tryToSerializeRddDeps(rdd: RDD[_]): Array[RDDTrace] = { + SerializationHelper.tryToSerializeRddAndDeps(closureSerializer, rdd) } + /** * Returns nicely formatted text representing the trace of the failed serialization * @@ -851,31 +821,15 @@ class DAGScheduler( * the serializer would be in the SerializationHelper, the Helper is not guaranteed to run in a * single thread unlike the DAGScheduler. * - * @param ref - The top-level reference that we are attempting to serialize + * @param rdd - The top-level reference that we are attempting to serialize * @return */ - def traceBrokenRef(ref: AnyRef): String = { - SerializationHelper.getSerializationTrace(closureSerializer, ref) + def traceBrokenRdd(rdd: RDD[_]): String = { + SerializationHelper.tryToSerializeRdd(closureSerializer, rdd) + .fold(l => l, r => "Successfully serialized " + rdd.toString) } - /** - * Use the SerializationHelper to execute a graph traversal of a broken reference to identify - * failures. - * - * Note: This is defined here since it uses the closure serializer. Although the better place for - * the serializer would be in the SerializationHelper, the Helper is not guaranteed to run in a - * single thread unlike the DAGScheduler. - * - * @param ref - The broken ref for which to generate a trace - * @return a Set of BrokenRef - a tuple of the un-serialiazble reference and the - * path to that reference - * - */ - def getBrokenRefs(ref : AnyRef): mutable.Set[BrokenRef] = { - SerializationHelper.getPathsToBrokenRefs(closureSerializer, ref) - } - - /** Called when stage's parents are available and we can now do its task. */ + /** Called when stage's parents are available and we can now do its task. */ private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry @@ -923,7 +877,9 @@ class DAGScheduler( // Before serialization print out the RDD and its references. if (debugSerialization) { - tryToSerializeRdd(stage.rdd) + SerializationHelper + .tryToSerializeRdd(closureSerializer, stage.rdd) + .fold(l => logDebug(l), r => {}) } val taskBinaryBytes: Array[Byte] = diff --git a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala index 4935aafbdca4..93abf02f463a 100644 --- a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala +++ b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala @@ -31,11 +31,6 @@ import org.apache.spark.rdd.RDD * internal RDD references. See SPARK-3694. */ object RDDWalker { - - // Keep track of both the RDD and its depth in the traversal graph. - val walkQueue = Queues.newArrayDeque[(RDD[_], Int)]() - var visited = mutable.Set[RDD[_]]() - /** * Traverse the dependencies of the RDD and store them within an Array along with their depths. * Return this data structure and subsequently process it. @@ -44,6 +39,11 @@ object RDDWalker { * @return Array[(RDD[_], depth : Int] - An array of results generated by the traversal function */ def walk(rddToWalk : RDD[_]): Array[(RDD[_], Int)] = { + + val walkQueue = Queues.newArrayDeque[(RDD[_], Int)]() + val visited = mutable.Set[RDD[_]]() + + // Keep track of both the RDD and its depth in the traversal graph. val results = new ArrayBuffer[(RDD[_], Int)]() // Implement as a queue to perform a BFS walkQueue.addFirst(rddToWalk,0) diff --git a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala index 51fb14cd570a..4ef6e99399c5 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala @@ -28,14 +28,18 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.Task import org.apache.spark.serializer.SerializerInstance +/** + * This enumeration defines variables use to standardize debugging output + */ object SerializationState extends Enumeration { - // Define vars to standardize debugging output type SerializationState = String val Failed = "Failed to serialize parent." val FailedDeps = "Failed to serialize dependencies." val Success = "Success" } +case class RDDTrace (rdd : RDD[_], depth : Int, result : SerializationHelper.SerializedRef) + /** * This class is designed to encapsulate some utilities to facilitate debugging serialization * problems in the DAGScheduler and the TaskSetManager. See SPARK-3694. @@ -44,59 +48,146 @@ object SerializationHelper { type PathToRef = mutable.LinkedList[AnyRef] type BrokenRef = (AnyRef, PathToRef) type SerializedRef = Either[String, ByteBuffer] - + /** - * Helper function to check whether a reference is serializable. + * Check whether a reference is serializable. * * If any dependency of an a reference is un-serializable, a NotSerializableException will be * thrown and then we can execute a serialization trace to identify the problem reference. * + * The stack trace is returned in the Left side + * * @param closureSerializer - An instance of a serializer (single-threaded) that will be used * @param ref - The top-level reference that we are attempting to serialize * @return SerializedRef - If serialization is successful, return success, else - * return a String, which clarifies why things failed. + * return a String, which clarifies why things failed. */ def tryToSerialize(closureSerializer: SerializerInstance, ref: AnyRef): SerializedRef = { val result: SerializedRef = try { Right(closureSerializer.serialize(ref)) } catch { - case e: NotSerializableException => Left(handleFailure(closureSerializer, ref)) - case NonFatal(e) => Left(handleFailure(closureSerializer, ref)) + case e: NotSerializableException => Left(getSerializationTrace(closureSerializer, ref)) + case NonFatal(e) => Left(getSerializationTrace(closureSerializer, ref)) } result } /** - * Handle failures differently whether the passed reference is an RDD or something else since for - * RDDs we want to explicitly identify failures in dependencies + * Check whether the serialization of the RDD or its dependencies was successful. * * @param closureSerializer - An instance of a serializer (single-threaded) that will be used - * @param ref - The top-level reference that we are attempting to serialize - * @return String - Return a String which clarifies why the serialization failed. Either a - * SerializationState for an RDD or the reference trace for all other references + * @param serialized - Results of attempting to serialize the rdd and its dependencies + * @return the serialized parent rdd if successful + * @throws java.io.NotSerializableException if rdd or its dependencies didn't serialize */ - def handleFailure(closureSerializer: SerializerInstance, - ref: AnyRef) : String = { - ref match { - case rdd: RDD[_] => - handleFailedRdd(closureSerializer, rdd) - case _ => - getSerializationTrace(closureSerializer,ref) + @throws(classOf[NotSerializableException]) + def tryToSerializeRddAndDeps(closureSerializer: SerializerInstance, + serialized : Array[RDDTrace]) : ByteBuffer = { + if (serialized.filter(trace => trace.result.isLeft).length > 0) { + throw new NotSerializableException("Failed to serialize dependencies.") } + + // If we get here we know that this (the serialization of the parent rdd) was successful + serialized(0).result.right.get + } + + /** + * When debugging RDD serialization failures generate the trace differently. + * This is because when RDDs have nested un-serializable dependencies the reference graph becomes + * much harder to trace. Thus, generate a reference trace only for the un-serializable RDDs and + * their parents - not their ancestors. We can still see ancestry from the initially logged + * output. + + * @param closureSerializer - An instance of a serializer (single-threaded) that will be used + * @param rdd - Rdd to attempt to serialize + */ + def tryToSerializeRdd(closureSerializer: SerializerInstance, + rdd: RDD[_]): SerializedRef = { + val serialized: Array[RDDTrace] = tryToSerializeRddAndDeps(closureSerializer, rdd) + + def handleException: Left[String, Nothing] = { + var failedString = "" + + // For convenience, first output a trace by depth of whether each dependency serialized + serialized.map { + case trace: RDDTrace => + val out = ("Depth " + trace.depth + ": " + + trace.rdd.toString + " - " + + trace.result.fold(l => l, r => SerializationState.Success)) + failedString += out + "\n" + } + + // Next, print a specific reference trace for each un-serializable RDD + serialized.map { + case trace: RDDTrace => + trace.result.fold(l => { + failedString += ("" + getSerializationTrace(closureSerializer, trace.rdd) + "\n") + }, r => {}) + } + Left(failedString) + } + + val result: SerializedRef = try { + Right(tryToSerializeRddAndDeps(closureSerializer,serialized)) + } catch { + case e: NotSerializableException => handleException + case NonFatal(e) => handleException + } + + result } /** - * Helper function to separate an un-serializable parent rdd from un-serializable dependencies + * Attempt to serialize an rdd and its dependencies and on a per-rdd basis provide a result. * + * The reason we want to do this is because for RDDs with nested un-serializable dependencies, it + * becomes challenging to read the serialization trace to identify failures. This approach lets us + * only print out the failed RDDs specifically. + * + * @param closureSerializer - An instance of a serializer (single-threaded) that will be used + * @param rdd - Rdd to attempt to serialize + * @return new Array[RDDTrace] where each entry represents one of the RDDs in the tree of the + * parent RDDs dependencies. Each entry provides a reference to the rdd, its depth in the + * tree and the result of serialization. + */ + def tryToSerializeRddAndDeps(closureSerializer: SerializerInstance, + rdd: RDD[_]): Array[RDDTrace] = { + // Walk the RDD so that we can display a trace on a per-dependency basis + val traversal: Array[(RDD[_], Int)] = RDDWalker.walk(rdd) + + def handleException(curRdd: RDD[_]): Left[String, Nothing] = { + Left(handleFailedRdd(closureSerializer, curRdd)) + } + + // Attempt to serialize each dependency of the RDD (track depth information to facilitate + // debugging). + val serialized = traversal.map { + case (curRdd, depth) => + val result: SerializedRef = try { + Right(closureSerializer.serialize(curRdd)) + } catch { + case e: NotSerializableException => handleException(curRdd) + case NonFatal(e) => handleException(curRdd) + } + + RDDTrace(curRdd, depth, result) + } + + serialized + } + + /** + * Helper function to separate an un-serializable parent rdd from un-serializable dependencies + * * @param closureSerializer - An instance of a serializer (single-threaded) that will be used * @param rdd - Rdd to attempt to serialize * @return String - Return a String (SerializationFailure), which clarifies why the serialization - * failed. + * failed. */ - def handleFailedRdd(closureSerializer: SerializerInstance, - rdd: RDD[_]): String = { + private def handleFailedRdd(closureSerializer: SerializerInstance, + rdd: RDD[_]): String = { if (rdd.dependencies.nonEmpty) { try { rdd.dependencies.foreach(dep => closureSerializer.serialize(dep: AnyRef)) @@ -114,7 +205,7 @@ object SerializationHelper { SerializationState.Failed } } - + /** * When an RDD is identified as un-serializable, use the generic ObjectWalker class to debug * the references of that RDD and generate a set of paths to broken references @@ -124,13 +215,13 @@ object SerializationHelper { * @return a Set of (AnyRef, LinkedList) - a tuple of the un-serialiazble reference and the * path to that reference */ - def getPathsToBrokenRefs(closureSerializer: SerializerInstance, - ref: AnyRef) : mutable.Set[BrokenRef] = { - val refGraph : mutable.LinkedList[AnyRef] = ObjectWalker.buildRefGraph(ref) + private def getPathsToBrokenRefs(closureSerializer: SerializerInstance, + ref: AnyRef): mutable.Set[BrokenRef] = { + val refGraph: mutable.LinkedList[AnyRef] = ObjectWalker.buildRefGraph(ref) val brokenRefs = mutable.Set[BrokenRef]() refGraph.foreach { - case ref : AnyRef => + case ref: AnyRef => try { closureSerializer.serialize(ref) } catch { @@ -141,22 +232,22 @@ object SerializationHelper { brokenRefs } - + /** * Returns nicely formatted text representing the trace of the failed serialization - * + * * @param closureSerializer - An instance of a serializer (single-threaded) that will be used * @param ref - The top-level reference that we are attempting to serialize * @return */ def getSerializationTrace(closureSerializer: SerializerInstance, - ref : AnyRef) : String = { + ref: AnyRef): String = { var trace = "Un-serializable reference trace for " + ref.toString + ":\n" trace += brokenRefsToString(getPathsToBrokenRefs(closureSerializer, ref)) trace } - def refString(ref : AnyRef) : String = { + def refString(ref: AnyRef): String = { val refCode = System.identityHashCode(ref) "Ref (" + ref.toString + ", Hash: " + refCode + ")" } @@ -164,30 +255,30 @@ object SerializationHelper { /** * Given a set of reference and the paths to those references (as a dependency tree), return * a cleanly formatted string showing these paths. - * + * * @param brokenRefPath - a tuple of the un-serialiazble reference and the path to that reference */ - def brokenRefsToString(brokenRefPath : mutable.Set[BrokenRef]) : String = { - var trace = "**********************\n" - + private def brokenRefsToString(brokenRefPath: mutable.Set[BrokenRef]): String = { + var trace = "**********************\n" + brokenRefPath.foreach(s => trace += brokenRefToString(s) + "**********************\n") trace } - + /** * Given a reference and a path to that reference (as a dependency tree), return a cleanly * formatted string showing this path. * @param brokenRefPath - a tuple of the un-serialiazble reference and the path to that reference */ - def brokenRefToString(brokenRefPath : (AnyRef, mutable.LinkedList[AnyRef])) : String = { + private def brokenRefToString(brokenRefPath: (AnyRef, mutable.LinkedList[AnyRef])): String = { val ref = brokenRefPath._1 val path = brokenRefPath._2 - + var trace = ref + ":\n" path.foreach(s => { - trace += "--- " + refString(s) + "\n" + trace += "--- " + refString(s) + "\n" }) - + trace } @@ -199,7 +290,7 @@ object SerializationHelper { * @param addedJars - The JAR dependencies * @return String - The task and dependencies as a string */ - def taskDebugString(task: Task[_], + private def taskDebugString(task: Task[_], addedFiles: HashMap[String, Long], addedJars: HashMap[String, Long]): String = { val taskStr = "[" + task.toString + "] \n" diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 1607b3a45ed4..f9c3f4a3bd65 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -30,8 +30,7 @@ import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} -import org.apache.spark.util.{SerializationState, SerializationHelper, CallSite} -import org.apache.spark.util.SerializationHelper.SerializedRef +import org.apache.spark.util.{CallSite, RDDTrace, SerializationState} import org.apache.spark.executor.TaskMetrics @@ -252,10 +251,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val unserializable = new UnserializableClass } - val trace : Array[SerializedRef] = scheduler.tryToSerializeRdd(unserializableRdd) - + val trace : Array[RDDTrace] = scheduler.tryToSerializeRddDeps(unserializableRdd) + assert(trace.length == 1) - assert(trace(0).isLeft) //Failed to serialize + assert(trace(0).result.isLeft) //Failed to serialize } test("Serialization trace for un-serializable task with serializable dependencies") { @@ -267,18 +266,16 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F class UnserializableClass val unserializable = new UnserializableClass } - - val trace : Array[SerializedRef] = scheduler.tryToSerializeRdd(finalRdd) - + // Generate results array as (Success/Failure (Boolean) , ResultString (String)) val results = Array((false, SerializationState.Failed), (true, SerializationState.Success), (true, SerializationState.Success)) - val zipped : Array[(SerializedRef, Int)] = scheduler.tryToSerializeRdd(finalRdd).zipWithIndex + val zipped : Array[(RDDTrace, Int)] = scheduler.tryToSerializeRddDeps(finalRdd).zipWithIndex zipped.map { - case (serializationState : SerializedRef, idx : Int) => - serializationState match { + case (trace : RDDTrace, idx : Int) => + trace.result match { case Right(r) => assert(results(idx)._1) //Success case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings } @@ -301,10 +298,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F (false, SerializationState.FailedDeps), (false, SerializationState.Failed)) - val zipped : Array[(SerializedRef, Int)] = scheduler.tryToSerializeRdd(finalRdd).zipWithIndex + val zipped : Array[(RDDTrace, Int)] = scheduler.tryToSerializeRddDeps(finalRdd).zipWithIndex zipped.map { - case (serializationState : SerializedRef, idx : Int) => - serializationState match { + case (trace : RDDTrace, idx : Int) => + trace.result match { case Right(r) => assert(results(idx)._1) //Success case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings } @@ -326,11 +323,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val results = Array((false, SerializationState.FailedDeps), (false, SerializationState.Failed), (true, SerializationState.Success)) - - val zipped : Array[(SerializedRef, Int)] = scheduler.tryToSerializeRdd(finalRdd).zipWithIndex + + val zipped : Array[(RDDTrace, Int)] = scheduler.tryToSerializeRddDeps(finalRdd).zipWithIndex zipped.map { - case (serializationState : SerializedRef, idx : Int) => - serializationState match { + case (trace : RDDTrace, idx : Int) => + trace.result match { case Right(r) => assert(results(idx)._1) //Success case Left(l) => assert(results(idx)._2.equals(l)) //Match failure strings } @@ -345,10 +342,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val baseRdd = new MyRDD(sc, 1, Nil) val midRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(midRdd))) - - val zipped : Array[(SerializedRef,Int)] = scheduler.tryToSerializeRdd(finalRdd).zipWithIndex + + val zipped : Array[(RDDTrace, Int)] = scheduler.tryToSerializeRddDeps(finalRdd).zipWithIndex + zipped.map { - case (serializationState : SerializedRef, idx : Int) => assert(serializationState.isRight) + case (trace : RDDTrace, idx : Int) => assert(trace.result.isRight) } } From 8e5f7107704d12084efdfac798bfb0791ac23e5f Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 18 Dec 2014 12:10:58 -0800 Subject: [PATCH 20/22] Got rid of unecessary EdgeRef class --- .../main/scala/org/apache/spark/util/ObjectWalker.scala | 7 ------- 1 file changed, 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala b/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala index 3312778d36c6..490efddb7c48 100644 --- a/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala +++ b/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala @@ -31,12 +31,6 @@ import scala.collection.mutable * https://gist.github.com/JoshRosen/d6a8972c99992e97d040 */ object ObjectWalker { - case class EdgeRef(cur : AnyRef, parent : EdgeRef) { - def equals(other : EdgeRef) : Boolean = { - cur.equals(other.cur) - } - } - def isTransient(field: Field): Boolean = Modifier.isTransient(field.getModifiers) def isStatic(field: Field): Boolean = Modifier.isStatic(field.getModifiers) def isPrimitive(field: Field): Boolean = field.getType.isPrimitive @@ -88,7 +82,6 @@ object ObjectWalker { } } } - } results } From 1d2d563c04a7cfb302ccacf42fcfdc8b488a3a61 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 15 Jan 2015 16:20:19 -0800 Subject: [PATCH 21/22] Updated to use scala queues insteadof google queues --- .../apache/spark/scheduler/DAGScheduler.scala | 20 ++++++------ .../spark/scheduler/TaskSetManager.scala | 19 ++++------- .../org/apache/spark/util/ObjectWalker.scala | 32 +++++++++++-------- .../org/apache/spark/util/RDDWalker.scala | 12 +++---- .../spark/util/SerializationHelper.scala | 8 ++--- 5 files changed, 44 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index eb482ddad94a..a5bb2f15bdcc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -810,7 +810,7 @@ class DAGScheduler( * Failure: String - The reason for the failure. * */ - def tryToSerializeRddDeps(rdd: RDD[_]): Array[RDDTrace] = { + private[spark] def tryToSerializeRddDeps(rdd: RDD[_]): Array[RDDTrace] = { SerializationHelper.tryToSerializeRddAndDeps(closureSerializer, rdd) } @@ -869,20 +869,10 @@ class DAGScheduler( // where the JobConf/Configuration object is not thread-safe. var taskBinary: Broadcast[Array[Byte]] = null - // Check if RDD serialization debugging is enabled - val debugSerialization: Boolean = sc.getConf.getBoolean("spark.serializer.debug", false) - try { // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). // For ResultTask, serialize and broadcast (rdd, func). - // Before serialization print out the RDD and its references. - if (debugSerialization) { - SerializationHelper - .tryToSerializeRdd(closureSerializer, stage.rdd) - .fold(l => logDebug(l), r => {}) - } - val taskBinaryBytes: Array[Byte] = if (stage.isShuffleMap) { closureSerializer.serialize((stage.rdd, stage.shuffleDep.get) : AnyRef).array() @@ -893,10 +883,18 @@ class DAGScheduler( } catch { // In the case of a failure during serialization, abort the stage. case e: NotSerializableException => + SerializationHelper + .tryToSerializeRdd(closureSerializer, stage.rdd) + .fold(l => logDebug(l), r => {}) + abortStage(stage, "Task not serializable: " + e.toString) runningStages -= stage return case NonFatal(e) => + SerializationHelper + .tryToSerializeRdd(closureSerializer, stage.rdd) + .fold(l => logDebug(l), r => {}) + abortStage(stage, s"Task serialization failed: $e\n${e.getStackTraceString}") runningStages -= stage return diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index f6502b328d61..a1b2beec4516 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -464,23 +464,18 @@ private[spark] class TaskSetManager( // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here // we assume the task can be serialized without exceptions. - // Check if serialization debugging is enabled - val debugSerialization: Boolean = sched.sc.getConf. - getBoolean("spark.serializer.debug", false) - - if (debugSerialization) { + Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) + } catch { + // If the task cannot be serialized, then there's no point to re-attempt the task, + // as it will always fail. So just abort the whole task-set and print a serialization + // trace to help identify the failure point. + case NonFatal(e) => SerializationHelper.tryToSerialize(ser, task).fold ( l => logDebug("Un-serializable reference trace for " + task.toString + ":\n" + l), r => {} ) - } - - Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) - } catch { - // If the task cannot be serialized, then there's no point to re-attempt the task, - // as it will always fail. So just abort the whole task-set. - case NonFatal(e) => + val msg = s"Failed to serialize task $taskId, not attempting to retry it." logError(msg, e) abort(s"$msg Exception during serialization: $e") diff --git a/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala b/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala index 490efddb7c48..f4c674c5e17b 100644 --- a/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala +++ b/core/src/main/scala/org/apache/spark/util/ObjectWalker.scala @@ -18,8 +18,6 @@ package org.apache.spark.util import java.lang.reflect.{Modifier, Field} -import com.google.common.collect.Queues - import scala.collection.mutable @@ -30,7 +28,7 @@ import scala.collection.mutable * This code is based on code written by Josh Rosen found here: * https://gist.github.com/JoshRosen/d6a8972c99992e97d040 */ -object ObjectWalker { +private[spark] object ObjectWalker { def isTransient(field: Field): Boolean = Modifier.isTransient(field.getModifiers) def isStatic(field: Field): Boolean = Modifier.isStatic(field.getModifiers) def isPrimitive(field: Field): Boolean = field.getType.isPrimitive @@ -50,13 +48,13 @@ object ObjectWalker { */ def buildRefGraph(rootObj: AnyRef): mutable.LinkedList[AnyRef] = { val visitedRefs = mutable.Set[AnyRef]() - val toVisit = Queues.newArrayDeque[AnyRef]() + val toVisit = new mutable.Queue[AnyRef]() var results = mutable.LinkedList[AnyRef]() - toVisit.add(rootObj) + toVisit += rootObj - while (!toVisit.isEmpty) { - val obj : AnyRef = toVisit.pollFirst() + while (toVisit.nonEmpty) { + val obj : AnyRef = toVisit.dequeue() // Store the last parent reference to enable quick retrieval of the path to a broken node if (!visitedRefs.contains(obj)) { @@ -66,11 +64,7 @@ object ObjectWalker { // Extract all the fields from the object that would be serialized. Transient and // static references are not serialized and primitive variables will always be serializable // and will not contain further references. - - for (field <- getAllFields(obj.getClass) - .filterNot(isStatic) - .filterNot(isTransient) - .filterNot(isPrimitive)) { + for (field <- getFieldsToTest(obj)) { // Extract the field object and pass to the visitor val originalAccessibility = field.isAccessible field.setAccessible(true) @@ -78,7 +72,7 @@ object ObjectWalker { field.setAccessible(originalAccessibility) if (fieldObj != null) { - toVisit.add(fieldObj) + toVisit += fieldObj } } } @@ -86,6 +80,18 @@ object ObjectWalker { results } + /** + * Get the serialiazble fields from an object reference + * @param obj - Reference to the object fo rwhich to generate a serialization trace + * @return a new Set containing the serializable fields of the object + */ + def getFieldsToTest(obj: AnyRef): mutable.Set[Field] = { + getAllFields(obj.getClass) + .filterNot(isStatic) + .filterNot(isTransient) + .filterNot(isPrimitive) + } + /** * Get all fields (including private ones) from this class and its superclasses. * @param cls - The class from which to retrieve fields diff --git a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala index 93abf02f463a..c8b96f0cd1ab 100644 --- a/core/src/main/scala/org/apache/spark/util/RDDWalker.scala +++ b/core/src/main/scala/org/apache/spark/util/RDDWalker.scala @@ -17,8 +17,6 @@ package org.apache.spark.util -import com.google.common.collect.Queues - import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.language.existentials @@ -30,7 +28,7 @@ import org.apache.spark.rdd.RDD * accomplished by walking the object graph linking these RDDs. This is useful for debugging * internal RDD references. See SPARK-3694. */ -object RDDWalker { +private[spark] object RDDWalker { /** * Traverse the dependencies of the RDD and store them within an Array along with their depths. * Return this data structure and subsequently process it. @@ -40,20 +38,20 @@ object RDDWalker { */ def walk(rddToWalk : RDD[_]): Array[(RDD[_], Int)] = { - val walkQueue = Queues.newArrayDeque[(RDD[_], Int)]() + val walkQueue = new mutable.Queue[(RDD[_], Int)]() val visited = mutable.Set[RDD[_]]() // Keep track of both the RDD and its depth in the traversal graph. val results = new ArrayBuffer[(RDD[_], Int)]() // Implement as a queue to perform a BFS - walkQueue.addFirst(rddToWalk,0) + walkQueue += ((rddToWalk,0)) while (!walkQueue.isEmpty) { // Pop from the queue - val (rddToProcess : RDD[_], depth:Int) = walkQueue.pollFirst() + val (rddToProcess : RDD[_], depth:Int) = walkQueue.dequeue() if (!visited.contains(rddToProcess)) { visited.add(rddToProcess) - rddToProcess.dependencies.foreach(s => walkQueue.addFirst(s.rdd, depth + 1)) + rddToProcess.dependencies.foreach(s => walkQueue += ((s.rdd, depth + 1))) results.append((rddToProcess, depth)) } } diff --git a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala index 4ef6e99399c5..4c0f3fa54e25 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala @@ -31,20 +31,20 @@ import org.apache.spark.serializer.SerializerInstance /** * This enumeration defines variables use to standardize debugging output */ -object SerializationState extends Enumeration { +private[spark] object SerializationState extends Enumeration { type SerializationState = String val Failed = "Failed to serialize parent." val FailedDeps = "Failed to serialize dependencies." val Success = "Success" } -case class RDDTrace (rdd : RDD[_], depth : Int, result : SerializationHelper.SerializedRef) +private[spark] case class RDDTrace (rdd : RDD[_], depth : Int, result : SerializationHelper.SerializedRef) /** * This class is designed to encapsulate some utilities to facilitate debugging serialization * problems in the DAGScheduler and the TaskSetManager. See SPARK-3694. */ -object SerializationHelper { +private[spark] object SerializationHelper { type PathToRef = mutable.LinkedList[AnyRef] type BrokenRef = (AnyRef, PathToRef) type SerializedRef = Either[String, ByteBuffer] @@ -249,7 +249,7 @@ object SerializationHelper { def refString(ref: AnyRef): String = { val refCode = System.identityHashCode(ref) - "Ref (" + ref.toString + ", Hash: " + refCode + ")" + "Ref (" + ref.toString + ")" } /** From 5b93dc18d541b82394fca97743e96726698586b0 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 15 Jan 2015 17:05:51 -0800 Subject: [PATCH 22/22] Fixed style issue --- .../scala/org/apache/spark/util/SerializationHelper.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala index 4c0f3fa54e25..1bdaf2162b84 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializationHelper.scala @@ -38,7 +38,9 @@ private[spark] object SerializationState extends Enumeration { val Success = "Success" } -private[spark] case class RDDTrace (rdd : RDD[_], depth : Int, result : SerializationHelper.SerializedRef) +private[spark] case class RDDTrace (rdd : RDD[_], + depth : Int, + result : SerializationHelper.SerializedRef) /** * This class is designed to encapsulate some utilities to facilitate debugging serialization