From b14841455ca44b6826d09585b6cc40b0a2aabce0 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 28 Jan 2013 09:24:41 -0800 Subject: track task completion in DAGScheduler, and send a stageCompleted event with taskInfo to SparkListeners --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 16 ++++++++++++++-- .../main/scala/spark/scheduler/DAGSchedulerEvent.scala | 4 +++- core/src/main/scala/spark/scheduler/SparkListener.scala | 9 +++++++++ core/src/main/scala/spark/scheduler/StageInfo.scala | 6 ++++++ .../scala/spark/scheduler/TaskSchedulerListener.scala | 3 ++- .../scala/spark/scheduler/cluster/TaskSetManager.scala | 6 +++--- .../scala/spark/scheduler/local/LocalScheduler.scala | 7 +++++-- 7 files changed, 42 insertions(+), 9 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/SparkListener.scala create mode 100644 core/src/main/scala/spark/scheduler/StageInfo.scala (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index bd541d4207..63f359181b 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -1,5 +1,6 @@ package spark.scheduler +import cluster.TaskInfo import java.net.URI import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.Future @@ -31,8 +32,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Map[Long, Any]) { - eventQueue.put(CompletionEvent(task, reason, result, accumUpdates)) + accumUpdates: Map[Long, Any], + taskInfo: TaskInfo) { + eventQueue.put(CompletionEvent(task, reason, result, accumUpdates, taskInfo)) } // Called by TaskScheduler when an executor fails. @@ -64,6 +66,10 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] + private val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] + + private val sparkListeners = Traversable[SparkListener]() + var cacheLocs = new HashMap[Int, Array[List[String]]] val env = SparkEnv.get @@ -141,6 +147,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val id = nextStageId.getAndIncrement() val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority) idToStage(id) = stage + stageToInfos(stage) = StageInfo(stage) stage } @@ -414,6 +421,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted } pendingTasks(stage) -= task + stageToInfos(stage).taskInfos += event.taskInfo task match { case rt: ResultTask[_, _] => resultStageToJob.get(stage) match { @@ -427,6 +435,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with activeJobs -= job resultStageToJob -= stage running -= stage + val stageComp = StageCompleted(stageToInfos(stage)) + sparkListeners.foreach{_.onStageCompleted(stageComp)} } } case None => @@ -446,6 +456,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with if (running.contains(stage) && pendingTasks(stage).isEmpty) { logInfo(stage + " (" + stage.origin + ") finished; looking for newly runnable stages") running -= stage + val stageComp = StageCompleted(stageToInfos(stage)) + sparkListeners.foreach{_.onStageCompleted(stageComp)} logInfo("running: " + running) logInfo("waiting: " + waiting) logInfo("failed: " + failed) diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index b34fa78c07..9f19e137e4 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -1,5 +1,6 @@ package spark.scheduler +import cluster.TaskInfo import scala.collection.mutable.Map import spark._ @@ -25,7 +26,8 @@ private[spark] case class CompletionEvent( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Map[Long, Any]) + accumUpdates: Map[Long, Any], + taskInfo: TaskInfo) extends DAGSchedulerEvent private[spark] case class ExecutorLost(execId: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala new file mode 100644 index 0000000000..54afc714b3 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -0,0 +1,9 @@ +package spark.scheduler + +trait SparkListener { + def onStageCompleted(stageCompleted: StageCompleted) +} + +sealed trait SparkListenerEvents + +case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala new file mode 100644 index 0000000000..473bdb1c14 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -0,0 +1,6 @@ +package spark.scheduler + +import cluster.TaskInfo +import collection._ + +case class StageInfo(val stage: Stage, val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo]()) diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index 9fcef86e46..4ede03cc2a 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -1,5 +1,6 @@ package spark.scheduler +import cluster.TaskInfo import scala.collection.mutable.Map import spark.TaskEndReason @@ -9,7 +10,7 @@ import spark.TaskEndReason */ private[spark] trait TaskSchedulerListener { // A task has finished or failed. - def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any]): Unit + def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], taskInfo: TaskInfo): Unit // A node was lost from the cluster. def executorLost(execId: String): Unit diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 26201ad0dd..0a57aeda8b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -262,7 +262,7 @@ private[spark] class TaskSetManager( tid, info.duration, tasksFinished, numTasks)) // Deserialize task result and pass it to the scheduler val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates) + sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info) // Mark finished and stop if we've finished all the tasks finished(index) = true if (tasksFinished == numTasks) { @@ -293,7 +293,7 @@ private[spark] class TaskSetManager( reason match { case fetchFailed: FetchFailed => logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress) - sched.listener.taskEnded(tasks(index), fetchFailed, null, null) + sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info) finished(index) = true tasksFinished += 1 sched.taskSetFinished(this) @@ -381,7 +381,7 @@ private[spark] class TaskSetManager( addPendingTask(index) // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our // stage finishes when a total of tasks.size tasks finish. - sched.listener.taskEnded(tasks(index), Resubmitted, null, null) + sched.listener.taskEnded(tasks(index), Resubmitted, null, null, info) } } } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 9ff7c02097..995ab79b5f 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -9,6 +9,7 @@ import scala.collection.mutable.HashMap import spark._ import executor.ExecutorURLClassLoader import spark.scheduler._ +import cluster.TaskInfo /** * A simple TaskScheduler implementation that runs tasks locally in a thread pool. Optionally @@ -54,6 +55,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon def runTask(task: Task[_], idInJob: Int, attemptId: Int) { logInfo("Running task " + idInJob) + val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local") // Set the Spark execution environment for the worker thread SparkEnv.set(env) try { @@ -81,10 +83,11 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]]( ser.serialize(Accumulators.values)) logInfo("Finished task " + idInJob) + info.markSuccessful() // If the threadpool has not already been shutdown, notify DAGScheduler if (!Thread.currentThread().isInterrupted) - listener.taskEnded(task, Success, resultToReturn, accumUpdates) + listener.taskEnded(task, Success, resultToReturn, accumUpdates, info) } catch { case t: Throwable => { logError("Exception in task " + idInJob, t) @@ -95,7 +98,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon } else { // TODO: Do something nicer here to return all the way to the user if (!Thread.currentThread().isInterrupted) - listener.taskEnded(task, new ExceptionFailure(t), null, null) + listener.taskEnded(task, new ExceptionFailure(t), null, null, info) } } } -- cgit v1.2.3 From b88daee91687f35766a4daa523ba780bc1f412d2 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 28 Jan 2013 18:20:37 -0800 Subject: simple util to summarize distributions --- core/src/main/scala/spark/util/Distribution.scala | 53 +++++++++++++++++++++++ 1 file changed, 53 insertions(+) create mode 100644 core/src/main/scala/spark/util/Distribution.scala (limited to 'core') diff --git a/core/src/main/scala/spark/util/Distribution.scala b/core/src/main/scala/spark/util/Distribution.scala new file mode 100644 index 0000000000..99bbb43ff1 --- /dev/null +++ b/core/src/main/scala/spark/util/Distribution.scala @@ -0,0 +1,53 @@ +package spark.util + +import java.io.PrintStream + +/** + * util for getting some stats from a small sample of numeric values, with some handy summary functions + * + * Entirely in memory, not intended as a good way to compute stats over large data sets. + */ +class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { + def this(data: Traversable[Double]) = this(data.toArray, 0, data.size) + java.util.Arrays.sort(data, startIdx, endIdx) + val length = endIdx - startIdx + + val defaultProbabilities = Array(0,0.25,0.5,0.75,1.0) + + /** + * Get the value of the distribution at the given probabilities. Probabilities should be + * given from 0 to 1 + * @param probabilities + */ + def getQuantiles(probabilities: Traversable[Double]) = { + probabilities.map{q =>data((q * length).toInt + startIdx)} + } + + def showQuantiles(out: PrintStream = System.out, probabilities: Traversable[Double] = defaultProbabilities) = { + out.println("min\t25%\t50%\t75%max") + probabilities.foreach{q => out.print(q + "\t")} + out.println + } + + def summary : (StatCounter, Traversable[Double]) = { + (StatCounter(data), getQuantiles(defaultProbabilities)) + } + + /** + * print a summary of this distribution to the given PrintStream. + * @param out + */ + def summary(out: PrintStream = System.out) { + val (statCounter, quantiles) = summary + out.println(statCounter) + Distribution.showQuantiles(out, quantiles) + } +} + +object Distribution { + def showQuantiles(out: PrintStream = System.out, quantiles: Traversable[Double]) { + out.println("min\t25%\t50%\t75%max") + quantiles.foreach{q => out.print(q + "\t")} + out.println + } +} \ No newline at end of file -- cgit v1.2.3 From 38b83bc66bd440810778b6bc543766e25c084112 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 28 Jan 2013 18:20:57 -0800 Subject: can get task runtime summary from task info --- core/src/main/scala/spark/scheduler/StageInfo.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 473bdb1c14..3b9c6d87f0 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -2,5 +2,10 @@ package spark.scheduler import cluster.TaskInfo import collection._ +import spark.util.Distribution -case class StageInfo(val stage: Stage, val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo]()) +case class StageInfo(val stage: Stage, val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo]()) { + def getTaskRuntimeDistribution = { + new Distribution(taskInfos.map{_.duration.toDouble}) + } +} -- cgit v1.2.3 From 01d77f329f5878b7c8672bbdc1859f3ca95d759d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 28 Jan 2013 18:32:52 -0800 Subject: expose stageInfo in SparkContext --- core/src/main/scala/spark/SparkContext.scala | 9 ++++++++- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 2 +- 2 files changed, 9 insertions(+), 2 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index be992250a9..38717a2c17 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -40,13 +40,16 @@ import spark.deploy.LocalSparkCluster import spark.partial.ApproximateEvaluator import spark.partial.PartialResult import rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD} -import scheduler.{ResultTask, ShuffleMapTask, DAGScheduler, TaskScheduler} +import scheduler._ import spark.scheduler.local.LocalScheduler import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import storage.BlockManagerUI +import storage.RDDInfo +import storage.StorageStatus import util.{MetadataCleaner, TimeStampedHashMap} import storage.{StorageStatus, StorageUtils, RDDInfo} +import scala.Some /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -482,6 +485,10 @@ class SparkContext( StorageUtils.rddInfoFromStorageStatus(getSlavesStorageStatus, this) } + def getStageInfo: Map[Stage,StageInfo] = { + dagScheduler.stageToInfos + } + /** * Return information about blocks stored in all of the slaves */ diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 63f359181b..3265d158fb 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -66,7 +66,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] - private val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] + private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] private val sparkListeners = Traversable[SparkListener]() -- cgit v1.2.3 From cec9c768c2eed8d4eefaae0003eeb3971317cef9 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 28 Jan 2013 20:02:53 -0800 Subject: convenient name available in StageInfo --- core/src/main/scala/spark/scheduler/StageInfo.scala | 3 +++ 1 file changed, 3 insertions(+) (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 3b9c6d87f0..21c6d1aee0 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -5,6 +5,9 @@ import collection._ import spark.util.Distribution case class StageInfo(val stage: Stage, val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo]()) { + + def name = stage.rdd.name + "(" + stage.origin + ")" + def getTaskRuntimeDistribution = { new Distribution(taskInfos.map{_.duration.toDouble}) } -- cgit v1.2.3 From 843084d69d1236c95d2b354c22b66dc5d9855406 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 10:15:28 -0800 Subject: track total bytes written by ShuffleMapTasks --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 5 ++++- core/src/main/scala/spark/scheduler/ShuffleMapTask.scala | 7 +++++++ core/src/main/scala/spark/scheduler/StageInfo.scala | 7 ++++++- 3 files changed, 17 insertions(+), 2 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 05aee790b4..c544249241 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -511,8 +511,11 @@ class DAGScheduler( } case smt: ShuffleMapTask => - val stage = idToStage(smt.stageId) val status = event.result.asInstanceOf[MapStatus] + smt.totalBytesWritten match { + case Some(b) => stageToInfos(stage).shuffleBytesWritten += b + case None => throw new RuntimeException("shuffle stask completed without tracking bytes written") + } val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) if (failedGeneration.contains(execId) && smt.generation <= failedGeneration(execId)) { diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index bed9f1864f..ef56b4318c 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -81,6 +81,9 @@ private[spark] class ShuffleMapTask( with Externalizable with Logging { + + var totalBytesWritten : Option[Long] = None + protected def this() = this(0, null, null, 0, null) var split = if (rdd == null) { @@ -130,14 +133,18 @@ private[spark] class ShuffleMapTask( val compressedSizes = new Array[Byte](numOutputSplits) + var totalBytes = 0l + val blockManager = SparkEnv.get.blockManager for (i <- 0 until numOutputSplits) { val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + i // Get a Scala iterator from Java map val iter: Iterator[(Any, Any)] = buckets(i).iterator val size = blockManager.put(blockId, iter, StorageLevel.DISK_ONLY, false) + totalBytes += size compressedSizes(i) = MapOutputTracker.compressSize(size) } + totalBytesWritten = Some(totalBytes) return new MapStatus(blockManager.blockManagerId, compressedSizes) } finally { diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 21c6d1aee0..489b9744de 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -4,7 +4,12 @@ import cluster.TaskInfo import collection._ import spark.util.Distribution -case class StageInfo(val stage: Stage, val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo]()) { +case class StageInfo( + val stage: Stage, + val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo](), + val shuffleBytesWritten : mutable.Buffer[Long] = mutable.Buffer[Long](), + val shuffleBytesRead : mutable.Buffer[Long] = mutable.Buffer[Long]() +) { def name = stage.rdd.name + "(" + stage.origin + ")" -- cgit v1.2.3 From 1ad77c4766313c479e89656928265f2016b0ebcb Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 10:13:08 -0800 Subject: add TimedIterator --- core/src/main/scala/spark/util/TimedIterator.scala | 32 ++++++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100644 core/src/main/scala/spark/util/TimedIterator.scala (limited to 'core') diff --git a/core/src/main/scala/spark/util/TimedIterator.scala b/core/src/main/scala/spark/util/TimedIterator.scala new file mode 100644 index 0000000000..539b01f4ce --- /dev/null +++ b/core/src/main/scala/spark/util/TimedIterator.scala @@ -0,0 +1,32 @@ +package spark.util + +/** + * A utility for tracking the total time an iterator takes to iterate through its elements. + * + * In general, this should only be used if you expect it to take a considerable amount of time + * (eg. milliseconds) to get each element -- otherwise, the timing won't be very accurate, + * and you are probably just adding more overhead + */ +class TimedIterator[+A](val sub: Iterator[A]) extends Iterator[A] { + private var netMillis = 0l + private var nElems = 0 + def hasNext = { + val start = System.currentTimeMillis() + val r = sub.hasNext + val end = System.currentTimeMillis() + netMillis += (end - start) + r + } + def next = { + val start = System.currentTimeMillis() + val r = sub.next + val end = System.currentTimeMillis() + netMillis += (end - start) + nElems += 1 + r + } + + def getNetMillis = netMillis + def getAverageTimePerItem = netMillis / nElems.toDouble + +} -- cgit v1.2.3 From 9df7e2ae55574f9058e26bd06ad976b6e1426647 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 10:14:00 -0800 Subject: Shuffle Fetchers use a timed iterator --- core/src/main/scala/spark/BlockStoreShuffleFetcher.scala | 3 ++- core/src/main/scala/spark/ShuffleFetcher.scala | 4 +++- 2 files changed, 5 insertions(+), 2 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index 86432d0127..cab1975f63 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -4,6 +4,7 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import spark.storage.BlockManagerId +import util.TimedIterator private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { override def fetch[K, V](shuffleId: Int, reduceId: Int) = { @@ -45,6 +46,6 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } } } - blockManager.getMultiple(blocksByAddress).flatMap(unpackBlock) + new TimedIterator(blockManager.getMultiple(blocksByAddress).flatMap(unpackBlock)) } } diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index d9a94d4021..237faa0a6e 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,11 +1,13 @@ package spark +import util.TimedIterator + private[spark] abstract class ShuffleFetcher { /** * Fetch the shuffle outputs for a given ShuffleDependency. * @return An iterator over the elements of the fetched shuffle outputs. */ - def fetch[K, V](shuffleId: Int, reduceId: Int) : Iterator[(K, V)] + def fetch[K, V](shuffleId: Int, reduceId: Int) : TimedIterator[(K, V)] /** Stop the fetcher */ def stop() {} -- cgit v1.2.3 From 295b53439819b3010293f021e82bf0192db04beb Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 10:17:15 -0800 Subject: task context keeps a handle on Task -- giant hack, temporary for tracking shuffle times & amount --- core/src/main/scala/spark/TaskContext.scala | 12 +++++++----- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 2 +- core/src/main/scala/spark/rdd/HadoopRDD.scala | 2 +- core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 2 +- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 2 +- core/src/main/scala/spark/scheduler/ResultTask.scala | 2 +- core/src/main/scala/spark/scheduler/ShuffleMapTask.scala | 2 +- core/src/test/scala/spark/scheduler/TaskContextSuite.scala | 2 +- 8 files changed, 14 insertions(+), 12 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index eab85f85a2..11819db3a3 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -1,19 +1,21 @@ package spark import scala.collection.mutable.ArrayBuffer +import scheduler.Task +class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long, val task: Task[_]) extends Serializable { + //by adding Task here, I'm destroying the separation between Task & TaskContext ... not sure why they need to + // be separate -class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long) extends Serializable { - - @transient val onCompleteCallbacks = new ArrayBuffer[() => Unit] + @transient val onCompleteCallbacks = new ArrayBuffer[TaskContext => Unit] // Add a callback function to be executed on task completion. An example use // is for HadoopRDD to register a callback to close the input stream. - def addOnCompleteCallback(f: () => Unit) { + def addOnCompleteCallback(f: TaskContext => Unit) { onCompleteCallbacks += f } def executeOnCompleteCallbacks() { - onCompleteCallbacks.foreach{_()} + onCompleteCallbacks.foreach{_.apply(this)} } } diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 96b593ba7c..ad08558590 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -104,7 +104,7 @@ private[spark] object CheckpointRDD extends Logging { val deserializeStream = serializer.deserializeStream(fileInputStream) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(() => deserializeStream.close()) + context.addOnCompleteCallback(_ => deserializeStream.close()) deserializeStream.asIterator.asInstanceOf[Iterator[T]] } diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index f547f53812..3f0b4ce23e 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -74,7 +74,7 @@ class HadoopRDD[K, V]( reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(() => reader.close()) + context.addOnCompleteCallback(_ => reader.close()) val key: K = reader.createKey() val value: V = reader.createValue() diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index c3b155fcbd..c80d30e125 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -63,7 +63,7 @@ class NewHadoopRDD[K, V]( reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(() => reader.close()) + context.addOnCompleteCallback(_ => reader.close()) var havePair = false var finished = false diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index c544249241..f07ee50670 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -394,7 +394,7 @@ class DAGScheduler( SparkEnv.set(env) val rdd = job.finalStage.rdd val split = rdd.splits(job.partitions(0)) - val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0) + val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0, null) //TODO figure out what to do about Task here try { val result = job.func(taskContext, rdd.iterator(split, taskContext)) job.listener.taskSucceeded(0, result) diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index 8cd4c661eb..e1b26a0cf5 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -71,7 +71,7 @@ private[spark] class ResultTask[T, U]( } override def run(attemptId: Long): U = { - val context = new TaskContext(stageId, partition, attemptId) + val context = new TaskContext(stageId, partition, attemptId, this) try { func(context, rdd.iterator(split, context)) } finally { diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index ef56b4318c..e97dd0ad3a 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -121,7 +121,7 @@ private[spark] class ShuffleMapTask( override def run(attemptId: Long): MapStatus = { val numOutputSplits = dep.partitioner.numPartitions - val taskContext = new TaskContext(stageId, partition, attemptId) + val taskContext = new TaskContext(stageId, partition, attemptId, this) try { // Partition the map output. val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)]) diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala index a5db7103f5..4af5de6cf6 100644 --- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala @@ -16,7 +16,7 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte val rdd = new RDD[String](sc, List()) { override def getSplits = Array[Split](StubSplit(0)) override def compute(split: Split, context: TaskContext) = { - context.addOnCompleteCallback(() => completed = true) + context.addOnCompleteCallback(_ => completed = true) sys.error("failed") } } -- cgit v1.2.3 From e319ac74c1621500697fa48f6f9d6846d5ea086d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 10:17:56 -0800 Subject: cogrouped RDD stores the amount of time taken to read shuffle data in each task --- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 4 +++- core/src/main/scala/spark/scheduler/Task.scala | 4 ++++ 2 files changed, 7 insertions(+), 1 deletion(-) (limited to 'core') diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 8fafd27bb6..a257e8dd85 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -110,7 +110,9 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) mySeq(depNum) += v } val fetcher = SparkEnv.get.shuffleFetcher - fetcher.fetch[K, Seq[Any]](shuffleId, split.index).foreach(mergePair) + val fetchItr = fetcher.fetch[K, Seq[Any]](shuffleId, split.index) + fetchItr.foreach(mergePair) + context.task.setShuffleReadMillis(fetchItr.getNetMillis) } } JavaConversions.mapAsScalaMap(map).iterator diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala index ef987fdeb6..b99629f67a 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/spark/scheduler/Task.scala @@ -16,6 +16,10 @@ private[spark] abstract class Task[T](val stageId: Int) extends Serializable { def preferredLocations: Seq[String] = Nil var generation: Long = -1 // Map output tracker generation. Will be set by TaskScheduler. + + private var shufflerReadMillis : Option[Long] = None + def setShuffleReadMillis(millis: Long) = shufflerReadMillis = Some(millis) + def getShuffleReadMillis = shufflerReadMillis } /** -- cgit v1.2.3 From b29f9cc978a31ae339186e0e64f64c6f045f482d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 13:40:30 -0800 Subject: BlockManager.getMultiple returns a custom iterator, to enable tracking of shuffle performance --- .../main/scala/spark/storage/BlockManager.scala | 310 +++++++++++---------- 1 file changed, 164 insertions(+), 146 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9893e9625d..96d1a20d1b 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -446,152 +446,8 @@ class BlockManager( * so that we can control the maxMegabytesInFlight for the fetch. */ def getMultiple(blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])]) - : Iterator[(String, Option[Iterator[Any]])] = { - - if (blocksByAddress == null) { - throw new IllegalArgumentException("BlocksByAddress is null") - } - val totalBlocks = blocksByAddress.map(_._2.size).sum - logDebug("Getting " + totalBlocks + " blocks") - var startTime = System.currentTimeMillis - val localBlockIds = new ArrayBuffer[String]() - val remoteBlockIds = new HashSet[String]() - - // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize - // the block (since we want all deserializaton to happen in the calling thread); can also - // represent a fetch failure if size == -1. - class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { - def failed: Boolean = size == -1 - } - - // A queue to hold our results. - val results = new LinkedBlockingQueue[FetchResult] - - // A request to fetch one or more blocks, complete with their sizes - class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { - val size = blocks.map(_._2).sum - } - - // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that - // the number of bytes in flight is limited to maxBytesInFlight - val fetchRequests = new Queue[FetchRequest] - - // Current bytes in flight from our requests - var bytesInFlight = 0L - - def sendRequest(req: FetchRequest) { - logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.memoryBytesToString(req.size), req.address.ip)) - val cmId = new ConnectionManagerId(req.address.ip, req.address.port) - val blockMessageArray = new BlockMessageArray(req.blocks.map { - case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId)) - }) - bytesInFlight += req.size - val sizeMap = req.blocks.toMap // so we can look up the size of each blockID - val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) - future.onSuccess { - case Some(message) => { - val bufferMessage = message.asInstanceOf[BufferMessage] - val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) - for (blockMessage <- blockMessageArray) { - if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { - throw new SparkException( - "Unexpected message " + blockMessage.getType + " received from " + cmId) - } - val blockId = blockMessage.getId - results.put(new FetchResult( - blockId, sizeMap(blockId), () => dataDeserialize(blockId, blockMessage.getData))) - logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) - } - } - case None => { - logError("Could not get block(s) from " + cmId) - for ((blockId, size) <- req.blocks) { - results.put(new FetchResult(blockId, -1, null)) - } - } - } - } - - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val remoteRequests = new ArrayBuffer[FetchRequest] - for ((address, blockInfos) <- blocksByAddress) { - if (address == blockManagerId) { - localBlockIds ++= blockInfos.map(_._1) - } else { - remoteBlockIds ++= blockInfos.map(_._1) - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(String, Long)] - while (iterator.hasNext) { - val (blockId, size) = iterator.next() - curBlocks += ((blockId, size)) - curRequestSize += size - if (curRequestSize >= minRequestSize) { - // Add this FetchRequest - remoteRequests += new FetchRequest(address, curBlocks) - curRequestSize = 0 - curBlocks = new ArrayBuffer[(String, Long)] - } - } - // Add in the final request - if (!curBlocks.isEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) - } - } - } - // Add the remote requests into our queue in a random order - fetchRequests ++= Utils.randomize(remoteRequests) - - // Send out initial requests for blocks, up to our maxBytesInFlight - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - - val numGets = remoteBlockIds.size - fetchRequests.size - logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) - - // Get the local blocks while remote blocks are being fetched. Note that it's okay to do - // these all at once because they will just memory-map some files, so they won't consume - // any memory that might exceed our maxBytesInFlight - startTime = System.currentTimeMillis - for (id <- localBlockIds) { - getLocal(id) match { - case Some(iter) => { - results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight - logDebug("Got local block " + id) - } - case None => { - throw new BlockException(id, "Could not get block " + id + " from local machine") - } - } - } - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - - // Return an iterator that will read fetched blocks off the queue as they arrive. - return new Iterator[(String, Option[Iterator[Any]])] { - var resultsGotten = 0 - - def hasNext: Boolean = resultsGotten < totalBlocks - - def next(): (String, Option[Iterator[Any]]) = { - resultsGotten += 1 - val result = results.take() - bytesInFlight -= result.size - if (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - (result.blockId, if (result.failed) None else Some(result.deserialize())) - } - } + : BlockFetcherIterator = { + return new BlockFetcherIterator(this, blocksByAddress) } def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean) @@ -986,3 +842,165 @@ object BlockManager extends Logging { } } } + + +class BlockFetcherIterator( + private val blockManager: BlockManager, + val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] +) extends Iterator[(String, Option[Iterator[Any]])] with Logging { + + import blockManager._ + + private var remoteBytesRead = 0l + + if (blocksByAddress == null) { + throw new IllegalArgumentException("BlocksByAddress is null") + } + val totalBlocks = blocksByAddress.map(_._2.size).sum + logDebug("Getting " + totalBlocks + " blocks") + var startTime = System.currentTimeMillis + val localBlockIds = new ArrayBuffer[String]() + val remoteBlockIds = new HashSet[String]() + + // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize + // the block (since we want all deserializaton to happen in the calling thread); can also + // represent a fetch failure if size == -1. + class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { + def failed: Boolean = size == -1 + } + + // A queue to hold our results. + val results = new LinkedBlockingQueue[FetchResult] + + // A request to fetch one or more blocks, complete with their sizes + class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { + val size = blocks.map(_._2).sum + } + + // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that + // the number of bytes in flight is limited to maxBytesInFlight + val fetchRequests = new Queue[FetchRequest] + + // Current bytes in flight from our requests + var bytesInFlight = 0L + + def sendRequest(req: FetchRequest) { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.memoryBytesToString(req.size), req.address.ip)) + val cmId = new ConnectionManagerId(req.address.ip, req.address.port) + val blockMessageArray = new BlockMessageArray(req.blocks.map { + case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId)) + }) + bytesInFlight += req.size + val sizeMap = req.blocks.toMap // so we can look up the size of each blockID + val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) + future.onSuccess { + case Some(message) => { + val bufferMessage = message.asInstanceOf[BufferMessage] + val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) + for (blockMessage <- blockMessageArray) { + if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { + throw new SparkException( + "Unexpected message " + blockMessage.getType + " received from " + cmId) + } + val blockId = blockMessage.getId + results.put(new FetchResult( + blockId, sizeMap(blockId), () => dataDeserialize(blockId, blockMessage.getData))) + remoteBytesRead += req.size + logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) + } + } + case None => { + logError("Could not get block(s) from " + cmId) + for ((blockId, size) <- req.blocks) { + results.put(new FetchResult(blockId, -1, null)) + } + } + } + } + + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // at most maxBytesInFlight in order to limit the amount of data in flight. + val remoteRequests = new ArrayBuffer[FetchRequest] + for ((address, blockInfos) <- blocksByAddress) { + if (address == blockManagerId) { + localBlockIds ++= blockInfos.map(_._1) + } else { + remoteBlockIds ++= blockInfos.map(_._1) + // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val minRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[(String, Long)] + while (iterator.hasNext) { + val (blockId, size) = iterator.next() + curBlocks += ((blockId, size)) + curRequestSize += size + if (curRequestSize >= minRequestSize) { + // Add this FetchRequest + remoteRequests += new FetchRequest(address, curBlocks) + curRequestSize = 0 + curBlocks = new ArrayBuffer[(String, Long)] + } + } + // Add in the final request + if (!curBlocks.isEmpty) { + remoteRequests += new FetchRequest(address, curBlocks) + } + } + } + // Add the remote requests into our queue in a random order + fetchRequests ++= Utils.randomize(remoteRequests) + + // Send out initial requests for blocks, up to our maxBytesInFlight + while (!fetchRequests.isEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + + val numGets = remoteBlockIds.size - fetchRequests.size + logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) + + // Get the local blocks while remote blocks are being fetched. Note that it's okay to do + // these all at once because they will just memory-map some files, so they won't consume + // any memory that might exceed our maxBytesInFlight + startTime = System.currentTimeMillis + for (id <- localBlockIds) { + getLocal(id) match { + case Some(iter) => { + results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight + logDebug("Got local block " + id) + } + case None => { + throw new BlockException(id, "Could not get block " + id + " from local machine") + } + } + } + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + + //an iterator that will read fetched blocks off the queue as they arrive. + var resultsGotten = 0 + + def hasNext: Boolean = resultsGotten < totalBlocks + + def next(): (String, Option[Iterator[Any]]) = { + resultsGotten += 1 + val result = results.take() + bytesInFlight -= result.size + if (!fetchRequests.isEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + (result.blockId, if (result.failed) None else Some(result.deserialize())) + } + + + //methods to profile the block fetching + + def numLocalBlocks = localBlockIds.size + def numRemoteBlocks = remoteBlockIds.size + +} -- cgit v1.2.3 From 696e4b21679a2f11e943abb06149476044e0bc82 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 14:29:16 -0800 Subject: track remoteFetchTime --- core/src/main/scala/spark/storage/BlockManager.scala | 4 ++++ 1 file changed, 4 insertions(+) (limited to 'core') diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 96d1a20d1b..d839a1eb89 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -852,6 +852,7 @@ class BlockFetcherIterator( import blockManager._ private var remoteBytesRead = 0l + private var remoteFetchTime = 0l if (blocksByAddress == null) { throw new IllegalArgumentException("BlocksByAddress is null") @@ -893,9 +894,12 @@ class BlockFetcherIterator( }) bytesInFlight += req.size val sizeMap = req.blocks.toMap // so we can look up the size of each blockID + val fetchStart = System.currentTimeMillis() val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) future.onSuccess { case Some(message) => { + val fetchDone = System.currentTimeMillis() + remoteFetchTime += fetchDone - fetchStart val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) for (blockMessage <- blockMessageArray) { -- cgit v1.2.3 From 1704b124d84e50697f66b68120ba340693b1d936 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 14:33:52 -0800 Subject: add as many fetch requests as we can, subject to maxBytesInFlight --- core/src/main/scala/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core') diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index d839a1eb89..71528101db 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -994,7 +994,7 @@ class BlockFetcherIterator( resultsGotten += 1 val result = results.take() bytesInFlight -= result.size - if (!fetchRequests.isEmpty && + while (!fetchRequests.isEmpty && (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { sendRequest(fetchRequests.dequeue()) } -- cgit v1.2.3 From 379564c7e05cfd0acceaee5641e0d06c86985bff Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 18:30:21 -0800 Subject: setup plumbing to get task metrics; lots of unfinished parts, but basic flow in place --- .../scala/spark/BlockStoreShuffleFetcher.scala | 11 +++--- core/src/main/scala/spark/ShuffleFetcher.scala | 3 +- core/src/main/scala/spark/executor/Executor.scala | 2 +- .../main/scala/spark/executor/TaskMetrics.scala | 16 +++++++++ core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 5 ++- core/src/main/scala/spark/scheduler/Task.scala | 7 ++-- .../main/scala/spark/scheduler/TaskResult.scala | 5 +-- .../main/scala/spark/storage/BlockManager.scala | 41 ++++++++++++++++++---- core/src/test/scala/spark/JavaAPISuite.java | 2 +- .../scala/spark/scheduler/DAGSchedulerSuite.scala | 10 +++--- 10 files changed, 78 insertions(+), 24 deletions(-) create mode 100644 core/src/main/scala/spark/executor/TaskMetrics.scala (limited to 'core') diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index cab1975f63..bb4e3de63c 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -3,19 +3,19 @@ package spark import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import spark.storage.BlockManagerId +import storage.{DelegateBlockFetchTracker, BlockManagerId} import util.TimedIterator private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { override def fetch[K, V](shuffleId: Int, reduceId: Int) = { logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) val blockManager = SparkEnv.get.blockManager - + val startTime = System.currentTimeMillis val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, reduceId) logDebug("Fetching map output location for shuffle %d, reduce %d took %d ms".format( shuffleId, reduceId, System.currentTimeMillis - startTime)) - + val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] for (((address, size), index) <- statuses.zipWithIndex) { splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size)) @@ -46,6 +46,9 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } } } - new TimedIterator(blockManager.getMultiple(blocksByAddress).flatMap(unpackBlock)) + val blockFetcherItr = blockManager.getMultiple(blocksByAddress) + val itr = new TimedIterator(blockFetcherItr.flatMap(unpackBlock)) with DelegateBlockFetchTracker + itr.setDelegate(blockFetcherItr) + itr } } diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index 237faa0a6e..d64b2611b7 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,5 +1,6 @@ package spark +import storage.BlockFetchTracker import util.TimedIterator private[spark] abstract class ShuffleFetcher { @@ -7,7 +8,7 @@ private[spark] abstract class ShuffleFetcher { * Fetch the shuffle outputs for a given ShuffleDependency. * @return An iterator over the elements of the fetched shuffle outputs. */ - def fetch[K, V](shuffleId: Int, reduceId: Int) : TimedIterator[(K, V)] + def fetch[K, V](shuffleId: Int, reduceId: Int) : TimedIterator[(K,V)] with BlockFetchTracker /** Stop the fetcher */ def stop() {} diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index bd21ba719a..0906702a3d 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -95,7 +95,7 @@ private[spark] class Executor extends Logging { env.mapOutputTracker.updateGeneration(task.generation) val value = task.run(taskId.toInt) val accumUpdates = Accumulators.values - val result = new TaskResult(value, accumUpdates) + val result = new TaskResult(value, accumUpdates, TaskMetrics(task)) val serializedResult = ser.serialize(result) logInfo("Serialized size of result for " + taskId + " is " + serializedResult.limit) context.statusUpdate(taskId, TaskState.FINISHED, serializedResult) diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala new file mode 100644 index 0000000000..6cdc6d0b7e --- /dev/null +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -0,0 +1,16 @@ +package spark.executor + +import spark.scheduler.Task + +case class TaskMetrics( + val totalBlocksFetched : Option[Int], + val remoteBlocksFetched: Option[Int], + val localBlocksFetched: Option[Int], + val remoteFetchWaitTime: Option[Long], + val remoteBytesRead: Option[Long] +) + +object TaskMetrics { + private[spark] def apply(task: Task[_]) : TaskMetrics = + TaskMetrics(None, None, None, task.remoteFetchWaitTime, task.remoteReadBytes) +} \ No newline at end of file diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index a257e8dd85..90b822ec85 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -112,7 +112,10 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) val fetcher = SparkEnv.get.shuffleFetcher val fetchItr = fetcher.fetch[K, Seq[Any]](shuffleId, split.index) fetchItr.foreach(mergePair) - context.task.setShuffleReadMillis(fetchItr.getNetMillis) + context.task.shuffleReadMillis = Some(fetchItr.getNetMillis) + context.task.remoteFetchTime = Some(fetchItr.remoteFetchTime) + context.task.remoteFetchWaitTime = Some(fetchItr.remoteFetchWaitTime) + context.task.remoteReadBytes = Some(fetchItr.remoteBytesRead) } } JavaConversions.mapAsScalaMap(map).iterator diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala index b99629f67a..9c1d5869be 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/spark/scheduler/Task.scala @@ -17,9 +17,10 @@ private[spark] abstract class Task[T](val stageId: Int) extends Serializable { var generation: Long = -1 // Map output tracker generation. Will be set by TaskScheduler. - private var shufflerReadMillis : Option[Long] = None - def setShuffleReadMillis(millis: Long) = shufflerReadMillis = Some(millis) - def getShuffleReadMillis = shufflerReadMillis + var shuffleReadMillis : Option[Long] = None + var remoteReadBytes : Option[Long] = None + var remoteFetchWaitTime : Option[Long] = None + var remoteFetchTime : Option[Long] = None } /** diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala index 9a54d0e854..003ee2bb2e 100644 --- a/core/src/main/scala/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/spark/scheduler/TaskResult.scala @@ -3,13 +3,14 @@ package spark.scheduler import java.io._ import scala.collection.mutable.Map +import spark.executor.TaskMetrics // Task result. Also contains updates to accumulator variables. // TODO: Use of distributed cache to return result is a hack to get around // what seems to be a bug with messages over 60KB in libprocess; fix it private[spark] -class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any]) extends Externalizable { - def this() = this(null.asInstanceOf[T], null) +class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], val metrics: TaskMetrics) extends Externalizable { + def this() = this(null.asInstanceOf[T], null, null) override def writeExternal(out: ObjectOutput) { out.writeObject(value) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 71528101db..677b2e6a42 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -844,15 +844,37 @@ object BlockManager extends Logging { } +private[spark] trait BlockFetchTracker { + def totalBlocks : Int + def numLocalBlocks: Int + def numRemoteBlocks: Int + def remoteFetchTime : Long + def remoteFetchWaitTime: Long + def remoteBytesRead : Long +} + +private[spark] trait DelegateBlockFetchTracker extends BlockFetchTracker { + var delegate : BlockFetchTracker = _ + def setDelegate(d: BlockFetchTracker) {delegate = d} + def totalBlocks = delegate.totalBlocks + def numLocalBlocks = delegate.numLocalBlocks + def numRemoteBlocks = delegate.numRemoteBlocks + def remoteFetchTime = delegate.remoteFetchTime + def remoteFetchWaitTime = delegate.remoteFetchWaitTime + def remoteBytesRead = delegate.remoteBytesRead +} + + class BlockFetcherIterator( private val blockManager: BlockManager, val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] -) extends Iterator[(String, Option[Iterator[Any]])] with Logging { +) extends Iterator[(String, Option[Iterator[Any]])] with Logging with BlockFetchTracker { import blockManager._ - private var remoteBytesRead = 0l - private var remoteFetchTime = 0l + private var _remoteBytesRead = 0l + private var _remoteFetchTime = 0l + private var _remoteFetchWaitTime = 0l if (blocksByAddress == null) { throw new IllegalArgumentException("BlocksByAddress is null") @@ -899,7 +921,7 @@ class BlockFetcherIterator( future.onSuccess { case Some(message) => { val fetchDone = System.currentTimeMillis() - remoteFetchTime += fetchDone - fetchStart + _remoteFetchTime += fetchDone - fetchStart val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) for (blockMessage <- blockMessageArray) { @@ -910,7 +932,7 @@ class BlockFetcherIterator( val blockId = blockMessage.getId results.put(new FetchResult( blockId, sizeMap(blockId), () => dataDeserialize(blockId, blockMessage.getData))) - remoteBytesRead += req.size + _remoteBytesRead += req.size logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } } @@ -992,7 +1014,10 @@ class BlockFetcherIterator( def next(): (String, Option[Iterator[Any]]) = { resultsGotten += 1 + val startFetchWait = System.currentTimeMillis() val result = results.take() + val stopFetchWait = System.currentTimeMillis() + _remoteFetchWaitTime += (stopFetchWait - startFetchWait) bytesInFlight -= result.size while (!fetchRequests.isEmpty && (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { @@ -1003,8 +1028,12 @@ class BlockFetcherIterator( //methods to profile the block fetching - def numLocalBlocks = localBlockIds.size def numRemoteBlocks = remoteBlockIds.size + def remoteFetchTime = _remoteFetchTime + def remoteFetchWaitTime = _remoteFetchWaitTime + + def remoteBytesRead = _remoteBytesRead + } diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 934e4c2f67..2d13edfd48 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -423,7 +423,7 @@ public class JavaAPISuite implements Serializable { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContext(0, 0, 0); + TaskContext context = new TaskContext(0, 0, 0, null); Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue()); } diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 83663ac702..f74b425dd5 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -265,7 +265,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { - runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, Map[Long, Any]())) + runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, Map[Long, Any](), null)) } } } @@ -463,14 +463,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val noAccum = Map[Long, Any]() // We rely on the event queue being ordered and increasing the generation number by 1 // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null)) // should work because it's a non-failed host - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum, null)) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null)) taskSet.tasks(1).generation = newGeneration val secondStage = interceptStage(reduceRdd) { - runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum)) + runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum, null)) } assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) -- cgit v1.2.3 From 04e828f7c13b83eff84cd2e57a57a821898b5e51 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 8 Feb 2013 19:07:36 -0800 Subject: general fixes to Distribution, plus some tests --- core/src/main/scala/spark/util/Distribution.scala | 34 +++++++++++++++------- .../test/scala/spark/util/DistributionSuite.scala | 25 ++++++++++++++++ 2 files changed, 48 insertions(+), 11 deletions(-) create mode 100644 core/src/test/scala/spark/util/DistributionSuite.scala (limited to 'core') diff --git a/core/src/main/scala/spark/util/Distribution.scala b/core/src/main/scala/spark/util/Distribution.scala index 99bbb43ff1..ccd9232c8b 100644 --- a/core/src/main/scala/spark/util/Distribution.scala +++ b/core/src/main/scala/spark/util/Distribution.scala @@ -6,8 +6,11 @@ import java.io.PrintStream * util for getting some stats from a small sample of numeric values, with some handy summary functions * * Entirely in memory, not intended as a good way to compute stats over large data sets. + * + * assumes you are giving it a non-empty set of data */ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { + require(startIdx < endIdx) def this(data: Traversable[Double]) = this(data.toArray, 0, data.size) java.util.Arrays.sort(data, startIdx, endIdx) val length = endIdx - startIdx @@ -19,34 +22,43 @@ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) * given from 0 to 1 * @param probabilities */ - def getQuantiles(probabilities: Traversable[Double]) = { - probabilities.map{q =>data((q * length).toInt + startIdx)} + def getQuantiles(probabilities: Traversable[Double] = defaultProbabilities) = { + probabilities.toIndexedSeq.map{p:Double => data(closestIndex(p))} } - def showQuantiles(out: PrintStream = System.out, probabilities: Traversable[Double] = defaultProbabilities) = { - out.println("min\t25%\t50%\t75%max") - probabilities.foreach{q => out.print(q + "\t")} - out.println + private def closestIndex(p: Double) = { + math.min((p * length).toInt + startIdx, endIdx - 1) } - def summary : (StatCounter, Traversable[Double]) = { - (StatCounter(data), getQuantiles(defaultProbabilities)) + def showQuantiles(out: PrintStream = System.out) = { + out.println("min\t25%\t50%\t75%\tmax") + getQuantiles(defaultProbabilities).foreach{q => out.print(q + "\t")} + out.println } + def statCounter = StatCounter(data.slice(startIdx, endIdx)) + /** * print a summary of this distribution to the given PrintStream. * @param out */ def summary(out: PrintStream = System.out) { - val (statCounter, quantiles) = summary out.println(statCounter) - Distribution.showQuantiles(out, quantiles) + showQuantiles(out) } } object Distribution { + + def apply(data: Traversable[Double]): Option[Distribution] = { + if (data.size > 0) + Some(new Distribution(data)) + else + None + } + def showQuantiles(out: PrintStream = System.out, quantiles: Traversable[Double]) { - out.println("min\t25%\t50%\t75%max") + out.println("min\t25%\t50%\t75%\tmax") quantiles.foreach{q => out.print(q + "\t")} out.println } diff --git a/core/src/test/scala/spark/util/DistributionSuite.scala b/core/src/test/scala/spark/util/DistributionSuite.scala new file mode 100644 index 0000000000..cc6249b1dd --- /dev/null +++ b/core/src/test/scala/spark/util/DistributionSuite.scala @@ -0,0 +1,25 @@ +package spark.util + +import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers + +/** + * + */ + +class DistributionSuite extends FunSuite with ShouldMatchers { + test("summary") { + val d = new Distribution((1 to 100).toArray.map{_.toDouble}) + val stats = d.statCounter + stats.count should be (100) + stats.mean should be (50.5) + stats.sum should be (50 * 101) + + val quantiles = d.getQuantiles() + quantiles(0) should be (1) + quantiles(1) should be (26) + quantiles(2) should be (51) + quantiles(3) should be (76) + quantiles(4) should be (100) + } +} -- cgit v1.2.3 From 921be765339ac6a1b1a12672d73620855984eade Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 9 Feb 2013 10:58:47 -0600 Subject: Use stubs instead of mocks for DAGSchedulerSuite. --- core/src/main/scala/spark/MapOutputTracker.scala | 18 +- core/src/main/scala/spark/SparkEnv.scala | 31 +- .../main/scala/spark/storage/BlockManager.scala | 4 +- .../scala/spark/storage/BlockManagerMaster.scala | 24 +- .../main/scala/spark/storage/ThreadingTest.scala | 5 +- .../test/scala/spark/MapOutputTrackerSuite.scala | 62 +- .../scala/spark/scheduler/DAGSchedulerSuite.scala | 633 ++++++--------------- .../scala/spark/storage/BlockManagerSuite.scala | 3 +- 8 files changed, 253 insertions(+), 527 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index 4735207585..866d630a6d 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -38,9 +38,10 @@ private[spark] class MapOutputTrackerActor(tracker: MapOutputTracker) extends Ac } } -private[spark] class MapOutputTracker(actorSystem: ActorSystem, isDriver: Boolean) extends Logging { +private[spark] class MapOutputTracker extends Logging { - val timeout = 10.seconds + // Set to the MapOutputTrackerActor living on the driver + var trackerActor: ActorRef = _ var mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] @@ -53,24 +54,13 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isDriver: Boolea var cacheGeneration = generation val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]] - val actorName: String = "MapOutputTracker" - var trackerActor: ActorRef = if (isDriver) { - val actor = actorSystem.actorOf(Props(new MapOutputTrackerActor(this)), name = actorName) - logInfo("Registered MapOutputTrackerActor actor") - actor - } else { - val ip = System.getProperty("spark.driver.host", "localhost") - val port = System.getProperty("spark.driver.port", "7077").toInt - val url = "akka://spark@%s:%s/user/%s".format(ip, port, actorName) - actorSystem.actorFor(url) - } - val metadataCleaner = new MetadataCleaner("MapOutputTracker", this.cleanup) // Send a message to the trackerActor and get its result within a default timeout, or // throw a SparkException if this fails. def askTracker(message: Any): Any = { try { + val timeout = 10.seconds val future = trackerActor.ask(message)(timeout) return Await.result(future, timeout) } catch { diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index d2193ae72b..7157fd2688 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -1,7 +1,6 @@ package spark -import akka.actor.ActorSystem -import akka.actor.ActorSystemImpl +import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider import serializer.Serializer @@ -83,11 +82,23 @@ object SparkEnv extends Logging { } val serializer = instantiateClass[Serializer]("spark.serializer", "spark.JavaSerializer") + + def registerOrLookup(name: String, newActor: => Actor): ActorRef = { + if (isDriver) { + logInfo("Registering " + name) + actorSystem.actorOf(Props(newActor), name = name) + } else { + val driverIp: String = System.getProperty("spark.driver.host", "localhost") + val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt + val url = "akka://spark@%s:%s/user/%s".format(driverIp, driverPort, name) + logInfo("Connecting to " + name + ": " + url) + actorSystem.actorFor(url) + } + } - val driverIp: String = System.getProperty("spark.driver.host", "localhost") - val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt - val blockManagerMaster = new BlockManagerMaster( - actorSystem, isDriver, isLocal, driverIp, driverPort) + val blockManagerMaster = new BlockManagerMaster(registerOrLookup( + "BlockManagerMaster", + new spark.storage.BlockManagerMasterActor(isLocal))) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer) val connectionManager = blockManager.connectionManager @@ -99,7 +110,12 @@ object SparkEnv extends Logging { val cacheManager = new CacheManager(blockManager) - val mapOutputTracker = new MapOutputTracker(actorSystem, isDriver) + // Have to assign trackerActor after initialization as MapOutputTrackerActor + // requires the MapOutputTracker itself + val mapOutputTracker = new MapOutputTracker() + mapOutputTracker.trackerActor = registerOrLookup( + "MapOutputTracker", + new MapOutputTrackerActor(mapOutputTracker)) val shuffleFetcher = instantiateClass[ShuffleFetcher]( "spark.shuffle.fetcher", "spark.BlockStoreShuffleFetcher") @@ -137,4 +153,5 @@ object SparkEnv extends Logging { httpFileServer, sparkFilesDir) } + } diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9893e9625d..d9838f65ab 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -88,7 +88,7 @@ class BlockManager( val host = System.getProperty("spark.hostname", Utils.localHostName()) - val slaveActor = master.actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), + val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) // Pending reregistration action being executed asynchronously or null if none @@ -946,7 +946,7 @@ class BlockManager( heartBeatTask.cancel() } connectionManager.stop() - master.actorSystem.stop(slaveActor) + actorSystem.stop(slaveActor) blockInfo.clear() memoryStore.clear() diskStore.clear() diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 7389bee150..991ae61712 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -15,32 +15,12 @@ import akka.util.duration._ import spark.{Logging, SparkException, Utils} -private[spark] class BlockManagerMaster( - val actorSystem: ActorSystem, - isDriver: Boolean, - isLocal: Boolean, - driverIp: String, - driverPort: Int) - extends Logging { +private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging { val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt - val DRIVER_AKKA_ACTOR_NAME = "BlockMasterManager" - val timeout = 10.seconds - var driverActor: ActorRef = { - if (isDriver) { - val driverActor = actorSystem.actorOf(Props(new BlockManagerMasterActor(isLocal)), - name = DRIVER_AKKA_ACTOR_NAME) - logInfo("Registered BlockManagerMaster Actor") - driverActor - } else { - val url = "akka://spark@%s:%s/user/%s".format(driverIp, driverPort, DRIVER_AKKA_ACTOR_NAME) - logInfo("Connecting to BlockManagerMaster: " + url) - actorSystem.actorFor(url) - } - } /** Remove a dead executor from the driver actor. This is only called on the driver side. */ def removeExecutor(execId: String) { @@ -59,7 +39,7 @@ private[spark] class BlockManagerMaster( /** Register the BlockManager's id with the driver. */ def registerBlockManager( - blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { logInfo("Trying to register BlockManager") tell(RegisterBlockManager(blockManagerId, maxMemSize, slaveActor)) logInfo("Registered BlockManager") diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala index a70d1c8e78..5c406e68cb 100644 --- a/core/src/main/scala/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/spark/storage/ThreadingTest.scala @@ -75,9 +75,8 @@ private[spark] object ThreadingTest { System.setProperty("spark.kryoserializer.buffer.mb", "1") val actorSystem = ActorSystem("test") val serializer = new KryoSerializer - val driverIp: String = System.getProperty("spark.driver.host", "localhost") - val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt - val blockManagerMaster = new BlockManagerMaster(actorSystem, true, true, driverIp, driverPort) + val blockManagerMaster = new BlockManagerMaster( + actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index dd19442dcb..b49cd81e23 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -31,13 +31,15 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master start and stop") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTracker(actorSystem, true) + val tracker = new MapOutputTracker() + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) tracker.stop() } test("master register and fetch") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTracker(actorSystem, true) + val tracker = new MapOutputTracker() + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -55,7 +57,8 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master register and unregister and fetch") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTracker(actorSystem, true) + val tracker = new MapOutputTracker() + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -77,35 +80,34 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { } test("remote fetch") { - try { - System.clearProperty("spark.driver.host") // In case some previous test had set it - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", "localhost", 0) - System.setProperty("spark.driver.port", boundPort.toString) - val masterTracker = new MapOutputTracker(actorSystem, true) - val slaveTracker = new MapOutputTracker(actorSystem, false) - masterTracker.registerShuffle(10, 1) - masterTracker.incrementGeneration() - slaveTracker.updateGeneration(masterTracker.getGeneration) - intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", "localhost", 0) + val masterTracker = new MapOutputTracker() + masterTracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(masterTracker))) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", "localhost", 0) + val slaveTracker = new MapOutputTracker() + slaveTracker.trackerActor = slaveSystem.actorFor("akka://spark@localhost:" + boundPort) + + masterTracker.registerShuffle(10, 1) + masterTracker.incrementGeneration() + slaveTracker.updateGeneration(masterTracker.getGeneration) + intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val size1000 = MapOutputTracker.decompressSize(compressedSize1000) - masterTracker.registerMapOutput(10, 0, new MapStatus( - BlockManagerId("a", "hostA", 1000), Array(compressedSize1000))) - masterTracker.incrementGeneration() - slaveTracker.updateGeneration(masterTracker.getGeneration) - assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000), size1000))) + val compressedSize1000 = MapOutputTracker.compressSize(1000L) + val size1000 = MapOutputTracker.decompressSize(compressedSize1000) + masterTracker.registerMapOutput(10, 0, new MapStatus( + BlockManagerId("a", "hostA", 1000), Array(compressedSize1000))) + masterTracker.incrementGeneration() + slaveTracker.updateGeneration(masterTracker.getGeneration) + assert(slaveTracker.getServerStatuses(10, 0).toSeq === + Seq((BlockManagerId("a", "hostA", 1000), size1000))) - masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) - masterTracker.incrementGeneration() - slaveTracker.updateGeneration(masterTracker.getGeneration) - intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } + masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) + masterTracker.incrementGeneration() + slaveTracker.updateGeneration(masterTracker.getGeneration) + intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } - // failure should be cached - intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } - } finally { - System.clearProperty("spark.driver.port") - } + // failure should be cached + intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } } } diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 83663ac702..edc5a7dfba 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -4,16 +4,6 @@ import scala.collection.mutable.{Map, HashMap} import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.TimeLimitedTests -import org.scalatest.mock.EasyMockSugar -import org.scalatest.time.{Span, Seconds} - -import org.easymock.EasyMock._ -import org.easymock.Capture -import org.easymock.EasyMock -import org.easymock.{IAnswer, IArgumentMatcher} - -import akka.actor.ActorSystem import spark.storage.BlockManager import spark.storage.BlockManagerId @@ -42,27 +32,26 @@ import spark.{FetchFailed, Success} * DAGScheduler#submitWaitingStages (via test utility functions like runEvent or respondToTaskSet) * and capturing the resulting TaskSets from the mock TaskScheduler. */ -class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar with TimeLimitedTests { +class DAGSchedulerSuite extends FunSuite with BeforeAndAfter { + + val sc: SparkContext = new SparkContext("local", "DAGSchedulerSuite") - // impose a time limit on this test in case we don't let the job finish, in which case - // JobWaiter#getResult will hang. - override val timeLimit = Span(5, Seconds) + /** Set of TaskSets the DAGScheduler has requested executed. */ + val taskSets = scala.collection.mutable.Buffer[TaskSet]() + val taskScheduler = new TaskScheduler() { + override def start() = {} + override def stop() = {} + override def submitTasks(taskSet: TaskSet) = { + // normally done by TaskSetManager + taskSet.tasks.foreach(_.generation = mapOutputTracker.getGeneration) + taskSets += taskSet + } + override def setListener(listener: TaskSchedulerListener) = {} + override def defaultParallelism() = 2 + } - val sc: SparkContext = new SparkContext("local", "DAGSchedulerSuite") - var scheduler: DAGScheduler = null - val taskScheduler = mock[TaskScheduler] - val blockManagerMaster = mock[BlockManagerMaster] var mapOutputTracker: MapOutputTracker = null - var schedulerThread: Thread = null - var schedulerException: Throwable = null - - /** - * Set of EasyMock argument matchers that match a TaskSet for a given RDD. - * We cache these so we do not create duplicate matchers for the same RDD. - * This allows us to easily setup a sequence of expectations for task sets for - * that RDD. - */ - val taskSetMatchers = new HashMap[MyRDD, IArgumentMatcher] + var scheduler: DAGScheduler = null /** * Set of cache locations to return from our mock BlockManagerMaster. @@ -70,68 +59,46 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar * list of cache locations silently. */ val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] - - /** - * JobWaiter for the last JobSubmitted event we pushed. To keep tests (most of which - * will only submit one job) from needing to explicitly track it. - */ - var lastJobWaiter: JobWaiter[Int] = null - - /** - * Array into which we are accumulating the results from the last job asynchronously. - */ - var lastJobResult: Array[Int] = null - - /** - * Tell EasyMockSugar what mock objects we want to be configured by expecting {...} - * and whenExecuting {...} */ - implicit val mocks = MockObjects(taskScheduler, blockManagerMaster) - - /** - * Utility function to reset mocks and set expectations on them. EasyMock wants mock objects - * to be reset after each time their expectations are set, and we tend to check mock object - * calls over a single call to DAGScheduler. - * - * We also set a default expectation here that blockManagerMaster.getLocations can be called - * and will return values from cacheLocations. - */ - def resetExpecting(f: => Unit) { - reset(taskScheduler) - reset(blockManagerMaster) - expecting { - expectGetLocations() - f + // stub out BlockManagerMaster.getLocations to use our cacheLocations + val blockManagerMaster = new BlockManagerMaster(null) { + override def getLocations(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = { + blockIds.map { name => + val pieces = name.split("_") + if (pieces(0) == "rdd") { + val key = pieces(1).toInt -> pieces(2).toInt + cacheLocations.getOrElse(key, Seq()) + } else { + Seq() + } + }.toSeq + } + override def removeExecutor(execId: String) { + // don't need to propagate to the driver, which we don't have + } } + + /** The list of results that DAGScheduler has collected. */ + val results = new HashMap[Int, Any]() + var failure: Exception = _ + val listener = new JobListener() { + override def taskSucceeded(index: Int, result: Any) = results.put(index, result) + override def jobFailed(exception: Exception) = { failure = exception } } before { - taskSetMatchers.clear() + taskSets.clear() cacheLocations.clear() - val actorSystem = ActorSystem("test") - mapOutputTracker = new MapOutputTracker(actorSystem, true) - resetExpecting { - taskScheduler.setListener(anyObject()) - } - whenExecuting { - scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) - } + results.clear() + mapOutputTracker = new MapOutputTracker() + scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) } after { - assert(scheduler.processEvent(StopDAGScheduler)) - resetExpecting { - taskScheduler.stop() - } - whenExecuting { - scheduler.stop() - } + scheduler.stop() sc.stop() System.clearProperty("spark.master.port") } - def makeBlockManagerId(host: String): BlockManagerId = - BlockManagerId("exec-" + host, host, 12345) - /** * Type of RDD we use for testing. Note that we should never call the real RDD compute methods. * This is a pair RDD type so it can always be used in ShuffleDependencies. @@ -143,7 +110,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar * preferredLocations (if any) that are passed to them. They are deliberately not executable * so we can test that DAGScheduler does not try to execute RDDs locally. */ - def makeRdd( + private def makeRdd( numSplits: Int, dependencies: List[Dependency[_]], locations: Seq[Seq[String]] = Nil @@ -164,55 +131,6 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } } - /** - * EasyMock matcher method. For use as an argument matcher for a TaskSet whose first task - * is from a particular RDD. - */ - def taskSetForRdd(rdd: MyRDD): TaskSet = { - val matcher = taskSetMatchers.getOrElseUpdate(rdd, - new IArgumentMatcher { - override def matches(actual: Any): Boolean = { - val taskSet = actual.asInstanceOf[TaskSet] - taskSet.tasks(0) match { - case rt: ResultTask[_, _] => rt.rdd.id == rdd.id - case smt: ShuffleMapTask => smt.rdd.id == rdd.id - case _ => false - } - } - override def appendTo(buf: StringBuffer) { - buf.append("taskSetForRdd(" + rdd + ")") - } - }) - EasyMock.reportMatcher(matcher) - return null - } - - /** - * Setup an EasyMock expectation to repsond to blockManagerMaster.getLocations() called from - * cacheLocations. - */ - def expectGetLocations(): Unit = { - EasyMock.expect(blockManagerMaster.getLocations(anyObject().asInstanceOf[Array[String]])). - andAnswer(new IAnswer[Seq[Seq[BlockManagerId]]] { - override def answer(): Seq[Seq[BlockManagerId]] = { - val blocks = getCurrentArguments()(0).asInstanceOf[Array[String]] - return blocks.map { name => - val pieces = name.split("_") - if (pieces(0) == "rdd") { - val key = pieces(1).toInt -> pieces(2).toInt - if (cacheLocations.contains(key)) { - cacheLocations(key) - } else { - Seq[BlockManagerId]() - } - } else { - Seq[BlockManagerId]() - } - }.toSeq - } - }).anyTimes() - } - /** * Process the supplied event as if it were the top of the DAGScheduler event queue, expecting * the scheduler not to exit. @@ -220,48 +138,21 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar * After processing the event, submit waiting stages as is done on most iterations of the * DAGScheduler event loop. */ - def runEvent(event: DAGSchedulerEvent) { + private def runEvent(event: DAGSchedulerEvent) { assert(!scheduler.processEvent(event)) scheduler.submitWaitingStages() } /** - * Expect a TaskSet for the specified RDD to be submitted to the TaskScheduler. Should be - * called from a resetExpecting { ... } block. - * - * Returns a easymock Capture that will contain the task set after the stage is submitted. - * Most tests should use interceptStage() instead of this directly. - */ - def expectStage(rdd: MyRDD): Capture[TaskSet] = { - val taskSetCapture = new Capture[TaskSet] - taskScheduler.submitTasks(and(capture(taskSetCapture), taskSetForRdd(rdd))) - return taskSetCapture - } - - /** - * Expect the supplied code snippet to submit a stage for the specified RDD. - * Return the resulting TaskSet. First marks all the tasks are belonging to the - * current MapOutputTracker generation. + * When we submit dummy Jobs, this is the compute function we supply. Except in a local test + * below, we do not expect this function to ever be executed; instead, we will return results + * directly through CompletionEvents. */ - def interceptStage(rdd: MyRDD)(f: => Unit): TaskSet = { - var capture: Capture[TaskSet] = null - resetExpecting { - capture = expectStage(rdd) - } - whenExecuting { - f - } - val taskSet = capture.getValue - for (task <- taskSet.tasks) { - task.generation = mapOutputTracker.getGeneration - } - return taskSet - } + private val jobComputeFunc = (context: TaskContext, it: Iterator[(_)]) => + it.next.asInstanceOf[Tuple2[_, _]]._1 - /** - * Send the given CompletionEvent messages for the tasks in the TaskSet. - */ - def respondToTaskSet(taskSet: TaskSet, results: Seq[(TaskEndReason, Any)]) { + /** Send the given CompletionEvent messages for the tasks in the TaskSet. */ + private def complete(taskSet: TaskSet, results: Seq[(TaskEndReason, Any)]) { assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { @@ -269,108 +160,59 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } } } - - /** - * Assert that the supplied TaskSet has exactly the given preferredLocations. - */ - def expectTaskSetLocations(taskSet: TaskSet, locations: Seq[Seq[String]]) { - assert(locations.size === taskSet.tasks.size) - for ((expectLocs, taskLocs) <- - taskSet.tasks.map(_.preferredLocations).zip(locations)) { - assert(expectLocs === taskLocs) - } + + /** Sends the rdd to the scheduler for scheduling. */ + private def submit( + rdd: RDD[_], + partitions: Array[Int], + func: (TaskContext, Iterator[_]) => _ = jobComputeFunc, + allowLocal: Boolean = false, + listener: JobListener = listener) { + runEvent(JobSubmitted(rdd, func, partitions, allowLocal, null, listener)) } - - /** - * When we submit dummy Jobs, this is the compute function we supply. Except in a local test - * below, we do not expect this function to ever be executed; instead, we will return results - * directly through CompletionEvents. - */ - def jobComputeFunc(context: TaskContext, it: Iterator[(Int, Int)]): Int = - it.next._1.asInstanceOf[Int] - - - /** - * Start a job to compute the given RDD. Returns the JobWaiter that will - * collect the result of the job via callbacks from DAGScheduler. - */ - def submitRdd(rdd: MyRDD, allowLocal: Boolean = false): (JobWaiter[Int], Array[Int]) = { - val resultArray = new Array[Int](rdd.splits.size) - val (toSubmit, waiter) = scheduler.prepareJob[(Int, Int), Int]( - rdd, - jobComputeFunc, - (0 to (rdd.splits.size - 1)), - "test-site", - allowLocal, - (i: Int, value: Int) => resultArray(i) = value - ) - lastJobWaiter = waiter - lastJobResult = resultArray - runEvent(toSubmit) - return (waiter, resultArray) - } - - /** - * Assert that a job we started has failed. - */ - def expectJobException(waiter: JobWaiter[Int] = lastJobWaiter) { - waiter.awaitResult() match { - case JobSucceeded => fail() - case JobFailed(_) => return - } - } - - /** - * Assert that a job we started has succeeded and has the given result. - */ - def expectJobResult(expected: Array[Int], waiter: JobWaiter[Int] = lastJobWaiter, - result: Array[Int] = lastJobResult) { - waiter.awaitResult match { - case JobSucceeded => - assert(expected === result) - case JobFailed(_) => - fail() - } + + /** Sends TaskSetFailed to the scheduler. */ + private def failed(taskSet: TaskSet, message: String) { + runEvent(TaskSetFailed(taskSet, message)) } - def makeMapStatus(host: String, reduces: Int): MapStatus = - new MapStatus(makeBlockManagerId(host), Array.fill[Byte](reduces)(2)) - test("zero split job") { val rdd = makeRdd(0, Nil) var numResults = 0 - def accumulateResult(partition: Int, value: Int) { - numResults += 1 + val fakeListener = new JobListener() { + override def taskSucceeded(partition: Int, value: Any) = numResults += 1 + override def jobFailed(exception: Exception) = throw exception } - scheduler.runJob(rdd, jobComputeFunc, Seq(), "test-site", false, accumulateResult) + submit(rdd, Array(), listener = fakeListener) assert(numResults === 0) } test("run trivial job") { val rdd = makeRdd(1, Nil) - val taskSet = interceptStage(rdd) { submitRdd(rdd) } - respondToTaskSet(taskSet, List( (Success, 42) )) - expectJobResult(Array(42)) + submit(rdd, Array(0)) + complete(taskSets(0), List((Success, 42))) + assert(results === Map(0 -> 42)) } test("local job") { val rdd = new MyRDD(sc, Nil) { - override def compute(split: Split, context: TaskContext): Iterator[(Int, Int)] = - Array(42 -> 0).iterator - override def getSplits() = Array( new Split { override def index = 0 } ) + override def compute(split: Split, context: TaskContext) = Array(42 -> 0).iterator + override def getSplits() = Array(new Split { override def index = 0 }) override def getPreferredLocations(split: Split) = Nil override def toString = "DAGSchedulerSuite Local RDD" } - submitRdd(rdd, true) - expectJobResult(Array(42)) + runEvent(JobSubmitted(rdd, jobComputeFunc, Array(0), true, null, listener)) + // this shouldn't be needed, but i haven't stubbed out runLocally yet + Thread.sleep(500) + assert(results === Map(0 -> 42)) } - + test("run trivial job w/ dependency") { val baseRdd = makeRdd(1, Nil) val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) - val taskSet = interceptStage(finalRdd) { submitRdd(finalRdd) } - respondToTaskSet(taskSet, List( (Success, 42) )) - expectJobResult(Array(42)) + submit(finalRdd, Array(0)) + complete(taskSets(0), Seq((Success, 42))) + assert(results === Map(0 -> 42)) } test("cache location preferences w/ dependency") { @@ -378,17 +220,17 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) cacheLocations(baseRdd.id -> 0) = Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")) - val taskSet = interceptStage(finalRdd) { submitRdd(finalRdd) } - expectTaskSetLocations(taskSet, List(Seq("hostA", "hostB"))) - respondToTaskSet(taskSet, List( (Success, 42) )) - expectJobResult(Array(42)) + submit(finalRdd, Array(0)) + val taskSet = taskSets(0) + assertLocations(taskSet, Seq(Seq("hostA", "hostB"))) + complete(taskSet, Seq((Success, 42))) + assert(results === Map(0 -> 42)) } test("trivial job failure") { - val rdd = makeRdd(1, Nil) - val taskSet = interceptStage(rdd) { submitRdd(rdd) } - runEvent(TaskSetFailed(taskSet, "test failure")) - expectJobException() + submit(makeRdd(1, Nil), Array(0)) + failed(taskSets(0), "some failure") + assert(failure.getMessage === "Job failed: some failure") } test("run trivial shuffle") { @@ -396,52 +238,39 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(1, List(shuffleDep)) - - val firstStage = interceptStage(shuffleMapRdd) { submitRdd(reduceRdd) } - val secondStage = interceptStage(reduceRdd) { - respondToTaskSet(firstStage, List( + submit(reduceRdd, Array(0)) + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)) - )) - } + (Success, makeMapStatus("hostB", 1)))) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) - respondToTaskSet(secondStage, List( (Success, 42) )) - expectJobResult(Array(42)) + complete(taskSets(1), Seq((Success, 42))) + assert(results === Map(0 -> 42)) } - + test("run trivial shuffle with fetch failure") { val shuffleMapRdd = makeRdd(2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(2, List(shuffleDep)) - - val firstStage = interceptStage(shuffleMapRdd) { submitRdd(reduceRdd) } - val secondStage = interceptStage(reduceRdd) { - respondToTaskSet(firstStage, List( + submit(reduceRdd, Array(0, 1)) + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)) - )) - } - resetExpecting { - blockManagerMaster.removeExecutor("exec-hostA") - } - whenExecuting { - respondToTaskSet(secondStage, List( + (Success, makeMapStatus("hostB", 1)))) + // the 2nd ResultTask failed + complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), null) - )) - } - val thirdStage = interceptStage(shuffleMapRdd) { - scheduler.resubmitFailedStages() - } - val fourthStage = interceptStage(reduceRdd) { - respondToTaskSet(thirdStage, List( (Success, makeMapStatus("hostA", 1)) )) - } - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === - Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) - respondToTaskSet(fourthStage, List( (Success, 43) )) - expectJobResult(Array(42, 43)) + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), null))) + // this will get called + // blockManagerMaster.removeExecutor("exec-hostA") + // ask the scheduler to try it again + scheduler.resubmitFailedStages() + // have the 2nd attempt pass + complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) + // we can see both result blocks now + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.ip) === Array("hostA", "hostB")) + complete(taskSets(3), Seq((Success, 43))) + assert(results === Map(0 -> 42, 1 -> 43)) } test("ignore late map task completions") { @@ -449,33 +278,27 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(2, List(shuffleDep)) - - val taskSet = interceptStage(shuffleMapRdd) { submitRdd(reduceRdd) } + submit(reduceRdd, Array(0, 1)) + // pretend we were told hostA went away val oldGeneration = mapOutputTracker.getGeneration - resetExpecting { - blockManagerMaster.removeExecutor("exec-hostA") - } - whenExecuting { - runEvent(ExecutorLost("exec-hostA")) - } + runEvent(ExecutorLost("exec-hostA")) val newGeneration = mapOutputTracker.getGeneration assert(newGeneration > oldGeneration) val noAccum = Map[Long, Any]() - // We rely on the event queue being ordered and increasing the generation number by 1 + val taskSet = taskSets(0) // should be ignored for being too old runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum)) // should work because it's a non-failed host runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum)) // should be ignored for being too old runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum)) + // should work because it's a new generation taskSet.tasks(1).generation = newGeneration - val secondStage = interceptStage(reduceRdd) { - runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum)) - } + runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum)) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) - respondToTaskSet(secondStage, List( (Success, 42), (Success, 43) )) - expectJobResult(Array(42, 43)) + complete(taskSets(1), Seq((Success, 42), (Success, 43))) + assert(results === Map(0 -> 42, 1 -> 43)) } test("run trivial shuffle with out-of-band failure and retry") { @@ -483,76 +306,49 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(1, List(shuffleDep)) - - val firstStage = interceptStage(shuffleMapRdd) { submitRdd(reduceRdd) } - resetExpecting { - blockManagerMaster.removeExecutor("exec-hostA") - } - whenExecuting { - runEvent(ExecutorLost("exec-hostA")) - } + submit(reduceRdd, Array(0)) + // blockManagerMaster.removeExecutor("exec-hostA") + // pretend we were told hostA went away + runEvent(ExecutorLost("exec-hostA")) // DAGScheduler will immediately resubmit the stage after it appears to have no pending tasks // rather than marking it is as failed and waiting. - val secondStage = interceptStage(shuffleMapRdd) { - respondToTaskSet(firstStage, List( + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)) - )) - } - val thirdStage = interceptStage(reduceRdd) { - respondToTaskSet(secondStage, List( - (Success, makeMapStatus("hostC", 1)) - )) - } - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === - Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) - respondToTaskSet(thirdStage, List( (Success, 42) )) - expectJobResult(Array(42)) - } - - test("recursive shuffle failures") { + (Success, makeMapStatus("hostB", 1)))) + // have hostC complete the resubmitted task + complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) + complete(taskSets(2), Seq((Success, 42))) + assert(results === Map(0 -> 42)) + } + + test("recursive shuffle failures") { val shuffleOneRdd = makeRdd(2, Nil) val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) val finalRdd = makeRdd(1, List(shuffleDepTwo)) - - val firstStage = interceptStage(shuffleOneRdd) { submitRdd(finalRdd) } - val secondStage = interceptStage(shuffleTwoRdd) { - respondToTaskSet(firstStage, List( + submit(finalRdd, Array(0)) + // have the first stage complete normally + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)) - )) - } - val thirdStage = interceptStage(finalRdd) { - respondToTaskSet(secondStage, List( + (Success, makeMapStatus("hostB", 2)))) + // have the second stage complete normally + complete(taskSets(1), Seq( (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostC", 1)) - )) - } - resetExpecting { - blockManagerMaster.removeExecutor("exec-hostA") - } - whenExecuting { - respondToTaskSet(thirdStage, List( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null) - )) - } - val recomputeOne = interceptStage(shuffleOneRdd) { - scheduler.resubmitFailedStages() - } - val recomputeTwo = interceptStage(shuffleTwoRdd) { - respondToTaskSet(recomputeOne, List( - (Success, makeMapStatus("hostA", 2)) - )) - } - val finalStage = interceptStage(finalRdd) { - respondToTaskSet(recomputeTwo, List( - (Success, makeMapStatus("hostA", 1)) - )) - } - respondToTaskSet(finalStage, List( (Success, 42) )) - expectJobResult(Array(42)) + (Success, makeMapStatus("hostC", 1)))) + // fail the third stage because hostA went down + complete(taskSets(2), Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null))) + // TODO assert this: + // blockManagerMaster.removeExecutor("exec-hostA") + // have DAGScheduler try again + scheduler.resubmitFailedStages() + complete(taskSets(3), Seq((Success, makeMapStatus("hostA", 2)))) + complete(taskSets(4), Seq((Success, makeMapStatus("hostA", 1)))) + complete(taskSets(5), Seq((Success, 42))) + assert(results === Map(0 -> 42)) } test("cached post-shuffle") { @@ -561,103 +357,44 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) val finalRdd = makeRdd(1, List(shuffleDepTwo)) - - val firstShuffleStage = interceptStage(shuffleOneRdd) { submitRdd(finalRdd) } + submit(finalRdd, Array(0)) cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) - val secondShuffleStage = interceptStage(shuffleTwoRdd) { - respondToTaskSet(firstShuffleStage, List( + // complete stage 2 + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)) - )) - } - val reduceStage = interceptStage(finalRdd) { - respondToTaskSet(secondShuffleStage, List( + (Success, makeMapStatus("hostB", 2)))) + // complete stage 1 + complete(taskSets(1), Seq( (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)) - )) - } - resetExpecting { - blockManagerMaster.removeExecutor("exec-hostA") - } - whenExecuting { - respondToTaskSet(reduceStage, List( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null) - )) - } + (Success, makeMapStatus("hostB", 1)))) + // pretend stage 0 failed because hostA went down + complete(taskSets(2), Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null))) + // TODO assert this: + // blockManagerMaster.removeExecutor("exec-hostA") // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun. - val recomputeTwo = interceptStage(shuffleTwoRdd) { - scheduler.resubmitFailedStages() - } - expectTaskSetLocations(recomputeTwo, Seq(Seq("hostD"))) - val finalRetry = interceptStage(finalRdd) { - respondToTaskSet(recomputeTwo, List( - (Success, makeMapStatus("hostD", 1)) - )) - } - respondToTaskSet(finalRetry, List( (Success, 42) )) - expectJobResult(Array(42)) + scheduler.resubmitFailedStages() + assertLocations(taskSets(3), Seq(Seq("hostD"))) + // allow hostD to recover + complete(taskSets(3), Seq((Success, makeMapStatus("hostD", 1)))) + complete(taskSets(4), Seq((Success, 42))) + assert(results === Map(0 -> 42)) } - test("cached post-shuffle but fails") { - val shuffleOneRdd = makeRdd(2, Nil) - val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) - val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) - val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) - val finalRdd = makeRdd(1, List(shuffleDepTwo)) - - val firstShuffleStage = interceptStage(shuffleOneRdd) { submitRdd(finalRdd) } - cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) - cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) - val secondShuffleStage = interceptStage(shuffleTwoRdd) { - respondToTaskSet(firstShuffleStage, List( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)) - )) - } - val reduceStage = interceptStage(finalRdd) { - respondToTaskSet(secondShuffleStage, List( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)) - )) - } - resetExpecting { - blockManagerMaster.removeExecutor("exec-hostA") - } - whenExecuting { - respondToTaskSet(reduceStage, List( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null) - )) - } - val recomputeTwoCached = interceptStage(shuffleTwoRdd) { - scheduler.resubmitFailedStages() - } - expectTaskSetLocations(recomputeTwoCached, Seq(Seq("hostD"))) - intercept[FetchFailedException]{ - mapOutputTracker.getServerStatuses(shuffleDepOne.shuffleId, 0) + /** Assert that the supplied TaskSet has exactly the given preferredLocations. */ + private def assertLocations(taskSet: TaskSet, locations: Seq[Seq[String]]) { + assert(locations.size === taskSet.tasks.size) + for ((expectLocs, taskLocs) <- + taskSet.tasks.map(_.preferredLocations).zip(locations)) { + assert(expectLocs === taskLocs) } + } - // Simulate the shuffle input data failing to be cached. - cacheLocations.remove(shuffleTwoRdd.id -> 0) - respondToTaskSet(recomputeTwoCached, List( - (FetchFailed(null, shuffleDepOne.shuffleId, 0, 0), null) - )) + private def makeMapStatus(host: String, reduces: Int): MapStatus = + new MapStatus(makeBlockManagerId(host), Array.fill[Byte](reduces)(2)) - // After the fetch failure, DAGScheduler should recheck the cache and decide to resubmit - // everything. - val recomputeOne = interceptStage(shuffleOneRdd) { - scheduler.resubmitFailedStages() - } - // We use hostA here to make sure DAGScheduler doesn't think it's still dead. - val recomputeTwoUncached = interceptStage(shuffleTwoRdd) { - respondToTaskSet(recomputeOne, List( (Success, makeMapStatus("hostA", 1)) )) - } - expectTaskSetLocations(recomputeTwoUncached, Seq(Seq[String]())) - val finalRetry = interceptStage(finalRdd) { - respondToTaskSet(recomputeTwoUncached, List( (Success, makeMapStatus("hostA", 1)) )) + private def makeBlockManagerId(host: String): BlockManagerId = + BlockManagerId("exec-" + host, host, 12345) - } - respondToTaskSet(finalRetry, List( (Success, 42) )) - expectJobResult(Array(42)) - } -} +} \ No newline at end of file diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index 2d177bbf67..53450609a1 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -31,7 +31,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT before { actorSystem = ActorSystem("test") - master = new BlockManagerMaster(actorSystem, true, true, "localhost", 7077) + master = new BlockManagerMaster( + actorSystem.actorOf(Props(new spark.storage.BlockManagerMasterActor(true)))) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case oldArch = System.setProperty("os.arch", "amd64") -- cgit v1.2.3 From b7d9e2439445da9b1ca8709f4ad8fcac9927dd76 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 10 Feb 2013 14:18:52 -0800 Subject: use TaskMetrics to gather all stats; lots of plumbing to get it all the way back to driver --- core/src/main/scala/spark/executor/TaskMetrics.scala | 9 ++++++--- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 3 +++ .../main/scala/spark/scheduler/DAGScheduler.scala | 15 +++++++-------- .../scala/spark/scheduler/DAGSchedulerEvent.scala | 4 +++- .../main/scala/spark/scheduler/ShuffleMapTask.scala | 5 +---- core/src/main/scala/spark/scheduler/StageInfo.scala | 20 ++++++++++++++++---- core/src/main/scala/spark/scheduler/Task.scala | 6 ++++++ core/src/main/scala/spark/scheduler/TaskResult.scala | 4 +++- .../spark/scheduler/TaskSchedulerListener.scala | 6 ++++-- .../scala/spark/scheduler/cluster/TaskInfo.scala | 2 ++ .../spark/scheduler/cluster/TaskSetManager.scala | 6 +++--- .../scala/spark/scheduler/local/LocalScheduler.scala | 4 ++-- .../scala/spark/scheduler/DAGSchedulerSuite.scala | 10 +++++----- 13 files changed, 61 insertions(+), 33 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 6cdc6d0b7e..b005b9ff27 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -7,10 +7,13 @@ case class TaskMetrics( val remoteBlocksFetched: Option[Int], val localBlocksFetched: Option[Int], val remoteFetchWaitTime: Option[Long], - val remoteBytesRead: Option[Long] + val remoteBytesRead: Option[Long], + val shuffleBytesWritten: Option[Long] ) object TaskMetrics { - private[spark] def apply(task: Task[_]) : TaskMetrics = - TaskMetrics(None, None, None, task.remoteFetchWaitTime, task.remoteReadBytes) + private[spark] def apply(task: Task[_]) : TaskMetrics = { + TaskMetrics(task.totalBlocksFetched, task.remoteBlocksFetched, task.localBlocksFetched, + task.remoteFetchWaitTime, task.remoteReadBytes, task.shuffleBytesWritten) + } } \ No newline at end of file diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 90b822ec85..4aa97f6ad9 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -116,6 +116,9 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) context.task.remoteFetchTime = Some(fetchItr.remoteFetchTime) context.task.remoteFetchWaitTime = Some(fetchItr.remoteFetchWaitTime) context.task.remoteReadBytes = Some(fetchItr.remoteBytesRead) + context.task.totalBlocksFetched = Some(fetchItr.totalBlocks) + context.task.localBlocksFetched = Some(fetchItr.numLocalBlocks) + context.task.remoteBlocksFetched = Some(fetchItr.numRemoteBlocks) } } JavaConversions.mapAsScalaMap(map).iterator diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index f07ee50670..c0651e69d5 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -7,9 +7,10 @@ import java.util.concurrent.Future import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.TimeUnit -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue, Map} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import spark._ +import executor.TaskMetrics import spark.partial.ApproximateActionListener import spark.partial.ApproximateEvaluator import spark.partial.PartialResult @@ -42,8 +43,9 @@ class DAGScheduler( reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], - taskInfo: TaskInfo) { - eventQueue.put(CompletionEvent(task, reason, result, accumUpdates, taskInfo)) + taskInfo: TaskInfo, + taskMetrics: TaskMetrics) { + eventQueue.put(CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics)) } // Called by TaskScheduler when an executor fails. @@ -77,7 +79,7 @@ class DAGScheduler( private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] - private val sparkListeners = Traversable[SparkListener]() + private[spark] var sparkListeners = ArrayBuffer[SparkListener]() var cacheLocs = new HashMap[Int, Array[List[String]]] @@ -491,6 +493,7 @@ class DAGScheduler( } pendingTasks(stage) -= task stageToInfos(stage).taskInfos += event.taskInfo + stageToInfos(stage).taskMetrics += event.taskMetrics task match { case rt: ResultTask[_, _] => resultStageToJob.get(stage) match { @@ -512,10 +515,6 @@ class DAGScheduler( case smt: ShuffleMapTask => val status = event.result.asInstanceOf[MapStatus] - smt.totalBytesWritten match { - case Some(b) => stageToInfos(stage).shuffleBytesWritten += b - case None => throw new RuntimeException("shuffle stask completed without tracking bytes written") - } val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) if (failedGeneration.contains(execId) && smt.generation <= failedGeneration(execId)) { diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 9f19e137e4..2a5cad7006 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -4,6 +4,7 @@ import cluster.TaskInfo import scala.collection.mutable.Map import spark._ +import executor.TaskMetrics /** * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue @@ -27,7 +28,8 @@ private[spark] case class CompletionEvent( reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], - taskInfo: TaskInfo) + taskInfo: TaskInfo, + taskMetrics: TaskMetrics) extends DAGSchedulerEvent private[spark] case class ExecutorLost(execId: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index e97dd0ad3a..744f046b0e 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -81,9 +81,6 @@ private[spark] class ShuffleMapTask( with Externalizable with Logging { - - var totalBytesWritten : Option[Long] = None - protected def this() = this(0, null, null, 0, null) var split = if (rdd == null) { @@ -144,7 +141,7 @@ private[spark] class ShuffleMapTask( totalBytes += size compressedSizes(i) = MapOutputTracker.compressSize(size) } - totalBytesWritten = Some(totalBytes) + shuffleBytesWritten = Some(totalBytes) return new MapStatus(blockManager.blockManagerId, compressedSizes) } finally { diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 489b9744de..c70b346e40 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -3,17 +3,29 @@ package spark.scheduler import cluster.TaskInfo import collection._ import spark.util.Distribution +import spark.executor.TaskMetrics case class StageInfo( val stage: Stage, val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo](), - val shuffleBytesWritten : mutable.Buffer[Long] = mutable.Buffer[Long](), - val shuffleBytesRead : mutable.Buffer[Long] = mutable.Buffer[Long]() + val taskMetrics: mutable.Buffer[TaskMetrics] = mutable.Buffer[TaskMetrics]() ) { - def name = stage.rdd.name + "(" + stage.origin + ")" + override def toString = stage.rdd.toString def getTaskRuntimeDistribution = { - new Distribution(taskInfos.map{_.duration.toDouble}) + Distribution(taskInfos.map{_.duration.toDouble}) + } + + def getShuffleBytesWrittenDistribution = { + Distribution(taskMetrics.flatMap{_.shuffleBytesWritten.map{_.toDouble}}) + } + + def getRemoteFetchWaitTimeDistribution = { + Distribution(taskMetrics.flatMap{_.remoteFetchWaitTime.map{_.toDouble}}) + } + + def getRemoteBytesReadDistribution = { + Distribution(taskMetrics.flatMap{_.remoteBytesRead.map{_.toDouble}}) } } diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala index 9c1d5869be..84fc3bc988 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/spark/scheduler/Task.scala @@ -21,6 +21,12 @@ private[spark] abstract class Task[T](val stageId: Int) extends Serializable { var remoteReadBytes : Option[Long] = None var remoteFetchWaitTime : Option[Long] = None var remoteFetchTime : Option[Long] = None + var totalBlocksFetched : Option[Int] = None + var remoteBlocksFetched: Option[Int] = None + var localBlocksFetched: Option[Int] = None + + var shuffleBytesWritten : Option[Long] = None + } /** diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala index 003ee2bb2e..6de0aa7adf 100644 --- a/core/src/main/scala/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/spark/scheduler/TaskResult.scala @@ -9,7 +9,7 @@ import spark.executor.TaskMetrics // TODO: Use of distributed cache to return result is a hack to get around // what seems to be a bug with messages over 60KB in libprocess; fix it private[spark] -class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], val metrics: TaskMetrics) extends Externalizable { +class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) extends Externalizable { def this() = this(null.asInstanceOf[T], null, null) override def writeExternal(out: ObjectOutput) { @@ -19,6 +19,7 @@ class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], val metrics: out.writeLong(key) out.writeObject(value) } + out.writeObject(metrics) } override def readExternal(in: ObjectInput) { @@ -32,5 +33,6 @@ class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], val metrics: accumUpdates(in.readLong()) = in.readObject() } } + metrics = in.readObject().asInstanceOf[TaskMetrics] } } diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index 4ede03cc2a..771518dddf 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -1,16 +1,18 @@ package spark.scheduler -import cluster.TaskInfo +import spark.scheduler.cluster.TaskInfo import scala.collection.mutable.Map import spark.TaskEndReason +import spark.executor.TaskMetrics /** * Interface for getting events back from the TaskScheduler. */ private[spark] trait TaskSchedulerListener { // A task has finished or failed. - def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], taskInfo: TaskInfo): Unit + def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], + taskInfo: TaskInfo, taskMetrics: TaskMetrics): Unit // A node was lost from the cluster. def executorLost(execId: String): Unit diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index 0f975ce1eb..a7e14094fb 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -1,5 +1,7 @@ package spark.scheduler.cluster +import spark.executor.TaskMetrics + /** * Information about a running task attempt inside a TaskSet. */ diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 25aa6ab94a..9ed098e501 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -259,7 +259,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe tid, info.duration, tasksFinished, numTasks)) // Deserialize task result and pass it to the scheduler val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info) + sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) // Mark finished and stop if we've finished all the tasks finished(index) = true if (tasksFinished == numTasks) { @@ -290,7 +290,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe reason match { case fetchFailed: FetchFailed => logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress) - sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info) + sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null) finished(index) = true tasksFinished += 1 sched.taskSetFinished(this) @@ -378,7 +378,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe addPendingTask(index) // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our // stage finishes when a total of tasks.size tasks finish. - sched.listener.taskEnded(tasks(index), Resubmitted, null, null, info) + sched.listener.taskEnded(tasks(index), Resubmitted, null, null, info, null) } } } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index e0075e9e8b..1221f2957c 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -87,7 +87,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon // If the threadpool has not already been shutdown, notify DAGScheduler if (!Thread.currentThread().isInterrupted) - listener.taskEnded(task, Success, resultToReturn, accumUpdates, info) + listener.taskEnded(task, Success, resultToReturn, accumUpdates, info, null) } catch { case t: Throwable => { logError("Exception in task " + idInJob, t) @@ -98,7 +98,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon } else { // TODO: Do something nicer here to return all the way to the user if (!Thread.currentThread().isInterrupted) - listener.taskEnded(task, new ExceptionFailure(t), null, null, info) + listener.taskEnded(task, new ExceptionFailure(t), null, null, info, null) } } } diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index f74b425dd5..dc2c51ed4a 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -265,7 +265,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { - runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, Map[Long, Any](), null)) + runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, Map[Long, Any](), null, null)) } } } @@ -463,14 +463,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val noAccum = Map[Long, Any]() // We rely on the event queue being ordered and increasing the generation number by 1 // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null, null)) // should work because it's a non-failed host - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum, null, null)) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null, null)) taskSet.tasks(1).generation = newGeneration val secondStage = interceptStage(reduceRdd) { - runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum, null)) + runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum, null, null)) } assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) -- cgit v1.2.3 From 383af599bb9e4935994be811415d1ee84f95ce56 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 10 Feb 2013 14:19:37 -0800 Subject: SparkContext.addSparkListener; "std" listener in StatsReportListener --- core/src/main/scala/spark/SparkContext.scala | 4 +++ .../main/scala/spark/scheduler/SparkListener.scala | 35 ++++++++++++++++++++++ 2 files changed, 39 insertions(+) (limited to 'core') diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 00b0b8270e..6ac2f7c7e9 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -469,6 +469,10 @@ class SparkContext( logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) } + def addSparkListener(listener: SparkListener) { + dagScheduler.sparkListeners += listener + } + /** * Return a map from the slave to the max memory available for caching and the remaining * memory available for caching. diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 54afc714b3..92d9c0330a 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -1,5 +1,8 @@ package spark.scheduler +import spark.util.Distribution +import spark.{Utils, Logging} + trait SparkListener { def onStageCompleted(stageCompleted: StageCompleted) } @@ -7,3 +10,35 @@ trait SparkListener { sealed trait SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents + + + +class StatsReportListener extends SparkListener with Logging { + def onStageCompleted(stageCompleted: StageCompleted) { + println("Finished stage: " + stageCompleted.stageInfo) + showDistribution("task runtime:", stageCompleted.stageInfo.getTaskRuntimeDistribution, "%4.0f") + showDistribution("shuffle bytes written:", stageCompleted.stageInfo.getShuffleBytesWrittenDistribution, d => Utils.memoryBytesToString(d.toLong)) + showDistribution("fetch wait time:",stageCompleted.stageInfo.getRemoteFetchWaitTimeDistribution, "%4.0f") + showDistribution("remote bytes read:", stageCompleted.stageInfo.getRemoteBytesReadDistribution, d => Utils.memoryBytesToString(d.toLong)) + } + + //for profiling, the extremes are more interesting + val percentiles = Array[Int](0,5,10,25,50,75,90,95,100) + val probabilities = percentiles.map{_ / 100.0} + val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" + + def showDistribution(heading: String, dOpt: Option[Distribution], format:String) { + def f(d:Double) = format.format(d) + showDistribution(heading, dOpt, f _) + } + + def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) { + dOpt.foreach { d => + val stats = d.statCounter + logInfo(heading + stats) + val quantiles = d.getQuantiles(probabilities).map{formatNumber} + logInfo(percentilesHeader) + logInfo("\t" + quantiles.mkString("\t")) + } + } +} \ No newline at end of file -- cgit v1.2.3 From d9461b15d35ed76ffaff23a4c07a84981428602a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 10 Feb 2013 21:41:40 -0800 Subject: cleanup a bunch of imports --- .../scala/spark/BlockStoreShuffleFetcher.scala | 4 ++-- core/src/main/scala/spark/ShuffleFetcher.scala | 4 ++-- core/src/main/scala/spark/SparkContext.scala | 26 ++++++++-------------- core/src/main/scala/spark/TaskContext.scala | 2 +- .../main/scala/spark/scheduler/DAGScheduler.scala | 7 ++---- .../scala/spark/scheduler/DAGSchedulerEvent.scala | 4 ++-- .../src/main/scala/spark/scheduler/StageInfo.scala | 4 ++-- .../spark/scheduler/local/LocalScheduler.scala | 6 ++--- 8 files changed, 22 insertions(+), 35 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index bb4e3de63c..25532dae28 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -3,8 +3,8 @@ package spark import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import storage.{DelegateBlockFetchTracker, BlockManagerId} -import util.TimedIterator +import spark.storage.{DelegateBlockFetchTracker, BlockManagerId} +import spark.util.TimedIterator private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { override def fetch[K, V](shuffleId: Int, reduceId: Int) = { diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index d64b2611b7..828cd943f4 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,7 +1,7 @@ package spark -import storage.BlockFetchTracker -import util.TimedIterator +import spark.storage.BlockFetchTracker +import spark.util.TimedIterator private[spark] abstract class ShuffleFetcher { /** diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 6ac2f7c7e9..e8da703d53 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -1,19 +1,15 @@ package spark import java.io._ -import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicInteger -import java.net.{URI, URLClassLoader} -import java.lang.ref.WeakReference +import java.net.URI import scala.collection.Map import scala.collection.generic.Growable -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ -import akka.actor.Actor -import akka.actor.Actor._ -import org.apache.hadoop.fs.{FileUtil, Path} +import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.SequenceFileInputFormat @@ -33,23 +29,19 @@ import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.hadoop.mapreduce.{Job => NewHadoopJob} -import org.apache.mesos.{Scheduler, MesosNativeLibrary} +import org.apache.mesos.MesosNativeLibrary -import spark.broadcast._ import spark.deploy.LocalSparkCluster import spark.partial.ApproximateEvaluator import spark.partial.PartialResult -import rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD} -import scheduler._ +import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD} +import spark.scheduler._ import spark.scheduler.local.LocalScheduler import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import storage.BlockManagerUI -import storage.RDDInfo -import storage.StorageStatus -import util.{MetadataCleaner, TimeStampedHashMap} -import storage.{StorageStatus, StorageUtils, RDDInfo} -import scala.Some +import spark.storage.BlockManagerUI +import spark.util.{MetadataCleaner, TimeStampedHashMap} +import spark.storage.{StorageStatus, StorageUtils, RDDInfo} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index 11819db3a3..d62229f0ce 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -1,7 +1,7 @@ package spark import scala.collection.mutable.ArrayBuffer -import scheduler.Task +import spark.scheduler.Task class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long, val task: Task[_]) extends Serializable { //by adding Task here, I'm destroying the separation between Task & TaskContext ... not sure why they need to diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index c0651e69d5..608962b5f1 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -1,22 +1,19 @@ package spark.scheduler import cluster.TaskInfo -import java.net.URI import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.Future import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.TimeUnit import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import spark._ -import executor.TaskMetrics +import spark.executor.TaskMetrics import spark.partial.ApproximateActionListener import spark.partial.ApproximateEvaluator import spark.partial.PartialResult import spark.storage.BlockManagerMaster -import spark.storage.BlockManagerId -import util.{MetadataCleaner, TimeStampedHashMap} +import spark.util.{MetadataCleaner, TimeStampedHashMap} /** * A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 2a5cad7006..ed0b9bf178 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -1,10 +1,10 @@ package spark.scheduler -import cluster.TaskInfo +import spark.scheduler.cluster.TaskInfo import scala.collection.mutable.Map import spark._ -import executor.TaskMetrics +import spark.executor.TaskMetrics /** * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index c70b346e40..df3d1ec4dc 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -1,7 +1,7 @@ package spark.scheduler -import cluster.TaskInfo -import collection._ +import spark.scheduler.cluster.TaskInfo +import scala.collection._ import spark.util.Distribution import spark.executor.TaskMetrics diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 1221f2957c..6b91728b74 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -1,15 +1,13 @@ package spark.scheduler.local import java.io.File -import java.net.URLClassLoader -import java.util.concurrent.Executors import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.HashMap import spark._ -import executor.ExecutorURLClassLoader +import spark.executor.ExecutorURLClassLoader import spark.scheduler._ -import cluster.TaskInfo +import spark.scheduler.cluster.TaskInfo /** * A simple TaskScheduler implementation that runs tasks locally in a thread pool. Optionally -- cgit v1.2.3 From e9f53ec0eadadfc6faa2bbf6ead4216ca3417073 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 11 Feb 2013 09:36:49 -0800 Subject: undo chnage to onCompleteCallbacks --- core/src/main/scala/spark/TaskContext.scala | 6 +++--- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 2 +- core/src/main/scala/spark/rdd/HadoopRDD.scala | 2 +- core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index d62229f0ce..6b67b17b8a 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -7,15 +7,15 @@ class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long, val t //by adding Task here, I'm destroying the separation between Task & TaskContext ... not sure why they need to // be separate - @transient val onCompleteCallbacks = new ArrayBuffer[TaskContext => Unit] + @transient val onCompleteCallbacks = new ArrayBuffer[() => Unit] // Add a callback function to be executed on task completion. An example use // is for HadoopRDD to register a callback to close the input stream. - def addOnCompleteCallback(f: TaskContext => Unit) { + def addOnCompleteCallback(f: () => Unit) { onCompleteCallbacks += f } def executeOnCompleteCallbacks() { - onCompleteCallbacks.foreach{_.apply(this)} + onCompleteCallbacks.foreach{_()} } } diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index ad08558590..96b593ba7c 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -104,7 +104,7 @@ private[spark] object CheckpointRDD extends Logging { val deserializeStream = serializer.deserializeStream(fileInputStream) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(_ => deserializeStream.close()) + context.addOnCompleteCallback(() => deserializeStream.close()) deserializeStream.asIterator.asInstanceOf[Iterator[T]] } diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index 3f0b4ce23e..f547f53812 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -74,7 +74,7 @@ class HadoopRDD[K, V]( reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(_ => reader.close()) + context.addOnCompleteCallback(() => reader.close()) val key: K = reader.createKey() val value: V = reader.createValue() diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index c80d30e125..c3b155fcbd 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -63,7 +63,7 @@ class NewHadoopRDD[K, V]( reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(_ => reader.close()) + context.addOnCompleteCallback(() => reader.close()) var havePair = false var finished = false -- cgit v1.2.3 From baab23abdfc94af32c35d5fc2035382d3faa0ec4 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 Feb 2013 14:13:01 -0800 Subject: TaskContext does not hold a reference to Task; instead, it has a shared instance of TaskMetrics with Task --- core/src/main/scala/spark/TaskContext.scala | 9 +++++++-- core/src/main/scala/spark/executor/Executor.scala | 2 +- .../src/main/scala/spark/executor/TaskMetrics.scala | 21 +++++++++------------ core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 14 +++++++------- .../main/scala/spark/scheduler/DAGScheduler.scala | 2 +- .../src/main/scala/spark/scheduler/ResultTask.scala | 3 ++- .../main/scala/spark/scheduler/ShuffleMapTask.scala | 5 +++-- core/src/main/scala/spark/scheduler/Task.scala | 14 +++----------- 8 files changed, 33 insertions(+), 37 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index 6b67b17b8a..5aadd5e261 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -1,9 +1,14 @@ package spark +import executor.TaskMetrics import scala.collection.mutable.ArrayBuffer -import spark.scheduler.Task -class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long, val task: Task[_]) extends Serializable { +class TaskContext( + val stageId: Int, + val splitId: Int, + val attemptId: Long, + val taskMetrics: TaskMetrics = TaskMetrics.empty() +) extends Serializable { //by adding Task here, I'm destroying the separation between Task & TaskContext ... not sure why they need to // be separate diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 0906702a3d..7f2c16b43a 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -95,7 +95,7 @@ private[spark] class Executor extends Logging { env.mapOutputTracker.updateGeneration(task.generation) val value = task.run(taskId.toInt) val accumUpdates = Accumulators.values - val result = new TaskResult(value, accumUpdates, TaskMetrics(task)) + val result = new TaskResult(value, accumUpdates, task.metrics.getOrElse(null)) val serializedResult = ser.serialize(result) logInfo("Serialized size of result for " + taskId + " is " + serializedResult.limit) context.statusUpdate(taskId, TaskState.FINISHED, serializedResult) diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index b005b9ff27..525abf8893 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -1,19 +1,16 @@ package spark.executor -import spark.scheduler.Task - case class TaskMetrics( - val totalBlocksFetched : Option[Int], - val remoteBlocksFetched: Option[Int], - val localBlocksFetched: Option[Int], - val remoteFetchWaitTime: Option[Long], - val remoteBytesRead: Option[Long], - val shuffleBytesWritten: Option[Long] + var shuffleReadMillis: Option[Long], + var totalBlocksFetched : Option[Int], + var remoteBlocksFetched: Option[Int], + var localBlocksFetched: Option[Int], + var remoteFetchWaitTime: Option[Long], + var remoteFetchTime: Option[Long], + var remoteBytesRead: Option[Long], + var shuffleBytesWritten: Option[Long] ) object TaskMetrics { - private[spark] def apply(task: Task[_]) : TaskMetrics = { - TaskMetrics(task.totalBlocksFetched, task.remoteBlocksFetched, task.localBlocksFetched, - task.remoteFetchWaitTime, task.remoteReadBytes, task.shuffleBytesWritten) - } + private[spark] def empty() : TaskMetrics = TaskMetrics(None,None,None,None,None,None,None,None) } \ No newline at end of file diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index a32a83ccae..66e3cb4c32 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -106,13 +106,13 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) for ((k, vs) <- fetchItr) { getSeq(k)(depNum) ++= vs } - context.task.shuffleReadMillis = Some(fetchItr.getNetMillis) - context.task.remoteFetchTime = Some(fetchItr.remoteFetchTime) - context.task.remoteFetchWaitTime = Some(fetchItr.remoteFetchWaitTime) - context.task.remoteReadBytes = Some(fetchItr.remoteBytesRead) - context.task.totalBlocksFetched = Some(fetchItr.totalBlocks) - context.task.localBlocksFetched = Some(fetchItr.numLocalBlocks) - context.task.remoteBlocksFetched = Some(fetchItr.numRemoteBlocks) + context.taskMetrics.shuffleReadMillis = Some(fetchItr.getNetMillis) + context.taskMetrics.remoteFetchTime = Some(fetchItr.remoteFetchTime) + context.taskMetrics.remoteFetchWaitTime = Some(fetchItr.remoteFetchWaitTime) + context.taskMetrics.remoteBytesRead = Some(fetchItr.remoteBytesRead) + context.taskMetrics.totalBlocksFetched = Some(fetchItr.totalBlocks) + context.taskMetrics.localBlocksFetched = Some(fetchItr.numLocalBlocks) + context.taskMetrics.remoteBlocksFetched = Some(fetchItr.numRemoteBlocks) } } JavaConversions.mapAsScalaMap(map).iterator diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 608962b5f1..04128f89fd 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -393,7 +393,7 @@ class DAGScheduler( SparkEnv.set(env) val rdd = job.finalStage.rdd val split = rdd.splits(job.partitions(0)) - val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0, null) //TODO figure out what to do about Task here + val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0) try { val result = job.func(taskContext, rdd.iterator(split, taskContext)) job.listener.taskSucceeded(0, result) diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index e1b26a0cf5..585e4a2899 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -71,7 +71,8 @@ private[spark] class ResultTask[T, U]( } override def run(attemptId: Long): U = { - val context = new TaskContext(stageId, partition, attemptId, this) + val context = new TaskContext(stageId, partition, attemptId) + metrics = Some(context.taskMetrics) try { func(context, rdd.iterator(split, context)) } finally { diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 744f046b0e..486dba0bb0 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -118,7 +118,8 @@ private[spark] class ShuffleMapTask( override def run(attemptId: Long): MapStatus = { val numOutputSplits = dep.partitioner.numPartitions - val taskContext = new TaskContext(stageId, partition, attemptId, this) + val taskContext = new TaskContext(stageId, partition, attemptId) + metrics = Some(taskContext.taskMetrics) try { // Partition the map output. val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)]) @@ -141,7 +142,7 @@ private[spark] class ShuffleMapTask( totalBytes += size compressedSizes(i) = MapOutputTracker.compressSize(size) } - shuffleBytesWritten = Some(totalBytes) + metrics.get.shuffleBytesWritten = Some(totalBytes) return new MapStatus(blockManager.blockManagerId, compressedSizes) } finally { diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala index 84fc3bc988..a6462c6968 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/spark/scheduler/Task.scala @@ -1,12 +1,12 @@ package spark.scheduler -import scala.collection.mutable.HashMap -import spark.serializer.{SerializerInstance, Serializer} +import spark.serializer.SerializerInstance import java.io.{DataInputStream, DataOutputStream} import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import spark.util.ByteBufferInputStream import scala.collection.mutable.HashMap +import spark.executor.TaskMetrics /** * A task to execute on a worker node. @@ -17,15 +17,7 @@ private[spark] abstract class Task[T](val stageId: Int) extends Serializable { var generation: Long = -1 // Map output tracker generation. Will be set by TaskScheduler. - var shuffleReadMillis : Option[Long] = None - var remoteReadBytes : Option[Long] = None - var remoteFetchWaitTime : Option[Long] = None - var remoteFetchTime : Option[Long] = None - var totalBlocksFetched : Option[Int] = None - var remoteBlocksFetched: Option[Int] = None - var localBlocksFetched: Option[Int] = None - - var shuffleBytesWritten : Option[Long] = None + var metrics: Option[TaskMetrics] = None } -- cgit v1.2.3 From 69f9a7035f7ccd0e779fa7986c0107cc386b6a6f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 Feb 2013 15:07:10 -0800 Subject: fully revert change to addOnCompleteCallback -- missed this in e9f53ec --- core/src/test/scala/spark/scheduler/TaskContextSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core') diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala index 4af5de6cf6..a5db7103f5 100644 --- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala @@ -16,7 +16,7 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte val rdd = new RDD[String](sc, List()) { override def getSplits = Array[Split](StubSplit(0)) override def compute(split: Split, context: TaskContext) = { - context.addOnCompleteCallback(_ => completed = true) + context.addOnCompleteCallback(() => completed = true) sys.error("failed") } } -- cgit v1.2.3 From f2fcabf2eaba4b87e4e57edfb651212687796487 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 11 Feb 2013 10:00:13 -0800 Subject: add timing around parts of executor & track result size --- core/src/main/scala/spark/executor/Executor.scala | 10 ++++++++++ core/src/main/scala/spark/executor/TaskMetrics.scala | 7 +++++-- .../main/scala/spark/scheduler/cluster/TaskSetManager.scala | 4 +++- 3 files changed, 18 insertions(+), 3 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 3e3daa29f7..4474ef4593 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -85,6 +85,7 @@ private[spark] class Executor extends Logging { extends Runnable { override def run() { + val startTime = System.currentTimeMillis() SparkEnv.set(env) Thread.currentThread.setContextClassLoader(urlClassLoader) val ser = SparkEnv.get.closureSerializer.newInstance() @@ -98,7 +99,16 @@ private[spark] class Executor extends Logging { val task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader) logInfo("Its generation is " + task.generation) env.mapOutputTracker.updateGeneration(task.generation) + val taskStart = System.currentTimeMillis() val value = task.run(taskId.toInt) + val taskFinish = System.currentTimeMillis() + task.metrics.foreach{ m => + m.executorDeserializeTime = (taskStart - startTime).toInt + m.executorRunTime = (taskFinish - taskStart).toInt + } + //TODO I'd also like to track the time it takes to serialize the task results, but that is huge headache, b/c + // we need to serialize the task metrics first. If TaskMetrics had a custom serialized format, we could + // just change the relevants bytes in the byte buffer val accumUpdates = Accumulators.values val result = new TaskResult(value, accumUpdates, task.metrics.getOrElse(null)) val serializedResult = ser.serialize(result) diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 525abf8893..8db71bdc17 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -8,9 +8,12 @@ case class TaskMetrics( var remoteFetchWaitTime: Option[Long], var remoteFetchTime: Option[Long], var remoteBytesRead: Option[Long], - var shuffleBytesWritten: Option[Long] + var shuffleBytesWritten: Option[Long], + var executorDeserializeTime: Int, + var executorRunTime:Int, + var resultSize: Long ) object TaskMetrics { - private[spark] def empty() : TaskMetrics = TaskMetrics(None,None,None,None,None,None,None,None) + private[spark] def empty() : TaskMetrics = TaskMetrics(None,None,None,None,None,None,None,None, -1, -1, -1) } \ No newline at end of file diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 9ed098e501..48876ffd79 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -259,7 +259,9 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe tid, info.duration, tasksFinished, numTasks)) // Deserialize task result and pass it to the scheduler val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + //lame way to get size into final metrics + val metricsWithSize = result.metrics.copy(resultSize = serializedData.limit()) + sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, metricsWithSize) // Mark finished and stop if we've finished all the tasks finished(index) = true if (tasksFinished == numTasks) { -- cgit v1.2.3 From 176cb20703ffc344a232fb725d50975c31495bcd Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 11 Feb 2013 18:10:54 -0800 Subject: add task result size; better formatting for time interval distributions; cleanup distribution formatting --- .../main/scala/spark/scheduler/SparkListener.scala | 46 +++++++++++++++++++--- .../src/main/scala/spark/scheduler/StageInfo.scala | 4 ++ 2 files changed, 45 insertions(+), 5 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 92d9c0330a..951fe1ae3b 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -15,13 +15,19 @@ case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents class StatsReportListener extends SparkListener with Logging { def onStageCompleted(stageCompleted: StageCompleted) { - println("Finished stage: " + stageCompleted.stageInfo) - showDistribution("task runtime:", stageCompleted.stageInfo.getTaskRuntimeDistribution, "%4.0f") - showDistribution("shuffle bytes written:", stageCompleted.stageInfo.getShuffleBytesWrittenDistribution, d => Utils.memoryBytesToString(d.toLong)) - showDistribution("fetch wait time:",stageCompleted.stageInfo.getRemoteFetchWaitTimeDistribution, "%4.0f") - showDistribution("remote bytes read:", stageCompleted.stageInfo.getRemoteBytesReadDistribution, d => Utils.memoryBytesToString(d.toLong)) + import spark.scheduler.StatsReportListener._ + logInfo("Finished stage: " + stageCompleted.stageInfo) + showMillisDistribution("task runtime:", stageCompleted.stageInfo.getTaskRuntimeDistribution) + showBytesDistribution("shuffle bytes written:", stageCompleted.stageInfo.getShuffleBytesWrittenDistribution) + showMillisDistribution("fetch wait time:",stageCompleted.stageInfo.getRemoteFetchWaitTimeDistribution) + showBytesDistribution("remote bytes read:", stageCompleted.stageInfo.getRemoteBytesReadDistribution) + showBytesDistribution("task result size:", stageCompleted.stageInfo.getTaskResultSizeDistribution) } +} + +object StatsReportListener { + //for profiling, the extremes are more interesting val percentiles = Array[Int](0,5,10,25,50,75,90,95,100) val probabilities = percentiles.map{_ / 100.0} @@ -32,6 +38,14 @@ class StatsReportListener extends SparkListener with Logging { showDistribution(heading, dOpt, f _) } + def showBytesDistribution(heading: String, dOpt: Option[Distribution]) { + showDistribution(heading, dOpt, d => Utils.memoryBytesToString(d.toLong)) + } + + def showMillisDistribution(heading: String, dOpt: Option[Distribution]) { + showDistribution(heading, dOpt, d => StatsReportListener.millisToString(d.toLong)) + } + def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) { dOpt.foreach { d => val stats = d.statCounter @@ -41,4 +55,26 @@ class StatsReportListener extends SparkListener with Logging { logInfo("\t" + quantiles.mkString("\t")) } } + + + val seconds = 1000L + val minutes = seconds * 60 + val hours = minutes * 60 + + /** + * reformat a time interval in milliseconds to a prettier format for output + */ + def millisToString(ms: Long) = { + val (size, units) = + if (ms > hours) { + (ms.toDouble / hours, "hours") + } else if (ms > minutes) { + (ms.toDouble / minutes, "min") + } else if (ms > seconds) { + (ms.toDouble / seconds, "s") + } else { + (ms.toDouble, "ms") + } + "%.1f %s".format(size, units) + } } \ No newline at end of file diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index df3d1ec4dc..299f43d1c5 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -28,4 +28,8 @@ case class StageInfo( def getRemoteBytesReadDistribution = { Distribution(taskMetrics.flatMap{_.remoteBytesRead.map{_.toDouble}}) } + + def getTaskResultSizeDistribution = { + Distribution(taskMetrics.flatMap{_.resultSize.map{_.toDouble}}) + } } -- cgit v1.2.3 From 6f62a5785872baeae5032c7a85bfcc05409f03d7 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 11 Feb 2013 18:37:07 -0800 Subject: add runtime breakdowns --- .../main/scala/spark/scheduler/SparkListener.scala | 11 ++++-- .../src/main/scala/spark/scheduler/StageInfo.scala | 39 +++++++++++++++++++++- 2 files changed, 47 insertions(+), 3 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 951fe1ae3b..f97164669e 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -16,17 +16,24 @@ case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents class StatsReportListener extends SparkListener with Logging { def onStageCompleted(stageCompleted: StageCompleted) { import spark.scheduler.StatsReportListener._ - logInfo("Finished stage: " + stageCompleted.stageInfo) + this.logInfo("Finished stage: " + stageCompleted.stageInfo) showMillisDistribution("task runtime:", stageCompleted.stageInfo.getTaskRuntimeDistribution) showBytesDistribution("shuffle bytes written:", stageCompleted.stageInfo.getShuffleBytesWrittenDistribution) + + //fetch & some io info showMillisDistribution("fetch wait time:",stageCompleted.stageInfo.getRemoteFetchWaitTimeDistribution) showBytesDistribution("remote bytes read:", stageCompleted.stageInfo.getRemoteBytesReadDistribution) showBytesDistribution("task result size:", stageCompleted.stageInfo.getTaskResultSizeDistribution) + + //runtime breakdown + showDistribution("executor (non-fetch) time pct: ", stageCompleted.stageInfo.getExectuorRuntimePercentage, "%2.0f \\%") + showDistribution("fetch wait time pct: ", stageCompleted.stageInfo.getFetchRuntimePercentage, "%2.0f \\%") + showDistribution("other time pct: ", stageCompleted.stageInfo.getOtherRuntimePercentage, "%2.0f \\%") } } -object StatsReportListener { +object StatsReportListener extends Logging { //for profiling, the extremes are more interesting val percentiles = Array[Int](0,5,10,25,50,75,90,95,100) diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 299f43d1c5..ac02d3445c 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -30,6 +30,43 @@ case class StageInfo( } def getTaskResultSizeDistribution = { - Distribution(taskMetrics.flatMap{_.resultSize.map{_.toDouble}}) + Distribution(taskMetrics.map{_.resultSize.toDouble}) + } + + lazy val runtimePercentages = taskMetrics.zip(taskInfos).map{ + case (metrics, info) => RuntimePercentage(info.duration, metrics) + } + + /** + * distribution of the percentage of task runtime of the executor itself, excluding time spent waiting on a fetch + */ + def getExectuorRuntimePercentage = { + Distribution(runtimePercentages.map{_.executorPct}) + } + + /** + * distribution of the percentage of task runtime spent waiting on a fetch + */ + def getFetchRuntimePercentage = { + Distribution(runtimePercentages.flatMap{_.fetchPct}) + } + + /** + * distribution of the percentage of task runtime spent not waiting on fetch, and not actively executing on + * a remote machine (eg., serializing task, sending it over network, sending results back over network) + */ + def getOtherRuntimePercentage = { + Distribution(runtimePercentages.map{_.other}) + } +} + +private[spark] case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double) +private[spark] object RuntimePercentage { + def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { + val denom = totalTime.toDouble + val fetch = metrics.remoteFetchWaitTime.map{_ / denom} + val exec = (metrics.executorRunTime - metrics.remoteFetchWaitTime.getOrElse(0l)) / denom + val other = 1.0 - (exec + fetch.getOrElse(0d)) + RuntimePercentage(exec, fetch, other) } } -- cgit v1.2.3 From d0bfac3eeda09c18c7af521584cfe9db9a5a10c3 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 12 Feb 2013 22:27:36 -0800 Subject: taskInfo tracks if a task is run on a preferred host --- core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala | 3 ++- core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala | 2 +- core/src/main/scala/spark/scheduler/local/LocalScheduler.scala | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index a7e14094fb..53a3c5dc4d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -11,7 +11,8 @@ class TaskInfo( val index: Int, val launchTime: Long, val executorId: String, - val host: String) { + val host: String, + val preferred: Boolean) { var finishTime: Long = 0 var failed = false diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 48876ffd79..236f81bb9f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -208,7 +208,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe taskSet.id, index, taskId, execId, host, prefStr)) // Do various bookkeeping copiesRunning(index) += 1 - val info = new TaskInfo(taskId, index, time, execId, host) + val info = new TaskInfo(taskId, index, time, execId, host, preferred) taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) if (preferred) { diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 6b91728b74..a76253ea14 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -53,7 +53,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon def runTask(task: Task[_], idInJob: Int, attemptId: Int) { logInfo("Running " + task) - val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local") + val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local", true) // Set the Spark execution environment for the worker thread SparkEnv.set(env) try { -- cgit v1.2.3 From 7960927cf4ae2c9bb543c34e1f66f525fb828263 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 12 Feb 2013 23:14:36 -0800 Subject: get rid of a bunch of boilerplate; more formatting happens in Listener, not StageInfo --- .../main/scala/spark/scheduler/SparkListener.scala | 62 ++++++++++++++++++---- .../src/main/scala/spark/scheduler/StageInfo.scala | 60 +-------------------- 2 files changed, 54 insertions(+), 68 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index f97164669e..0a57b94636 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -1,7 +1,9 @@ package spark.scheduler +import spark.scheduler.cluster.TaskInfo import spark.util.Distribution import spark.{Utils, Logging} +import spark.executor.TaskMetrics trait SparkListener { def onStageCompleted(stageCompleted: StageCompleted) @@ -16,19 +18,23 @@ case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents class StatsReportListener extends SparkListener with Logging { def onStageCompleted(stageCompleted: StageCompleted) { import spark.scheduler.StatsReportListener._ + implicit val sc = stageCompleted this.logInfo("Finished stage: " + stageCompleted.stageInfo) - showMillisDistribution("task runtime:", stageCompleted.stageInfo.getTaskRuntimeDistribution) - showBytesDistribution("shuffle bytes written:", stageCompleted.stageInfo.getShuffleBytesWrittenDistribution) + showMillisDistribution("task runtime:", (info, _) => Some(info.duration)) + showBytesDistribution("shuffle bytes written:",(_,metric) => metric.shuffleBytesWritten) //fetch & some io info - showMillisDistribution("fetch wait time:",stageCompleted.stageInfo.getRemoteFetchWaitTimeDistribution) - showBytesDistribution("remote bytes read:", stageCompleted.stageInfo.getRemoteBytesReadDistribution) - showBytesDistribution("task result size:", stageCompleted.stageInfo.getTaskResultSizeDistribution) + showMillisDistribution("fetch wait time:",(_, metric) => metric.remoteFetchWaitTime) + showBytesDistribution("remote bytes read:", (_, metric) => metric.remoteBytesRead) + showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) //runtime breakdown - showDistribution("executor (non-fetch) time pct: ", stageCompleted.stageInfo.getExectuorRuntimePercentage, "%2.0f \\%") - showDistribution("fetch wait time pct: ", stageCompleted.stageInfo.getFetchRuntimePercentage, "%2.0f \\%") - showDistribution("other time pct: ", stageCompleted.stageInfo.getOtherRuntimePercentage, "%2.0f \\%") + val runtimePcts = stageCompleted.stageInfo.taskInfos.zip(stageCompleted.stageInfo.taskMetrics).map{ + case (info, metrics) => RuntimePercentage(info.duration, metrics) + } + showDistribution("executor (non-fetch) time pct: ", Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%") + showDistribution("fetch wait time pct: ", Distribution(runtimePcts.flatMap{_.fetchPct.map{_ * 100}}), "%2.0f %%") + showDistribution("other time pct: ", Distribution(runtimePcts.map{_.other * 100}), "%2.0f %%") } } @@ -40,11 +46,31 @@ object StatsReportListener extends Logging { val probabilities = percentiles.map{_ / 100.0} val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" + def extractDoubleDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Double]): Option[Distribution] = { + Distribution(stage.stageInfo.taskInfos.zip(stage.stageInfo.taskMetrics).flatMap{ + case ((info,metric)) => getMetric(info, metric)}) + } + + //is there some way to setup the types that I can get rid of this completely? + def extractLongDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Long]): Option[Distribution] = { + extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map{_.toDouble}) + } + def showDistribution(heading: String, dOpt: Option[Distribution], format:String) { def f(d:Double) = format.format(d) showDistribution(heading, dOpt, f _) } + def showDistribution(heading:String, format: String, getMetric: (TaskInfo,TaskMetrics) => Option[Double]) + (implicit stage: StageCompleted) { + showDistribution(heading, extractDoubleDistribution(stage, getMetric), format) + } + + def showBytesDistribution(heading:String, getMetric: (TaskInfo,TaskMetrics) => Option[Long]) + (implicit stage: StageCompleted) { + showBytesDistribution(heading, extractLongDistribution(stage, getMetric)) + } + def showBytesDistribution(heading: String, dOpt: Option[Distribution]) { showDistribution(heading, dOpt, d => Utils.memoryBytesToString(d.toLong)) } @@ -53,6 +79,11 @@ object StatsReportListener extends Logging { showDistribution(heading, dOpt, d => StatsReportListener.millisToString(d.toLong)) } + def showMillisDistribution(heading: String, getMetric: (TaskInfo, TaskMetrics) => Option[Long]) + (implicit stage: StageCompleted) { + showMillisDistribution(heading, extractLongDistribution(stage, getMetric)) + } + def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) { dOpt.foreach { d => val stats = d.statCounter @@ -84,4 +115,17 @@ object StatsReportListener extends Logging { } "%.1f %s".format(size, units) } -} \ No newline at end of file +} + + + +case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double) +object RuntimePercentage { + def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { + val denom = totalTime.toDouble + val fetch = metrics.remoteFetchWaitTime.map{_ / denom} + val exec = (metrics.executorRunTime - metrics.remoteFetchWaitTime.getOrElse(0l)) / denom + val other = 1.0 - (exec + fetch.getOrElse(0d)) + RuntimePercentage(exec, fetch, other) + } +} diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index ac02d3445c..179e384cf9 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -10,63 +10,5 @@ case class StageInfo( val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo](), val taskMetrics: mutable.Buffer[TaskMetrics] = mutable.Buffer[TaskMetrics]() ) { - override def toString = stage.rdd.toString - - def getTaskRuntimeDistribution = { - Distribution(taskInfos.map{_.duration.toDouble}) - } - - def getShuffleBytesWrittenDistribution = { - Distribution(taskMetrics.flatMap{_.shuffleBytesWritten.map{_.toDouble}}) - } - - def getRemoteFetchWaitTimeDistribution = { - Distribution(taskMetrics.flatMap{_.remoteFetchWaitTime.map{_.toDouble}}) - } - - def getRemoteBytesReadDistribution = { - Distribution(taskMetrics.flatMap{_.remoteBytesRead.map{_.toDouble}}) - } - - def getTaskResultSizeDistribution = { - Distribution(taskMetrics.map{_.resultSize.toDouble}) - } - - lazy val runtimePercentages = taskMetrics.zip(taskInfos).map{ - case (metrics, info) => RuntimePercentage(info.duration, metrics) - } - - /** - * distribution of the percentage of task runtime of the executor itself, excluding time spent waiting on a fetch - */ - def getExectuorRuntimePercentage = { - Distribution(runtimePercentages.map{_.executorPct}) - } - - /** - * distribution of the percentage of task runtime spent waiting on a fetch - */ - def getFetchRuntimePercentage = { - Distribution(runtimePercentages.flatMap{_.fetchPct}) - } - - /** - * distribution of the percentage of task runtime spent not waiting on fetch, and not actively executing on - * a remote machine (eg., serializing task, sending it over network, sending results back over network) - */ - def getOtherRuntimePercentage = { - Distribution(runtimePercentages.map{_.other}) - } -} - -private[spark] case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double) -private[spark] object RuntimePercentage { - def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { - val denom = totalTime.toDouble - val fetch = metrics.remoteFetchWaitTime.map{_ / denom} - val exec = (metrics.executorRunTime - metrics.remoteFetchWaitTime.getOrElse(0l)) / denom - val other = 1.0 - (exec + fetch.getOrElse(0d)) - RuntimePercentage(exec, fetch, other) - } -} +} \ No newline at end of file -- cgit v1.2.3 From 394d3acc3e66df9377819505824c57008d78a228 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 13 Feb 2013 12:15:45 -0800 Subject: store taskInfo & metrics together in a tuple --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 3 +-- core/src/main/scala/spark/scheduler/SparkListener.scala | 4 ++-- core/src/main/scala/spark/scheduler/StageInfo.scala | 4 +--- 3 files changed, 4 insertions(+), 7 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index f2b518e72d..ce3b62cc4d 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -489,8 +489,7 @@ class DAGScheduler( Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted } pendingTasks(stage) -= task - stageToInfos(stage).taskInfos += event.taskInfo - stageToInfos(stage).taskMetrics += event.taskMetrics + stageToInfos(stage).taskInfos += event.taskInfo -> event.taskMetrics task match { case rt: ResultTask[_, _] => resultStageToJob.get(stage) match { diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 0a57b94636..f844a90680 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -29,7 +29,7 @@ class StatsReportListener extends SparkListener with Logging { showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) //runtime breakdown - val runtimePcts = stageCompleted.stageInfo.taskInfos.zip(stageCompleted.stageInfo.taskMetrics).map{ + val runtimePcts = stageCompleted.stageInfo.taskInfos.map{ case (info, metrics) => RuntimePercentage(info.duration, metrics) } showDistribution("executor (non-fetch) time pct: ", Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%") @@ -47,7 +47,7 @@ object StatsReportListener extends Logging { val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" def extractDoubleDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Double]): Option[Distribution] = { - Distribution(stage.stageInfo.taskInfos.zip(stage.stageInfo.taskMetrics).flatMap{ + Distribution(stage.stageInfo.taskInfos.flatMap{ case ((info,metric)) => getMetric(info, metric)}) } diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 179e384cf9..8d83ff10c4 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -2,13 +2,11 @@ package spark.scheduler import spark.scheduler.cluster.TaskInfo import scala.collection._ -import spark.util.Distribution import spark.executor.TaskMetrics case class StageInfo( val stage: Stage, - val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo](), - val taskMetrics: mutable.Buffer[TaskMetrics] = mutable.Buffer[TaskMetrics]() + val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer[(TaskInfo, TaskMetrics)]() ) { override def toString = stage.rdd.toString } \ No newline at end of file -- cgit v1.2.3 From 796e934d316750aa39c7fbb6b99cbb25d3aa2236 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 Feb 2013 15:07:29 -0800 Subject: add some docs & some cleanup --- core/src/main/scala/spark/executor/TaskMetrics.scala | 17 ++++++++++++++++- core/src/main/scala/spark/scheduler/SparkListener.scala | 7 ++++++- .../main/scala/spark/scheduler/cluster/TaskInfo.scala | 2 -- 3 files changed, 22 insertions(+), 4 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 8db71bdc17..c66abdf2ca 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -1,10 +1,25 @@ package spark.executor +/** + * + * @param totalBlocksFetched total number of blocks fetched in a shuffle (remote or local) + * @param remoteBlocksFetched number of remote blocks fetched in a shuffle + * @param localBlocksFetched local blocks fetched in a shuffle + * @param shuffleReadMillis total time to read shuffle data + * @param remoteFetchWaitTime total time that is spent blocked waiting for shuffle to fetch remote data + * @param remoteFetchTime the total amount of time for all the shuffle fetches. This adds up time from overlapping + * shuffles, so can be longer than task time + * @param remoteBytesRead total number of remote bytes read from a shuffle + * @param shuffleBytesWritten number of bytes written for a shuffle + * @param executorDeserializeTime time taken on the executor to deserialize this task + * @param executorRunTime time the executor spends actually running the task (including fetching shuffle data) + * @param resultSize the number of bytes this task transmitted back to the driver as the TaskResult + */ case class TaskMetrics( - var shuffleReadMillis: Option[Long], var totalBlocksFetched : Option[Int], var remoteBlocksFetched: Option[Int], var localBlocksFetched: Option[Int], + var shuffleReadMillis: Option[Long], var remoteFetchWaitTime: Option[Long], var remoteFetchTime: Option[Long], var remoteBytesRead: Option[Long], diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index f844a90680..0915b3eb5b 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -6,6 +6,9 @@ import spark.{Utils, Logging} import spark.executor.TaskMetrics trait SparkListener { + /** + * called when a stage is completed, with information on the completed stage + */ def onStageCompleted(stageCompleted: StageCompleted) } @@ -14,7 +17,9 @@ sealed trait SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents - +/** + * Simple SparkListener that logs a few summary statistics when each stage completes + */ class StatsReportListener extends SparkListener with Logging { def onStageCompleted(stageCompleted: StageCompleted) { import spark.scheduler.StatsReportListener._ diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index 53a3c5dc4d..dfe3c5a85b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -1,7 +1,5 @@ package spark.scheduler.cluster -import spark.executor.TaskMetrics - /** * Information about a running task attempt inside a TaskSet. */ -- cgit v1.2.3 From 81bd07da265a9fdaad366d8fd46a70df6d5c9806 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 Feb 2013 15:21:45 -0800 Subject: sparkListeners should be a val --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index ce3b62cc4d..1bf5054f4d 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -76,7 +76,7 @@ class DAGScheduler( private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] - private[spark] var sparkListeners = ArrayBuffer[SparkListener]() + private[spark] val sparkListeners = ArrayBuffer[SparkListener]() var cacheLocs = new HashMap[Int, Array[List[String]]] -- cgit v1.2.3 From 9230617f238b4aab8de95173d9f1cdc0b18cdb43 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 Feb 2013 16:55:14 -0800 Subject: add cleanup iterator --- .../main/scala/spark/util/CleanupIterator.scala | 25 ++++++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 core/src/main/scala/spark/util/CleanupIterator.scala (limited to 'core') diff --git a/core/src/main/scala/spark/util/CleanupIterator.scala b/core/src/main/scala/spark/util/CleanupIterator.scala new file mode 100644 index 0000000000..d2093c0230 --- /dev/null +++ b/core/src/main/scala/spark/util/CleanupIterator.scala @@ -0,0 +1,25 @@ +package spark.util + +/** + * Wrapper around an iterator which calls a cleanup method when its finished iterating through its elements + */ +abstract class CleanupIterator[+A, +I <: Iterator[A]](sub: I) extends Iterator[A]{ + def next = sub.next + def hasNext = { + val r = sub.hasNext + if (!r) { + cleanup + } + r + } + + def cleanup +} + +object CleanupIterator { + def apply[A, I <: Iterator[A]](sub: I, cleanupFunction: => Unit) : CleanupIterator[A,I] = { + new CleanupIterator[A,I](sub) { + def cleanup = cleanupFunction + } + } +} \ No newline at end of file -- cgit v1.2.3 From 0f37b43b402ab706a75af30e8912d6a4bd1bd38e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 Feb 2013 16:56:28 -0800 Subject: make the ShuffleFetcher responsible for collecting shuffle metrics, which gives us metrics for CoGroupedRDD and ShuffledRDD --- core/src/main/scala/spark/BlockStoreShuffleFetcher.scala | 16 +++++++++++++--- core/src/main/scala/spark/ShuffleFetcher.scala | 5 ++--- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 9 +-------- core/src/main/scala/spark/rdd/ShuffledRDD.scala | 2 +- 4 files changed, 17 insertions(+), 15 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index 25532dae28..9f5ebe3fd1 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -1,13 +1,14 @@ package spark +import executor.TaskMetrics import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import spark.storage.{DelegateBlockFetchTracker, BlockManagerId} -import spark.util.TimedIterator +import util.{CleanupIterator, TimedIterator} private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { - override def fetch[K, V](shuffleId: Int, reduceId: Int) = { + override def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) = { logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) val blockManager = SparkEnv.get.blockManager @@ -46,9 +47,18 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } } } + val blockFetcherItr = blockManager.getMultiple(blocksByAddress) val itr = new TimedIterator(blockFetcherItr.flatMap(unpackBlock)) with DelegateBlockFetchTracker itr.setDelegate(blockFetcherItr) - itr + CleanupIterator[(K,V), Iterator[(K,V)]](itr, { + metrics.shuffleReadMillis = Some(itr.getNetMillis) + metrics.remoteFetchTime = Some(itr.remoteFetchTime) + metrics.remoteFetchWaitTime = Some(itr.remoteFetchWaitTime) + metrics.remoteBytesRead = Some(itr.remoteBytesRead) + metrics.totalBlocksFetched = Some(itr.totalBlocks) + metrics.localBlocksFetched = Some(itr.numLocalBlocks) + metrics.remoteBlocksFetched = Some(itr.numRemoteBlocks) + }) } } diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index 828cd943f4..442e9f0269 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,14 +1,13 @@ package spark -import spark.storage.BlockFetchTracker -import spark.util.TimedIterator +import executor.TaskMetrics private[spark] abstract class ShuffleFetcher { /** * Fetch the shuffle outputs for a given ShuffleDependency. * @return An iterator over the elements of the fetched shuffle outputs. */ - def fetch[K, V](shuffleId: Int, reduceId: Int) : TimedIterator[(K,V)] with BlockFetchTracker + def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) : Iterator[(K,V)] /** Stop the fetcher */ def stop() {} diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 38efaa12b5..65b4621b87 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -102,17 +102,10 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner) case ShuffleCoGroupSplitDep(shuffleId) => { // Read map outputs of shuffle val fetcher = SparkEnv.get.shuffleFetcher - val fetchItr = fetcher.fetch[K, Seq[Any]](shuffleId, split.index) + val fetchItr = fetcher.fetch[K, Seq[Any]](shuffleId, split.index, context.taskMetrics) for ((k, vs) <- fetchItr) { getSeq(k)(depNum) ++= vs } - context.taskMetrics.shuffleReadMillis = Some(fetchItr.getNetMillis) - context.taskMetrics.remoteFetchTime = Some(fetchItr.remoteFetchTime) - context.taskMetrics.remoteFetchWaitTime = Some(fetchItr.remoteFetchWaitTime) - context.taskMetrics.remoteBytesRead = Some(fetchItr.remoteBytesRead) - context.taskMetrics.totalBlocksFetched = Some(fetchItr.totalBlocks) - context.taskMetrics.localBlocksFetched = Some(fetchItr.numLocalBlocks) - context.taskMetrics.remoteBlocksFetched = Some(fetchItr.numRemoteBlocks) } } JavaConversions.mapAsScalaMap(map).iterator diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index c2f118305f..51f02409b6 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -28,6 +28,6 @@ class ShuffledRDD[K, V]( override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId - SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index) + SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index, context.taskMetrics) } } -- cgit v1.2.3 From 6ae9a22c3e15b3a4e2b0445cb89d855091746a18 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 10:28:08 -0800 Subject: Get spark.default.paralellism on each call to defaultPartitioner, instead of only once, in case the user changes it across Spark uses --- core/src/main/scala/spark/Partitioner.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala index eec0e8dd79..6f8cd17c88 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/spark/Partitioner.scala @@ -10,9 +10,6 @@ abstract class Partitioner extends Serializable { } object Partitioner { - - private val useDefaultParallelism = System.getProperty("spark.default.parallelism") != null - /** * Choose a partitioner to use for a cogroup-like operation between a number of RDDs. * @@ -33,7 +30,7 @@ object Partitioner { for (r <- bySize if r.partitioner != None) { return r.partitioner.get } - if (useDefaultParallelism) { + if (System.getProperty("spark.default.parallelism") != null) { return new HashPartitioner(rdd.context.defaultParallelism) } else { return new HashPartitioner(bySize.head.partitions.size) -- cgit v1.2.3 From 8f17387d9723c1359c86ff1773cf6613f02bb9c6 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 25 Feb 2013 10:31:06 -0800 Subject: remove bogus comment --- core/src/main/scala/spark/TaskContext.scala | 2 -- 1 file changed, 2 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index 5aadd5e261..dd0609026a 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -9,8 +9,6 @@ class TaskContext( val attemptId: Long, val taskMetrics: TaskMetrics = TaskMetrics.empty() ) extends Serializable { - //by adding Task here, I'm destroying the separation between Task & TaskContext ... not sure why they need to - // be separate @transient val onCompleteCallbacks = new ArrayBuffer[() => Unit] -- cgit v1.2.3 From 85a85646d99bdb256f1f667d81647eab45bc6e28 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 25 Feb 2013 12:25:43 -0800 Subject: Disable remote lifecycle logging from Akka. This changes the default setting to `off` for remote lifecycle events. When this is on, it is very chatty at the INFO level. It also prints out several ERROR messages sometimes when sc.stop() is called. --- core/src/main/scala/spark/util/AkkaUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core') diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 30aec5a663..f7b9e2d4ce 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -37,7 +37,7 @@ private[spark] object AkkaUtils { akka.stdout-loglevel = "ERROR" akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.remote.transport = "akka.remote.netty.NettyRemoteTransport" - akka.remote.log-remote-lifecycle-events = on + akka.remote.log-remote-lifecycle-events = off akka.remote.netty.hostname = "%s" akka.remote.netty.port = %d akka.remote.netty.connection-timeout = %ds -- cgit v1.2.3 From 1ef58dadccb878f70ffd638b2d0f260c2939eda9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 14:01:24 -0800 Subject: Add a config property for Akka lifecycle event logging --- core/src/main/scala/spark/util/AkkaUtils.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index f7b9e2d4ce..3e805b7831 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -31,20 +31,22 @@ private[spark] object AkkaUtils { val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt + val lifecycleEvents = System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean val akkaConf = ConfigFactory.parseString(""" akka.daemonic = on akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] akka.stdout-loglevel = "ERROR" akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.remote.transport = "akka.remote.netty.NettyRemoteTransport" - akka.remote.log-remote-lifecycle-events = off akka.remote.netty.hostname = "%s" akka.remote.netty.port = %d akka.remote.netty.connection-timeout = %ds akka.remote.netty.message-frame-size = %d MiB akka.remote.netty.execution-pool-size = %d akka.actor.default-dispatcher.throughput = %d - """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize)) + akka.remote.log-remote-lifecycle-events = %s + """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize, + if (lifecycleEvents) "on" else "off")) val actorSystem = ActorSystem(name, akkaConf, getClass.getClassLoader) -- cgit v1.2.3 From 568bdaf8ae784c9b832f564cb99f1b81ad487f73 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 14:34:55 -0800 Subject: Set spark.deploy.spreadOut to true by default in 0.7 (improves locality) --- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core') diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index b7f167425f..d5188a4779 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -43,7 +43,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "false").toBoolean + val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean override def preStart() { logInfo("Starting Spark master at spark://" + ip + ":" + port) -- cgit v1.2.3 From 490f056cddc3dc02066a1e2414be6576d6441d51 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 15:13:30 -0800 Subject: Allow passing sparkHome and JARs to StreamingContext constructor Also warns if spark.cleaner.ttl is not set in the version where you pass your own SparkContext. --- core/src/main/scala/spark/SparkContext.scala | 2 +- .../scala/spark/api/java/JavaSparkContext.scala | 4 +-- docs/scala-programming-guide.md | 4 +-- docs/streaming-programming-guide.md | 10 ++---- .../main/scala/spark/streaming/Checkpoint.scala | 1 + .../scala/spark/streaming/StreamingContext.scala | 40 ++++++++++++++++------ .../streaming/api/java/JavaStreamingContext.scala | 36 +++++++++++++++++++ .../streaming/dstream/KafkaInputDStream.scala | 1 + 8 files changed, 76 insertions(+), 22 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index df23710d46..7503b1a5ea 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -64,7 +64,7 @@ class SparkContext( val appName: String, val sparkHome: String = null, val jars: Seq[String] = Nil, - environment: Map[String, String] = Map()) + val environment: Map[String, String] = Map()) extends Logging { // Ensure logging is initialized before we spawn any threads diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala index f75fc27c7b..5f18b1e15b 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala @@ -31,8 +31,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI * @param sparkHome The SPARK_HOME directory on the slave nodes - * @param jars Collection of JARs to send to the cluster. These can be paths on the local file - * system or HDFS, HTTP, HTTPS, or FTP URLs. + * @param jarFile JAR file to send to the cluster. This can be a path on the local file system + * or an HDFS, HTTP, HTTPS, or FTP URL. */ def this(master: String, appName: String, sparkHome: String, jarFile: String) = this(new SparkContext(master, appName, sparkHome, Seq(jarFile))) diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index b98718a553..2315aadbdf 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -38,10 +38,10 @@ The first thing a Spark program must do is to create a `SparkContext` object, wh This is done through the following constructor: {% highlight scala %} -new SparkContext(master, jobName, [sparkHome], [jars]) +new SparkContext(master, appName, [sparkHome], [jars]) {% endhighlight %} -The `master` parameter is a string specifying a [Mesos](running-on-mesos.html) cluster to connect to, or a special "local" string to run in local mode, as described below. `jobName` is a name for your job, which will be shown in the Mesos web UI when running on a cluster. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. +The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable. For example, to run on four cores, use diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 7e913b783d..42a4a5619d 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -17,16 +17,12 @@ This guide shows some how to start programming with DStreams. The first thing a Spark Streaming program must do is create a `StreamingContext` object, which tells Spark how to access a cluster. A `StreamingContext` can be created by using {% highlight scala %} -new StreamingContext(master, jobName, batchDuration) +new StreamingContext(master, appName, batchDuration, [sparkHome], [jars]) {% endhighlight %} -The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing mode. The `jobName` is the name of the streaming job, which is the same as the jobName used in SparkContext. It is used to identify this job in the cluster UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Starting with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. - -This constructor creates a SparkContext object using the given `master` and `jobName` parameters. However, if you already have a SparkContext or you need to create a custom SparkContext by specifying list of JARs, then a StreamingContext can be created from the existing SparkContext, by using -{% highlight scala %} -new StreamingContext(sparkContext, batchDuration) -{% endhighlight %} +The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing. The `appName` is a name of your program, which will be shown on your cluster's web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Start with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. Finally, `sparkHome` and `jars` are necessary when running on a cluster to specify the location of your code, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster). +This constructor creates a SparkContext for your job as well, which can be accessed with `streamingContext.sparkContext`. # Attaching Input Sources - InputDStreams diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index e7a392fbbf..e303e33e5e 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -17,6 +17,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) val framework = ssc.sc.appName val sparkHome = ssc.sc.sparkHome val jars = ssc.sc.jars + val environment = ssc.sc.environment val graph = ssc.graph val checkpointDir = ssc.checkpointDir val checkpointDuration = ssc.checkpointDuration diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 25c67b279b..31b5d2c8bc 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -6,7 +6,7 @@ import akka.zeromq.Subscribe import spark.streaming.dstream._ -import spark.{RDD, Logging, SparkEnv, SparkContext} +import spark._ import spark.streaming.receivers.ActorReceiver import spark.streaming.receivers.ReceiverSupervisorStrategy import spark.streaming.receivers.ZeroMQReceiver @@ -14,18 +14,18 @@ import spark.storage.StorageLevel import spark.util.MetadataCleaner import spark.streaming.receivers.ActorReceiver - import scala.collection.mutable.Queue +import scala.collection.Map import java.io.InputStream import java.util.concurrent.atomic.AtomicInteger +import java.util.UUID import org.apache.hadoop.io.LongWritable import org.apache.hadoop.io.Text import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path -import java.util.UUID import twitter4j.Status /** @@ -44,7 +44,9 @@ class StreamingContext private ( * @param sparkContext Existing SparkContext * @param batchDuration The time interval at which streaming data will be divided into batches */ - def this(sparkContext: SparkContext, batchDuration: Duration) = this(sparkContext, null, batchDuration) + def this(sparkContext: SparkContext, batchDuration: Duration) = { + this(sparkContext, null, batchDuration) + } /** * Create a StreamingContext by providing the details necessary for creating a new SparkContext. @@ -52,8 +54,17 @@ class StreamingContext private ( * @param appName A name for your job, to display on the cluster web UI * @param batchDuration The time interval at which streaming data will be divided into batches */ - def this(master: String, appName: String, batchDuration: Duration) = - this(StreamingContext.createNewSparkContext(master, appName), null, batchDuration) + def this( + master: String, + appName: String, + batchDuration: Duration, + sparkHome: String = null, + jars: Seq[String] = Nil, + environment: Map[String, String] = Map()) = { + this(StreamingContext.createNewSparkContext(master, appName, sparkHome, jars, environment), + null, batchDuration) + } + /** * Re-create a StreamingContext from a checkpoint file. @@ -65,15 +76,20 @@ class StreamingContext private ( initLogging() if (sc_ == null && cp_ == null) { - throw new Exception("Streaming Context cannot be initilalized with " + + throw new Exception("Spark Streaming cannot be initialized with " + "both SparkContext and checkpoint as null") } + if (MetadataCleaner.getDelaySeconds < 0) { + throw new SparkException("Spark Streaming cannot be used without setting spark.cleaner.ttl; " + + "set this property before creating a SparkContext (use SPARK_JAVA_OPTS for the shell)") + } + protected[streaming] val isCheckpointPresent = (cp_ != null) protected[streaming] val sc: SparkContext = { if (isCheckpointPresent) { - new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars) + new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars, cp_.environment) } else { sc_ } @@ -478,8 +494,12 @@ object StreamingContext { new PairDStreamFunctions[K, V](stream) } - protected[streaming] def createNewSparkContext(master: String, appName: String): SparkContext = { - + protected[streaming] def createNewSparkContext( + master: String, + appName: String, + sparkHome: String, + jars: Seq[String], + environment: Map[String, String]): SparkContext = { // Set the default cleaner delay to an hour if not already set. // This should be sufficient for even 1 second interval. if (MetadataCleaner.getDelaySeconds < 0) { diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index f3b40b5b88..b528ebbc19 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -45,6 +45,42 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Creates a StreamingContext. + * @param master Name of the Spark Master + * @param appName Name to be used when registering with the scheduler + * @param batchDuration The time interval at which streaming data will be divided into batches + * @param sparkHome The SPARK_HOME directory on the slave nodes + * @param jars Collection of JARs to send to the cluster. These can be paths on the local file + * system or HDFS, HTTP, HTTPS, or FTP URLs. + */ + def this( + master: String, + appName: String, + batchDuration: Duration, + sparkHome: String, + jars: Array[String]) = + this(new StreamingContext(master, appName, batchDuration, sparkHome, jars)) + + /** + * Creates a StreamingContext. + * @param master Name of the Spark Master + * @param appName Name to be used when registering with the scheduler + * @param batchDuration The time interval at which streaming data will be divided into batches + * @param sparkHome The SPARK_HOME directory on the slave nodes + * @param jars Collection of JARs to send to the cluster. These can be paths on the local file + * system or HDFS, HTTP, HTTPS, or FTP URLs. + * @param environment Environment variables to set on worker nodes + */ + def this( + master: String, + appName: String, + batchDuration: Duration, + sparkHome: String, + jars: Array[String], + environment: JMap[String, String]) = + this(new StreamingContext(master, appName, batchDuration, sparkHome, jars, environment)) + + /** + * Creates a StreamingContext using an existing SparkContext. * @param sparkContext The underlying JavaSparkContext to use * @param batchDuration The time interval at which streaming data will be divided into batches */ diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index dc7139cc27..ddd9becf32 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -13,6 +13,7 @@ import kafka.serializer.StringDecoder import kafka.utils.{Utils, ZKGroupTopicDirs} import kafka.utils.ZkUtils._ +import scala.collection.Map import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ -- cgit v1.2.3 From c02e064938059133570547b686d655f2a543964e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 25 Feb 2013 17:27:46 -0800 Subject: Fixed replication bug in BlockManager --- core/src/main/scala/spark/storage/MemoryStore.scala | 4 ++-- core/src/test/scala/spark/DistributedSuite.scala | 16 +++++++++++++++- 2 files changed, 17 insertions(+), 3 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala index ae88ff0bb1..949588476c 100644 --- a/core/src/main/scala/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/spark/storage/MemoryStore.scala @@ -32,8 +32,8 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } override def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) { + bytes.rewind() if (level.deserialized) { - bytes.rewind() val values = blockManager.dataDeserialize(blockId, bytes) val elements = new ArrayBuffer[Any] elements ++= values @@ -58,7 +58,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else { val bytes = blockManager.dataSerialize(blockId, values.iterator) tryToPut(blockId, bytes, bytes.limit, false) - PutResult(bytes.limit(), Right(bytes)) + PutResult(bytes.limit(), Right(bytes.duplicate())) } } diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index caa4ba3a37..4104b33c8b 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -1,5 +1,6 @@ package spark +import network.ConnectionManagerId import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter import org.scalatest.matchers.ShouldMatchers @@ -13,7 +14,7 @@ import com.google.common.io.Files import scala.collection.mutable.ArrayBuffer import SparkContext._ -import storage.StorageLevel +import storage.{GetBlock, BlockManagerWorker, StorageLevel} class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter with LocalSparkContext { @@ -140,9 +141,22 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter test("caching in memory and disk, serialized, replicated") { sc = new SparkContext(clusterUrl, "test") val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_AND_DISK_SER_2) + assert(data.count() === 1000) assert(data.count() === 1000) assert(data.count() === 1000) + + // Get all the locations of the first partition and try to fetch the partitions + // from those locations. + val blockIds = data.partitions.indices.map(index => "rdd_%d_%d".format(data.id, index)).toArray + val blockId = blockIds(0) + val blockManager = SparkEnv.get.blockManager + blockManager.master.getLocations(blockId).foreach(id => { + val bytes = BlockManagerWorker.syncGetBlock( + GetBlock(blockId), ConnectionManagerId(id.ip, id.port)) + val deserialized = blockManager.dataDeserialize(blockId, bytes).asInstanceOf[Iterator[Int]].toList + assert(deserialized === (1 to 100).toList) + }) } test("compute without caching when no partitions fit in memory") { -- cgit v1.2.3 From a65aa549ff9fdf205dd2c1cc04a832f4972c6c3c Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Mon, 25 Feb 2013 23:49:32 -0600 Subject: Override DAGScheduler.runLocally so we can remove the Thread.sleep. --- .../main/scala/spark/scheduler/DAGScheduler.scala | 37 ++++++++++++---------- .../scala/spark/scheduler/DAGSchedulerSuite.scala | 9 ++++-- 2 files changed, 27 insertions(+), 19 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index bf0837c066..9e7791fbb4 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -379,29 +379,34 @@ class DAGScheduler( * We run the operation in a separate thread just in case it takes a bunch of time, so that we * don't block the DAGScheduler event loop or other concurrent jobs. */ - private def runLocally(job: ActiveJob) { + protected def runLocally(job: ActiveJob) { logInfo("Computing the requested partition locally") new Thread("Local computation of job " + job.runId) { override def run() { - try { - SparkEnv.set(env) - val rdd = job.finalStage.rdd - val split = rdd.partitions(job.partitions(0)) - val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0) - try { - val result = job.func(taskContext, rdd.iterator(split, taskContext)) - job.listener.taskSucceeded(0, result) - } finally { - taskContext.executeOnCompleteCallbacks() - } - } catch { - case e: Exception => - job.listener.jobFailed(e) - } + runLocallyWithinThread(job) } }.start() } + // Broken out for easier testing in DAGSchedulerSuite. + protected def runLocallyWithinThread(job: ActiveJob) { + try { + SparkEnv.set(env) + val rdd = job.finalStage.rdd + val split = rdd.partitions(job.partitions(0)) + val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0) + try { + val result = job.func(taskContext, rdd.iterator(split, taskContext)) + job.listener.taskSucceeded(0, result) + } finally { + taskContext.executeOnCompleteCallbacks() + } + } catch { + case e: Exception => + job.listener.jobFailed(e) + } + } + /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 07cccc7ce0..29b565ecad 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -90,7 +90,12 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter { cacheLocations.clear() results.clear() mapOutputTracker = new MapOutputTracker() - scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) + scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) { + override def runLocally(job: ActiveJob) { + // don't bother with the thread while unit testing + runLocallyWithinThread(job) + } + } } after { @@ -203,8 +208,6 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter { override def toString = "DAGSchedulerSuite Local RDD" } runEvent(JobSubmitted(rdd, jobComputeFunc, Array(0), true, null, listener)) - // this shouldn't be needed, but i haven't stubbed out runLocally yet - Thread.sleep(500) assert(results === Map(0 -> 42)) } -- cgit v1.2.3 From db957e5bd74437b3ecbf20406796e133257ae668 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 26 Feb 2013 01:38:50 -0600 Subject: Fix MapOutputTrackerSuite. --- core/src/test/scala/spark/MapOutputTrackerSuite.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) (limited to 'core') diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index b49cd81e23..3abc584b6a 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -82,11 +82,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("remote fetch") { val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", "localhost", 0) val masterTracker = new MapOutputTracker() - masterTracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(masterTracker))) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker") val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", "localhost", 0) val slaveTracker = new MapOutputTracker() - slaveTracker.trackerActor = slaveSystem.actorFor("akka://spark@localhost:" + boundPort) + slaveTracker.trackerActor = slaveSystem.actorFor( + "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker") masterTracker.registerShuffle(10, 1) masterTracker.incrementGeneration() -- cgit v1.2.3 From 73697e289174583fe7b0e829eb8387d1a1aa8721 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 26 Feb 2013 12:07:59 -0800 Subject: Fix overly large thread names in PySpark --- core/src/main/scala/spark/api/python/PythonRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 8c73477384..9b4d54ab4e 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -51,7 +51,7 @@ private[spark] class PythonRDD[T: ClassManifest]( val env = SparkEnv.get // Start a thread to print the process's stderr to ours - new Thread("stderr reader for " + command) { + new Thread("stderr reader for " + pythonExec) { override def run() { for (line <- Source.fromInputStream(proc.getErrorStream).getLines) { System.err.println(line) @@ -60,7 +60,7 @@ private[spark] class PythonRDD[T: ClassManifest]( }.start() // Start a thread to feed the process input from our parent's iterator - new Thread("stdin writer for " + command) { + new Thread("stdin writer for " + pythonExec) { override def run() { SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) -- cgit v1.2.3 From ece3edfffa02f90a71569961b91bf44041f21afe Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 26 Feb 2013 12:11:03 -0800 Subject: Fix a problem with no hosts being counted as alive in the first job --- core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index d9c2f9517b..26fdef101b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -140,6 +140,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Mark each slave as alive and remember its hostname for (o <- offers) { executorIdToHost(o.executorId) = o.hostname + if (!executorsByHost.contains(o.hostname)) { + executorsByHost(o.hostname) = new HashSet() + } } // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) @@ -159,9 +162,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) taskSetTaskIds(manager.taskSet.id) += tid taskIdToExecutorId(tid) = execId activeExecutorIds += execId - if (!executorsByHost.contains(host)) { - executorsByHost(host) = new HashSet() - } executorsByHost(host) += execId availableCpus(i) -= 1 launchedTask = true -- cgit v1.2.3 From 4ab387bcdbfb895438de97b13859450e9e54b878 Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Wed, 27 Feb 2013 13:52:44 -0800 Subject: Fixed master datastructure updates after removing an application; and a typo. --- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- core/src/main/scala/spark/storage/BlockManagerMaster.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index d5188a4779..4af22cf9b6 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -263,7 +263,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor apps -= app idToApp -= app.id actorToApp -= app.driver - addressToWorker -= app.driver.path.address + addressToApp -= app.driver.path.address completedApps += app // Remember it in our history waitingApps -= app for (exec <- app.executors.values) { diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 7389bee150..d93cfc48d0 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -26,7 +26,7 @@ private[spark] class BlockManagerMaster( val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt - val DRIVER_AKKA_ACTOR_NAME = "BlockMasterManager" + val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" val timeout = 10.seconds var driverActor: ActorRef = { -- cgit v1.2.3 From 44134e12bb5cf45b16ca48d68e6a509bab69d256 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 28 Feb 2013 15:14:59 -0800 Subject: Fixed SPARK-706: Failures in block manager put leads to read task hanging. --- .../main/scala/spark/storage/BlockManager.scala | 151 ++++++++++++++------- .../scala/spark/storage/BlockManagerSuite.scala | 20 ++- 2 files changed, 120 insertions(+), 51 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 2462721fb8..f61f573171 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -40,21 +40,36 @@ class BlockManager( class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) { var pending: Boolean = true var size: Long = -1L + var failed: Boolean = false - /** Wait for this BlockInfo to be marked as ready (i.e. block is finished writing) */ - def waitForReady() { + /** + * Wait for this BlockInfo to be marked as ready (i.e. block is finished writing). + * Return true if the block is available, false otherwise. + */ + def waitForReady(): Boolean = { if (pending) { synchronized { while (pending) this.wait() } } + !failed } /** Mark this BlockInfo as ready (i.e. block is finished writing) */ def markReady(sizeInBytes: Long) { - pending = false - size = sizeInBytes synchronized { + pending = false + failed = false + size = sizeInBytes + this.notifyAll() + } + } + + /** Mark this BlockInfo as ready but failed */ + def markFailure() { + synchronized { + failed = true + pending = false this.notifyAll() } } @@ -277,7 +292,14 @@ class BlockManager( val info = blockInfo.get(blockId).orNull if (info != null) { info.synchronized { - info.waitForReady() // In case the block is still being put() by another thread + + // In the another thread is writing the block, wait for it to become ready. + if (!info.waitForReady()) { + // If we get here, the block write failed. + logWarning("Block " + blockId + " was marked as failure.") + return None + } + val level = info.level logDebug("Level for block " + blockId + " is " + level) @@ -362,7 +384,14 @@ class BlockManager( val info = blockInfo.get(blockId).orNull if (info != null) { info.synchronized { - info.waitForReady() // In case the block is still being put() by another thread + + // In the another thread is writing the block, wait for it to become ready. + if (!info.waitForReady()) { + // If we get here, the block write failed. + logWarning("Block " + blockId + " was marked as failure.") + return None + } + val level = info.level logDebug("Level for block " + blockId + " is " + level) @@ -423,12 +452,11 @@ class BlockManager( val data = BlockManagerWorker.syncGetBlock( GetBlock(blockId), ConnectionManagerId(loc.ip, loc.port)) if (data != null) { - logDebug("Data is not null: " + data) return Some(dataDeserialize(blockId, data)) } - logDebug("Data is null") + logDebug("The value of block " + blockId + " is null") } - logDebug("Data not found") + logDebug("Block " + blockId + " not found") return None } @@ -618,9 +646,8 @@ class BlockManager( } val oldBlock = blockInfo.get(blockId).orNull - if (oldBlock != null) { + if (oldBlock != null && oldBlock.waitForReady()) { logWarning("Block " + blockId + " already exists on this machine; not re-adding it") - oldBlock.waitForReady() return oldBlock.size } @@ -648,31 +675,45 @@ class BlockManager( logTrace("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs) + " to get into synchronized block") - if (level.useMemory) { - // Save it just to memory first, even if it also has useDisk set to true; we will later - // drop it to disk if the memory store can't hold it. - val res = memoryStore.putValues(blockId, values, level, true) - size = res.size - res.data match { - case Right(newBytes) => bytesAfterPut = newBytes - case Left(newIterator) => valuesAfterPut = newIterator - } - } else { - // Save directly to disk. - val askForBytes = level.replication > 1 // Don't get back the bytes unless we replicate them - val res = diskStore.putValues(blockId, values, level, askForBytes) - size = res.size - res.data match { - case Right(newBytes) => bytesAfterPut = newBytes - case _ => + try { + if (level.useMemory) { + // Save it just to memory first, even if it also has useDisk set to true; we will later + // drop it to disk if the memory store can't hold it. + val res = memoryStore.putValues(blockId, values, level, true) + size = res.size + res.data match { + case Right(newBytes) => bytesAfterPut = newBytes + case Left(newIterator) => valuesAfterPut = newIterator + } + } else { + // Save directly to disk. + // Don't get back the bytes unless we replicate them. + val askForBytes = level.replication > 1 + val res = diskStore.putValues(blockId, values, level, askForBytes) + size = res.size + res.data match { + case Right(newBytes) => bytesAfterPut = newBytes + case _ => + } } - } - // Now that the block is in either the memory or disk store, let other threads read it, - // and tell the master about it. - myInfo.markReady(size) - if (tellMaster) { - reportBlockStatus(blockId, myInfo) + // Now that the block is in either the memory or disk store, let other threads read it, + // and tell the master about it. + myInfo.markReady(size) + if (tellMaster) { + reportBlockStatus(blockId, myInfo) + } + } catch { + // If we failed at putting the block to memory/disk, notify other possible readers + // that it has failed, and then remove it from the block info map. + case e: Exception => { + // Note that the remove must happen before markFailure otherwise another thread + // could've inserted a new BlockInfo before we remove it. + blockInfo.remove(blockId) + myInfo.markFailure() + logWarning("Putting block " + blockId + " failed", e) + throw e + } } } logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs)) @@ -742,28 +783,38 @@ class BlockManager( logDebug("PutBytes for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs) + " to get into synchronized block") - if (level.useMemory) { - // Store it only in memory at first, even if useDisk is also set to true - bytes.rewind() - memoryStore.putBytes(blockId, bytes, level) - } else { - bytes.rewind() - diskStore.putBytes(blockId, bytes, level) - } + try { + if (level.useMemory) { + // Store it only in memory at first, even if useDisk is also set to true + bytes.rewind() + memoryStore.putBytes(blockId, bytes, level) + } else { + bytes.rewind() + diskStore.putBytes(blockId, bytes, level) + } - // Now that the block is in either the memory or disk store, let other threads read it, - // and tell the master about it. - myInfo.markReady(bytes.limit) - if (tellMaster) { - reportBlockStatus(blockId, myInfo) + // Now that the block is in either the memory or disk store, let other threads read it, + // and tell the master about it. + myInfo.markReady(bytes.limit) + if (tellMaster) { + reportBlockStatus(blockId, myInfo) + } + } catch { + // If we failed at putting the block to memory/disk, notify other possible readers + // that it has failed, and then remove it from the block info map. + case e: Exception => { + // Note that the remove must happen before markFailure otherwise another thread + // could've inserted a new BlockInfo before we remove it. + blockInfo.remove(blockId) + myInfo.markFailure() + logWarning("Putting block " + blockId + " failed", e) + throw e + } } } // If replication had started, then wait for it to finish if (level.replication > 1) { - if (replicationFuture == null) { - throw new Exception("Unexpected") - } Await.ready(replicationFuture, Duration.Inf) } diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index 2d177bbf67..61e793b31f 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -12,6 +12,7 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ import org.scalatest.time.SpanSugar._ +import spark.JavaSerializer import spark.KryoSerializer import spark.SizeEstimator import spark.util.ByteBufferInputStream @@ -262,7 +263,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT t1.join() t2.join() t3.join() - + store.dropFromMemory("a1", null) store.dropFromMemory("a2", null) store.waitForAsyncReregister() @@ -582,4 +583,21 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT System.clearProperty("spark.rdd.compress") } } + + test("block store put failure") { + // Use Java serializer so we can create an unserializable error. + store = new BlockManager("", actorSystem, master, new JavaSerializer, 1200) + + // The put should fail since a1 is not serializable. + class UnserializableClass + val a1 = new UnserializableClass + intercept[java.io.NotSerializableException] { + store.putSingle("a1", a1, StorageLevel.DISK_ONLY) + } + + // Make sure get a1 doesn't hang and returns None. + failAfter(1 second) { + assert(store.getSingle("a1") == None, "a1 should not be in store") + } + } } -- cgit v1.2.3 From 8b06b359da38eb0e76f12f1db4b16edf12fccdda Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 28 Feb 2013 23:34:34 -0800 Subject: bump version to 0.7.1-SNAPSHOT in the subproject poms to keep the maven build building. --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) (limited to 'core') diff --git a/bagel/pom.xml b/bagel/pom.xml index a8256a6e8b..667d28c1a2 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-SNAPSHOT + 0.7.1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 66c62151fe..9d46d94c1c 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-SNAPSHOT + 0.7.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 7d975875fa..2adeec8786 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-SNAPSHOT + 0.7.1-SNAPSHOT ../pom.xml diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 0667b71cc7..a60028bb53 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-SNAPSHOT + 0.7.1-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 4a296fa630..a1b3ccece8 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-SNAPSHOT + 0.7.1-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 15523eadcb..d1a766aeac 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-SNAPSHOT + 0.7.1-SNAPSHOT ../pom.xml -- cgit v1.2.3 From 8fef5b9c5f595b621b5b0218d659f6a5392b3250 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 3 Mar 2013 16:34:04 -0800 Subject: refactoring of TaskMetrics --- .../scala/spark/BlockStoreShuffleFetcher.scala | 18 ++-- .../main/scala/spark/executor/TaskMetrics.scala | 95 +++++++++++++++------- core/src/main/scala/spark/rdd/SubtractedRDD.scala | 2 +- .../scala/spark/scheduler/ShuffleMapTask.scala | 5 +- .../main/scala/spark/scheduler/SparkListener.scala | 44 ++++++---- .../spark/scheduler/cluster/TaskSetManager.scala | 5 +- 6 files changed, 110 insertions(+), 59 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index 9f5ebe3fd1..45fc8cbf7e 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -1,6 +1,6 @@ package spark -import executor.TaskMetrics +import executor.{ShuffleReadMetrics, TaskMetrics} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap @@ -52,13 +52,15 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin val itr = new TimedIterator(blockFetcherItr.flatMap(unpackBlock)) with DelegateBlockFetchTracker itr.setDelegate(blockFetcherItr) CleanupIterator[(K,V), Iterator[(K,V)]](itr, { - metrics.shuffleReadMillis = Some(itr.getNetMillis) - metrics.remoteFetchTime = Some(itr.remoteFetchTime) - metrics.remoteFetchWaitTime = Some(itr.remoteFetchWaitTime) - metrics.remoteBytesRead = Some(itr.remoteBytesRead) - metrics.totalBlocksFetched = Some(itr.totalBlocks) - metrics.localBlocksFetched = Some(itr.numLocalBlocks) - metrics.remoteBlocksFetched = Some(itr.numRemoteBlocks) + val shuffleMetrics = new ShuffleReadMetrics + shuffleMetrics.shuffleReadMillis = itr.getNetMillis + shuffleMetrics.remoteFetchTime = itr.remoteFetchTime + shuffleMetrics.remoteFetchWaitTime = itr.remoteFetchWaitTime + shuffleMetrics.remoteBytesRead = itr.remoteBytesRead + shuffleMetrics.totalBlocksFetched = itr.totalBlocks + shuffleMetrics.localBlocksFetched = itr.numLocalBlocks + shuffleMetrics.remoteBlocksFetched = itr.numRemoteBlocks + metrics.shuffleReadMetrics = Some(shuffleMetrics) }) } } diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index c66abdf2ca..800305cd6c 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -1,34 +1,71 @@ package spark.executor -/** - * - * @param totalBlocksFetched total number of blocks fetched in a shuffle (remote or local) - * @param remoteBlocksFetched number of remote blocks fetched in a shuffle - * @param localBlocksFetched local blocks fetched in a shuffle - * @param shuffleReadMillis total time to read shuffle data - * @param remoteFetchWaitTime total time that is spent blocked waiting for shuffle to fetch remote data - * @param remoteFetchTime the total amount of time for all the shuffle fetches. This adds up time from overlapping - * shuffles, so can be longer than task time - * @param remoteBytesRead total number of remote bytes read from a shuffle - * @param shuffleBytesWritten number of bytes written for a shuffle - * @param executorDeserializeTime time taken on the executor to deserialize this task - * @param executorRunTime time the executor spends actually running the task (including fetching shuffle data) - * @param resultSize the number of bytes this task transmitted back to the driver as the TaskResult - */ -case class TaskMetrics( - var totalBlocksFetched : Option[Int], - var remoteBlocksFetched: Option[Int], - var localBlocksFetched: Option[Int], - var shuffleReadMillis: Option[Long], - var remoteFetchWaitTime: Option[Long], - var remoteFetchTime: Option[Long], - var remoteBytesRead: Option[Long], - var shuffleBytesWritten: Option[Long], - var executorDeserializeTime: Int, - var executorRunTime:Int, - var resultSize: Long -) +class TaskMetrics{ + /** + * Time taken on the executor to deserialize this task + */ + var executorDeserializeTime: Int = _ + /** + * Time the executor spends actually running the task (including fetching shuffle data) + */ + var executorRunTime:Int = _ + /** + * The number of bytes this task transmitted back to the driver as the TaskResult + */ + var resultSize: Long = _ + + /** + * If this task reads from shuffle output, metrics on getting shuffle data will be collected here + */ + var shuffleReadMetrics: Option[ShuffleReadMetrics] = None + + /** + * If this task writes to shuffle output, metrics on the written shuffle data will be collected here + */ + var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None + +} object TaskMetrics { - private[spark] def empty() : TaskMetrics = TaskMetrics(None,None,None,None,None,None,None,None, -1, -1, -1) + private[spark] def empty() : TaskMetrics = new TaskMetrics +} + + +class ShuffleReadMetrics { + /** + * Total number of blocks fetched in a shuffle (remote or local) + */ + var totalBlocksFetched : Int = _ + /** + * Number of remote blocks fetched in a shuffle + */ + var remoteBlocksFetched: Int = _ + /** + * Local blocks fetched in a shuffle + */ + var localBlocksFetched: Int = _ + /** + * Total time to read shuffle data + */ + var shuffleReadMillis: Long = _ + /** + * Total time that is spent blocked waiting for shuffle to fetch remote data + */ + var remoteFetchWaitTime: Long = _ + /** + * The total amount of time for all the shuffle fetches. This adds up time from overlapping + * shuffles, so can be longer than task time + */ + var remoteFetchTime: Long = _ + /** + * Total number of remote bytes read from a shuffle + */ + var remoteBytesRead: Long = _ +} + +class ShuffleWriteMetrics { + /** + * Number of bytes written for a shuffle + */ + var shuffleBytesWritten: Long = _ } \ No newline at end of file diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index daf9cc993c..43ec90cac5 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -89,7 +89,7 @@ private[spark] class SubtractedRDD[T: ClassManifest]( for (k <- rdd.iterator(itsSplit, context)) op(k.asInstanceOf[T]) case ShuffleCoGroupSplitDep(shuffleId) => - for ((k, _) <- SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index)) + for ((k, _) <- SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index, context.taskMetrics)) op(k.asInstanceOf[T]) } // the first dep is rdd1; add all keys to the set diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 0b567d1312..36d087a4d0 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -13,6 +13,7 @@ import com.ning.compress.lzf.LZFInputStream import com.ning.compress.lzf.LZFOutputStream import spark._ +import executor.ShuffleWriteMetrics import spark.storage._ import util.{TimeStampedHashMap, MetadataCleaner} @@ -142,7 +143,9 @@ private[spark] class ShuffleMapTask( totalBytes += size compressedSizes(i) = MapOutputTracker.compressSize(size) } - metrics.get.shuffleBytesWritten = Some(totalBytes) + val shuffleMetrics = new ShuffleWriteMetrics + shuffleMetrics.shuffleBytesWritten = totalBytes + metrics.get.shuffleWriteMetrics = Some(shuffleMetrics) return new MapStatus(blockManager.blockManagerId, compressedSizes) } finally { diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 0915b3eb5b..21185227ab 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -26,11 +26,13 @@ class StatsReportListener extends SparkListener with Logging { implicit val sc = stageCompleted this.logInfo("Finished stage: " + stageCompleted.stageInfo) showMillisDistribution("task runtime:", (info, _) => Some(info.duration)) - showBytesDistribution("shuffle bytes written:",(_,metric) => metric.shuffleBytesWritten) - //fetch & some io info - showMillisDistribution("fetch wait time:",(_, metric) => metric.remoteFetchWaitTime) - showBytesDistribution("remote bytes read:", (_, metric) => metric.remoteBytesRead) + //shuffle write + showBytesDistribution("shuffle bytes written:",(_,metric) => metric.shuffleWriteMetrics.map{_.shuffleBytesWritten}) + + //fetch & io + showMillisDistribution("fetch wait time:",(_, metric) => metric.shuffleReadMetrics.map{_.remoteFetchWaitTime}) + showBytesDistribution("remote bytes read:", (_, metric) => metric.shuffleReadMetrics.map{_.remoteBytesRead}) showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) //runtime breakdown @@ -61,6 +63,18 @@ object StatsReportListener extends Logging { extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map{_.toDouble}) } + def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) { + val stats = d.statCounter + logInfo(heading + stats) + val quantiles = d.getQuantiles(probabilities).map{formatNumber} + logInfo(percentilesHeader) + logInfo("\t" + quantiles.mkString("\t")) + } + + def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) { + dOpt.foreach { d => showDistribution(heading, d, formatNumber)} + } + def showDistribution(heading: String, dOpt: Option[Distribution], format:String) { def f(d:Double) = format.format(d) showDistribution(heading, dOpt, f _) @@ -77,11 +91,15 @@ object StatsReportListener extends Logging { } def showBytesDistribution(heading: String, dOpt: Option[Distribution]) { - showDistribution(heading, dOpt, d => Utils.memoryBytesToString(d.toLong)) + dOpt.foreach{dist => showBytesDistribution(heading, dist)} + } + + def showBytesDistribution(heading: String, dist: Distribution) { + showDistribution(heading, dist, (d => Utils.memoryBytesToString(d.toLong)): Double => String) } def showMillisDistribution(heading: String, dOpt: Option[Distribution]) { - showDistribution(heading, dOpt, d => StatsReportListener.millisToString(d.toLong)) + showDistribution(heading, dOpt, (d => StatsReportListener.millisToString(d.toLong)): Double => String) } def showMillisDistribution(heading: String, getMetric: (TaskInfo, TaskMetrics) => Option[Long]) @@ -89,15 +107,6 @@ object StatsReportListener extends Logging { showMillisDistribution(heading, extractLongDistribution(stage, getMetric)) } - def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) { - dOpt.foreach { d => - val stats = d.statCounter - logInfo(heading + stats) - val quantiles = d.getQuantiles(probabilities).map{formatNumber} - logInfo(percentilesHeader) - logInfo("\t" + quantiles.mkString("\t")) - } - } val seconds = 1000L @@ -128,8 +137,9 @@ case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], othe object RuntimePercentage { def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { val denom = totalTime.toDouble - val fetch = metrics.remoteFetchWaitTime.map{_ / denom} - val exec = (metrics.executorRunTime - metrics.remoteFetchWaitTime.getOrElse(0l)) / denom + val fetchTime = metrics.shuffleReadMetrics.map{_.remoteFetchWaitTime} + val fetch = fetchTime.map{_ / denom} + val exec = (metrics.executorRunTime - fetchTime.getOrElse(0l)) / denom val other = 1.0 - (exec + fetch.getOrElse(0d)) RuntimePercentage(exec, fetch, other) } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 236f81bb9f..c9f2c48804 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -259,9 +259,8 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe tid, info.duration, tasksFinished, numTasks)) // Deserialize task result and pass it to the scheduler val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - //lame way to get size into final metrics - val metricsWithSize = result.metrics.copy(resultSize = serializedData.limit()) - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, metricsWithSize) + result.metrics.resultSize = serializedData.limit() + sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) // Mark finished and stop if we've finished all the tasks finished(index) = true if (tasksFinished == numTasks) { -- cgit v1.2.3 From f1006b99ffce497f4cad79d9bddef278e360056d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 3 Mar 2013 16:39:05 -0800 Subject: change CleanupIterator to CompletionIterator --- .../scala/spark/BlockStoreShuffleFetcher.scala | 4 ++-- .../main/scala/spark/util/CleanupIterator.scala | 25 ---------------------- .../main/scala/spark/util/CompletionIterator.scala | 25 ++++++++++++++++++++++ 3 files changed, 27 insertions(+), 27 deletions(-) delete mode 100644 core/src/main/scala/spark/util/CleanupIterator.scala create mode 100644 core/src/main/scala/spark/util/CompletionIterator.scala (limited to 'core') diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index 45fc8cbf7e..53b0389c3a 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -5,7 +5,7 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import spark.storage.{DelegateBlockFetchTracker, BlockManagerId} -import util.{CleanupIterator, TimedIterator} +import util.{CompletionIterator, TimedIterator} private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { override def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) = { @@ -51,7 +51,7 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin val blockFetcherItr = blockManager.getMultiple(blocksByAddress) val itr = new TimedIterator(blockFetcherItr.flatMap(unpackBlock)) with DelegateBlockFetchTracker itr.setDelegate(blockFetcherItr) - CleanupIterator[(K,V), Iterator[(K,V)]](itr, { + CompletionIterator[(K,V), Iterator[(K,V)]](itr, { val shuffleMetrics = new ShuffleReadMetrics shuffleMetrics.shuffleReadMillis = itr.getNetMillis shuffleMetrics.remoteFetchTime = itr.remoteFetchTime diff --git a/core/src/main/scala/spark/util/CleanupIterator.scala b/core/src/main/scala/spark/util/CleanupIterator.scala deleted file mode 100644 index d2093c0230..0000000000 --- a/core/src/main/scala/spark/util/CleanupIterator.scala +++ /dev/null @@ -1,25 +0,0 @@ -package spark.util - -/** - * Wrapper around an iterator which calls a cleanup method when its finished iterating through its elements - */ -abstract class CleanupIterator[+A, +I <: Iterator[A]](sub: I) extends Iterator[A]{ - def next = sub.next - def hasNext = { - val r = sub.hasNext - if (!r) { - cleanup - } - r - } - - def cleanup -} - -object CleanupIterator { - def apply[A, I <: Iterator[A]](sub: I, cleanupFunction: => Unit) : CleanupIterator[A,I] = { - new CleanupIterator[A,I](sub) { - def cleanup = cleanupFunction - } - } -} \ No newline at end of file diff --git a/core/src/main/scala/spark/util/CompletionIterator.scala b/core/src/main/scala/spark/util/CompletionIterator.scala new file mode 100644 index 0000000000..8139183780 --- /dev/null +++ b/core/src/main/scala/spark/util/CompletionIterator.scala @@ -0,0 +1,25 @@ +package spark.util + +/** + * Wrapper around an iterator which calls a completion method after it successfully iterates through all the elements + */ +abstract class CompletionIterator[+A, +I <: Iterator[A]](sub: I) extends Iterator[A]{ + def next = sub.next + def hasNext = { + val r = sub.hasNext + if (!r) { + completion + } + r + } + + def completion() +} + +object CompletionIterator { + def apply[A, I <: Iterator[A]](sub: I, completionFunction: => Unit) : CompletionIterator[A,I] = { + new CompletionIterator[A,I](sub) { + def completion() = completionFunction + } + } +} \ No newline at end of file -- cgit v1.2.3 From 0bd1d00c2ab75c6133269611d498cac321d9d389 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 3 Mar 2013 16:46:45 -0800 Subject: minor cleanup based on feedback in review request --- .../scala/spark/storage/BlockFetchTracker.scala | 10 ++++++++++ .../main/scala/spark/storage/BlockManager.scala | 22 ---------------------- .../spark/storage/DelegateBlockFetchTracker.scala | 12 ++++++++++++ core/src/main/scala/spark/util/Distribution.scala | 4 ++-- 4 files changed, 24 insertions(+), 24 deletions(-) create mode 100644 core/src/main/scala/spark/storage/BlockFetchTracker.scala create mode 100644 core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala (limited to 'core') diff --git a/core/src/main/scala/spark/storage/BlockFetchTracker.scala b/core/src/main/scala/spark/storage/BlockFetchTracker.scala new file mode 100644 index 0000000000..ababb04305 --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockFetchTracker.scala @@ -0,0 +1,10 @@ +package spark.storage + +private[spark] trait BlockFetchTracker { + def totalBlocks : Int + def numLocalBlocks: Int + def numRemoteBlocks: Int + def remoteFetchTime : Long + def remoteFetchWaitTime: Long + def remoteBytesRead : Long +} diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 677b2e6a42..4964060b1c 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -843,28 +843,6 @@ object BlockManager extends Logging { } } - -private[spark] trait BlockFetchTracker { - def totalBlocks : Int - def numLocalBlocks: Int - def numRemoteBlocks: Int - def remoteFetchTime : Long - def remoteFetchWaitTime: Long - def remoteBytesRead : Long -} - -private[spark] trait DelegateBlockFetchTracker extends BlockFetchTracker { - var delegate : BlockFetchTracker = _ - def setDelegate(d: BlockFetchTracker) {delegate = d} - def totalBlocks = delegate.totalBlocks - def numLocalBlocks = delegate.numLocalBlocks - def numRemoteBlocks = delegate.numRemoteBlocks - def remoteFetchTime = delegate.remoteFetchTime - def remoteFetchWaitTime = delegate.remoteFetchWaitTime - def remoteBytesRead = delegate.remoteBytesRead -} - - class BlockFetcherIterator( private val blockManager: BlockManager, val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] diff --git a/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala b/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala new file mode 100644 index 0000000000..5c491877ba --- /dev/null +++ b/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala @@ -0,0 +1,12 @@ +package spark.storage + +private[spark] trait DelegateBlockFetchTracker extends BlockFetchTracker { + var delegate : BlockFetchTracker = _ + def setDelegate(d: BlockFetchTracker) {delegate = d} + def totalBlocks = delegate.totalBlocks + def numLocalBlocks = delegate.numLocalBlocks + def numRemoteBlocks = delegate.numRemoteBlocks + def remoteFetchTime = delegate.remoteFetchTime + def remoteFetchWaitTime = delegate.remoteFetchWaitTime + def remoteBytesRead = delegate.remoteBytesRead +} diff --git a/core/src/main/scala/spark/util/Distribution.scala b/core/src/main/scala/spark/util/Distribution.scala index ccd9232c8b..24738b4307 100644 --- a/core/src/main/scala/spark/util/Distribution.scala +++ b/core/src/main/scala/spark/util/Distribution.scala @@ -3,11 +3,11 @@ package spark.util import java.io.PrintStream /** - * util for getting some stats from a small sample of numeric values, with some handy summary functions + * Util for getting some stats from a small sample of numeric values, with some handy summary functions. * * Entirely in memory, not intended as a good way to compute stats over large data sets. * - * assumes you are giving it a non-empty set of data + * Assumes you are giving it a non-empty set of data */ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { require(startIdx < endIdx) -- cgit v1.2.3 From 9f0dc829cbaa9aa5011fb917010d13ea5e0a19d7 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 4 Mar 2013 12:08:31 -0800 Subject: Fix TaskMetrics not being serializable --- core/src/main/scala/spark/executor/TaskMetrics.scala | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 800305cd6c..b9c07830f5 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -1,14 +1,16 @@ package spark.executor -class TaskMetrics{ +class TaskMetrics extends Serializable { /** * Time taken on the executor to deserialize this task */ var executorDeserializeTime: Int = _ + /** * Time the executor spends actually running the task (including fetching shuffle data) */ var executorRunTime:Int = _ + /** * The number of bytes this task transmitted back to the driver as the TaskResult */ @@ -23,49 +25,54 @@ class TaskMetrics{ * If this task writes to shuffle output, metrics on the written shuffle data will be collected here */ var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None - } object TaskMetrics { - private[spark] def empty() : TaskMetrics = new TaskMetrics + private[spark] def empty(): TaskMetrics = new TaskMetrics } -class ShuffleReadMetrics { +class ShuffleReadMetrics extends Serializable { /** * Total number of blocks fetched in a shuffle (remote or local) */ var totalBlocksFetched : Int = _ + /** * Number of remote blocks fetched in a shuffle */ var remoteBlocksFetched: Int = _ + /** * Local blocks fetched in a shuffle */ var localBlocksFetched: Int = _ + /** * Total time to read shuffle data */ var shuffleReadMillis: Long = _ + /** * Total time that is spent blocked waiting for shuffle to fetch remote data */ var remoteFetchWaitTime: Long = _ + /** * The total amount of time for all the shuffle fetches. This adds up time from overlapping * shuffles, so can be longer than task time */ var remoteFetchTime: Long = _ + /** * Total number of remote bytes read from a shuffle */ var remoteBytesRead: Long = _ } -class ShuffleWriteMetrics { +class ShuffleWriteMetrics extends Serializable { /** * Number of bytes written for a shuffle */ var shuffleBytesWritten: Long = _ -} \ No newline at end of file +} -- cgit v1.2.3 From 664e5fd24b64d5062b461a9e4f0b72cec106cdbc Mon Sep 17 00:00:00 2001 From: Hiral Patel Date: Thu, 7 Mar 2013 22:16:11 -0800 Subject: Fix reference bug in Kryo serializer, add test, update version --- core/src/main/scala/spark/KryoSerializer.scala | 29 ++++++++++++++-------- .../src/test/scala/spark/KryoSerializerSuite.scala | 1 + project/SparkBuild.scala | 2 +- 3 files changed, 20 insertions(+), 12 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index 0bd73e936b..d723ab7b1e 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -157,27 +157,34 @@ class KryoSerializer extends spark.serializer.Serializer with Logging { // Register maps with a special serializer since they have complex internal structure class ScalaMapSerializer(buildMap: Array[(Any, Any)] => scala.collection.Map[Any, Any]) - extends KSerializer[Array[(Any, Any)] => scala.collection.Map[Any, Any]] { + extends KSerializer[Array[(Any, Any)] => scala.collection.Map[Any, Any]] { + + //hack, look at https://groups.google.com/forum/#!msg/kryo-users/Eu5V4bxCfws/k-8UQ22y59AJ + private final val FAKE_REFERENCE = new Object() override def write( - kryo: Kryo, - output: KryoOutput, - obj: Array[(Any, Any)] => scala.collection.Map[Any, Any]) { + kryo: Kryo, + output: KryoOutput, + obj: Array[(Any, Any)] => scala.collection.Map[Any, Any]) { val map = obj.asInstanceOf[scala.collection.Map[Any, Any]] - kryo.writeObject(output, map.size.asInstanceOf[java.lang.Integer]) + output.writeInt(map.size) for ((k, v) <- map) { kryo.writeClassAndObject(output, k) kryo.writeClassAndObject(output, v) } } override def read ( - kryo: Kryo, - input: KryoInput, - cls: Class[Array[(Any, Any)] => scala.collection.Map[Any, Any]]) + kryo: Kryo, + input: KryoInput, + cls: Class[Array[(Any, Any)] => scala.collection.Map[Any, Any]]) : Array[(Any, Any)] => scala.collection.Map[Any, Any] = { - val size = kryo.readObject(input, classOf[java.lang.Integer]).intValue + kryo.reference(FAKE_REFERENCE) + val size = input.readInt() val elems = new Array[(Any, Any)](size) - for (i <- 0 until size) - elems(i) = (kryo.readClassAndObject(input), kryo.readClassAndObject(input)) + for (i <- 0 until size) { + val k = kryo.readClassAndObject(input) + val v = kryo.readClassAndObject(input) + elems(i)=(k,v) + } buildMap(elems).asInstanceOf[Array[(Any, Any)] => scala.collection.Map[Any, Any]] } } diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index 06d446ea24..327e2ff848 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -82,6 +82,7 @@ class KryoSerializerSuite extends FunSuite { check(mutable.HashMap(1 -> "one", 2 -> "two")) check(mutable.HashMap("one" -> 1, "two" -> 2)) check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) + check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three"))) } test("custom registrator") { diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5b241cc4ef..d44bf3b5e3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -132,7 +132,7 @@ object SparkBuild extends Build { "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION, "asm" % "asm-all" % "3.3.1", "com.google.protobuf" % "protobuf-java" % "2.4.1", - "de.javakaffee" % "kryo-serializers" % "0.20", + "de.javakaffee" % "kryo-serializers" % "0.22", "com.typesafe.akka" % "akka-actor" % "2.0.3", "com.typesafe.akka" % "akka-remote" % "2.0.3", "com.typesafe.akka" % "akka-slf4j" % "2.0.3", -- cgit v1.2.3 From d0216cb38bf976a3e98d1748e1a0ca76c8c7e713 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Sat, 9 Mar 2013 00:05:42 -0800 Subject: Prevent DAGSchedulerSuite from corrupting driver.port. Use the LocalSparkContext abstraction to properly manage clearing spark.driver.port. --- core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) (limited to 'core') diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index b3e6ab4c0f..3cdeb84674 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -15,6 +15,8 @@ import org.easymock.{IAnswer, IArgumentMatcher} import akka.actor.ActorSystem +import spark.LocalSparkContext + import spark.storage.BlockManager import spark.storage.BlockManagerId import spark.storage.BlockManagerMaster @@ -42,13 +44,13 @@ import spark.{FetchFailed, Success} * DAGScheduler#submitWaitingStages (via test utility functions like runEvent or respondToTaskSet) * and capturing the resulting TaskSets from the mock TaskScheduler. */ -class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar with TimeLimitedTests { +class DAGSchedulerSuite extends FunSuite + with BeforeAndAfter with EasyMockSugar with TimeLimitedTests with LocalSparkContext { // impose a time limit on this test in case we don't let the job finish, in which case // JobWaiter#getResult will hang. override val timeLimit = Span(5, Seconds) - val sc: SparkContext = new SparkContext("local", "DAGSchedulerSuite") var scheduler: DAGScheduler = null val taskScheduler = mock[TaskScheduler] val blockManagerMaster = mock[BlockManagerMaster] @@ -105,6 +107,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } before { + sc = new SparkContext("local", "DAGSchedulerSuite") taskSetMatchers.clear() cacheLocations.clear() val actorSystem = ActorSystem("test") @@ -125,8 +128,6 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting { scheduler.stop() } - sc.stop() - System.clearProperty("spark.master.port") } def makeBlockManagerId(host: String): BlockManagerId = -- cgit v1.2.3 From b0983c5762b583c186a3b64606fa2625af962940 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 5 Mar 2013 18:37:25 -0800 Subject: Notify standalone deploy client of application death. Usually, this isn't necessary since the application will be removed as a result of the deploy client disconnecting, but occassionally, the standalone deploy master removes an application otherwise. Also mark applications as FAILED instead of FINISHED when they are killed as a result of their executors failing too many times. --- core/src/main/scala/spark/deploy/DeployMessage.scala | 2 +- core/src/main/scala/spark/deploy/client/Client.scala | 5 +++++ core/src/main/scala/spark/deploy/master/Master.scala | 17 +++++++++++------ 3 files changed, 17 insertions(+), 7 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index 3cbf4fdd98..8a3e64e4c2 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -65,7 +65,7 @@ case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String exitStatus: Option[Int]) private[spark] -case class appKilled(message: String) +case class ApplicationRemoved(message: String) // Internal message in Client diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 1a95524cf9..2fc5e657f9 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -54,6 +54,11 @@ private[spark] class Client( appId = appId_ listener.connected(appId) + case ApplicationRemoved(message) => + logError("Master removed our application: %s; stopping client".format(message)) + markDisconnected() + context.stop(self) + case ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int) => val fullId = appId + "/" + id logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, host, cores)) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 4af22cf9b6..71b9d0801d 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -107,7 +107,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } else { logError("Application %s with ID %s failed %d times, removing it".format( appInfo.desc.name, appInfo.id, appInfo.retryCount)) - removeApplication(appInfo) + removeApplication(appInfo, ApplicationState.FAILED) } } } @@ -129,19 +129,19 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor // The disconnected actor could've been either a worker or an app; remove whichever of // those we have an entry for in the corresponding actor hashmap actorToWorker.get(actor).foreach(removeWorker) - actorToApp.get(actor).foreach(removeApplication) + actorToApp.get(actor).foreach(finishApplication) } case RemoteClientDisconnected(transport, address) => { // The disconnected client could've been either a worker or an app; remove whichever it was addressToWorker.get(address).foreach(removeWorker) - addressToApp.get(address).foreach(removeApplication) + addressToApp.get(address).foreach(finishApplication) } case RemoteClientShutdown(transport, address) => { // The disconnected client could've been either a worker or an app; remove whichever it was addressToWorker.get(address).foreach(removeWorker) - addressToApp.get(address).foreach(removeApplication) + addressToApp.get(address).foreach(finishApplication) } case RequestMasterState => { @@ -257,7 +257,11 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor return app } - def removeApplication(app: ApplicationInfo) { + def finishApplication(app: ApplicationInfo) { + removeApplication(app, ApplicationState.FINISHED) + } + + def removeApplication(app: ApplicationInfo, state: ApplicationState.Value) { if (apps.contains(app)) { logInfo("Removing app " + app.id) apps -= app @@ -270,7 +274,8 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor exec.worker.removeExecutor(exec) exec.worker.actor ! KillExecutor(exec.application.id, exec.id) } - app.markFinished(ApplicationState.FINISHED) // TODO: Mark it as FAILED if it failed + app.markFinished(state) + app.driver ! ApplicationRemoved(state.toString) schedule() } } -- cgit v1.2.3 From ec30188a2ad6325356a5ccfe85cefee6872e8646 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sat, 9 Mar 2013 21:16:53 -0800 Subject: rename remoteFetchWaitTime to fetchWaitTime, since it also includes time from local fetches --- core/src/main/scala/spark/BlockStoreShuffleFetcher.scala | 2 +- core/src/main/scala/spark/executor/TaskMetrics.scala | 4 ++-- core/src/main/scala/spark/scheduler/SparkListener.scala | 4 ++-- core/src/main/scala/spark/storage/BlockFetchTracker.scala | 2 +- core/src/main/scala/spark/storage/BlockManager.scala | 6 +++--- core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala | 2 +- 6 files changed, 10 insertions(+), 10 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index 53b0389c3a..c27ed36406 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -55,7 +55,7 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin val shuffleMetrics = new ShuffleReadMetrics shuffleMetrics.shuffleReadMillis = itr.getNetMillis shuffleMetrics.remoteFetchTime = itr.remoteFetchTime - shuffleMetrics.remoteFetchWaitTime = itr.remoteFetchWaitTime + shuffleMetrics.fetchWaitTime = itr.fetchWaitTime shuffleMetrics.remoteBytesRead = itr.remoteBytesRead shuffleMetrics.totalBlocksFetched = itr.totalBlocks shuffleMetrics.localBlocksFetched = itr.numLocalBlocks diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index b9c07830f5..93bbb6b458 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -54,9 +54,9 @@ class ShuffleReadMetrics extends Serializable { var shuffleReadMillis: Long = _ /** - * Total time that is spent blocked waiting for shuffle to fetch remote data + * Total time that is spent blocked waiting for shuffle to fetch data */ - var remoteFetchWaitTime: Long = _ + var fetchWaitTime: Long = _ /** * The total amount of time for all the shuffle fetches. This adds up time from overlapping diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 21185227ab..a65140b145 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -31,7 +31,7 @@ class StatsReportListener extends SparkListener with Logging { showBytesDistribution("shuffle bytes written:",(_,metric) => metric.shuffleWriteMetrics.map{_.shuffleBytesWritten}) //fetch & io - showMillisDistribution("fetch wait time:",(_, metric) => metric.shuffleReadMetrics.map{_.remoteFetchWaitTime}) + showMillisDistribution("fetch wait time:",(_, metric) => metric.shuffleReadMetrics.map{_.fetchWaitTime}) showBytesDistribution("remote bytes read:", (_, metric) => metric.shuffleReadMetrics.map{_.remoteBytesRead}) showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) @@ -137,7 +137,7 @@ case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], othe object RuntimePercentage { def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { val denom = totalTime.toDouble - val fetchTime = metrics.shuffleReadMetrics.map{_.remoteFetchWaitTime} + val fetchTime = metrics.shuffleReadMetrics.map{_.fetchWaitTime} val fetch = fetchTime.map{_ / denom} val exec = (metrics.executorRunTime - fetchTime.getOrElse(0l)) / denom val other = 1.0 - (exec + fetch.getOrElse(0d)) diff --git a/core/src/main/scala/spark/storage/BlockFetchTracker.scala b/core/src/main/scala/spark/storage/BlockFetchTracker.scala index ababb04305..993aece1f7 100644 --- a/core/src/main/scala/spark/storage/BlockFetchTracker.scala +++ b/core/src/main/scala/spark/storage/BlockFetchTracker.scala @@ -5,6 +5,6 @@ private[spark] trait BlockFetchTracker { def numLocalBlocks: Int def numRemoteBlocks: Int def remoteFetchTime : Long - def remoteFetchWaitTime: Long + def fetchWaitTime: Long def remoteBytesRead : Long } diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 5849045a55..254ce1a4a4 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -903,7 +903,7 @@ class BlockFetcherIterator( private var _remoteBytesRead = 0l private var _remoteFetchTime = 0l - private var _remoteFetchWaitTime = 0l + private var _fetchWaitTime = 0l if (blocksByAddress == null) { throw new IllegalArgumentException("BlocksByAddress is null") @@ -1046,7 +1046,7 @@ class BlockFetcherIterator( val startFetchWait = System.currentTimeMillis() val result = results.take() val stopFetchWait = System.currentTimeMillis() - _remoteFetchWaitTime += (stopFetchWait - startFetchWait) + _fetchWaitTime += (stopFetchWait - startFetchWait) bytesInFlight -= result.size while (!fetchRequests.isEmpty && (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { @@ -1061,7 +1061,7 @@ class BlockFetcherIterator( def numRemoteBlocks = remoteBlockIds.size def remoteFetchTime = _remoteFetchTime - def remoteFetchWaitTime = _remoteFetchWaitTime + def fetchWaitTime = _fetchWaitTime def remoteBytesRead = _remoteBytesRead diff --git a/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala b/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala index 5c491877ba..f6c28dce52 100644 --- a/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala +++ b/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala @@ -7,6 +7,6 @@ private[spark] trait DelegateBlockFetchTracker extends BlockFetchTracker { def numLocalBlocks = delegate.numLocalBlocks def numRemoteBlocks = delegate.numRemoteBlocks def remoteFetchTime = delegate.remoteFetchTime - def remoteFetchWaitTime = delegate.remoteFetchWaitTime + def fetchWaitTime = delegate.fetchWaitTime def remoteBytesRead = delegate.remoteBytesRead } -- cgit v1.2.3 From 20f01a0a1be1349990bb86426f99f4f446432f0c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sat, 9 Mar 2013 21:17:31 -0800 Subject: enable task metrics in local mode, add tests --- .../spark/scheduler/local/LocalScheduler.scala | 10 ++- .../scala/spark/scheduler/SparkListenerSuite.scala | 80 ++++++++++++++++++++++ 2 files changed, 88 insertions(+), 2 deletions(-) create mode 100644 core/src/test/scala/spark/scheduler/SparkListenerSuite.scala (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index a76253ea14..9e1bde3fbe 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -67,8 +67,10 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon logInfo("Size of task " + idInJob + " is " + bytes.limit + " bytes") val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(bytes) updateDependencies(taskFiles, taskJars) // Download any files added with addFile + val deserStart = System.currentTimeMillis() val deserializedTask = ser.deserialize[Task[_]]( taskBytes, Thread.currentThread.getContextClassLoader) + val deserTime = System.currentTimeMillis() - deserStart // Run it val result: Any = deserializedTask.run(attemptId) @@ -77,15 +79,19 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon // executor does. This is useful to catch serialization errors early // on in development (so when users move their local Spark programs // to the cluster, they don't get surprised by serialization errors). - val resultToReturn = ser.deserialize[Any](ser.serialize(result)) + val serResult = ser.serialize(result) + deserializedTask.metrics.get.resultSize = serResult.limit() + val resultToReturn = ser.deserialize[Any](serResult) val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]]( ser.serialize(Accumulators.values)) logInfo("Finished " + task) info.markSuccessful() + deserializedTask.metrics.get.executorRunTime = info.duration.toInt //close enough + deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt // If the threadpool has not already been shutdown, notify DAGScheduler if (!Thread.currentThread().isInterrupted) - listener.taskEnded(task, Success, resultToReturn, accumUpdates, info, null) + listener.taskEnded(task, Success, resultToReturn, accumUpdates, info, deserializedTask.metrics.getOrElse(null)) } catch { case t: Throwable => { logError("Exception in task " + idInJob, t) diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala new file mode 100644 index 0000000000..dd9f2d7e91 --- /dev/null +++ b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala @@ -0,0 +1,80 @@ +package spark.scheduler + +import org.scalatest.FunSuite +import spark.{SparkContext, LocalSparkContext} +import scala.collection.mutable +import org.scalatest.matchers.ShouldMatchers +import spark.SparkContext._ + +/** + * + */ + +class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { + + test("local metrics") { + sc = new SparkContext("local[4]", "test") + val listener = new SaveStageInfo + sc.addSparkListener(listener) + sc.addSparkListener(new StatsReportListener) + + val d = sc.parallelize(1 to 1e4.toInt, 64) + d.count + listener.stageInfos.size should be (1) + + val d2 = d.map{i => i -> i * 2}.setName("shuffle input 1") + + val d3 = d.map{i => i -> (0 to (i % 5))}.setName("shuffle input 2") + + val d4 = d2.cogroup(d3, 64).map{case(k,(v1,v2)) => k -> (v1.size, v2.size)} + d4.setName("A Cogroup") + + d4.collectAsMap + + listener.stageInfos.size should be (4) + listener.stageInfos.foreach {stageInfo => + //small test, so some tasks might take less than 1 millisecond, but average should be greater than 1 ms + checkNonZeroAvg(stageInfo.taskInfos.map{_._1.duration}, stageInfo + " duration") + checkNonZeroAvg(stageInfo.taskInfos.map{_._2.executorRunTime.toLong}, stageInfo + " executorRunTime") + checkNonZeroAvg(stageInfo.taskInfos.map{_._2.executorDeserializeTime.toLong}, stageInfo + " executorDeserializeTime") + if (stageInfo.stage.rdd.name == d4.name) { + checkNonZeroAvg(stageInfo.taskInfos.map{_._2.shuffleReadMetrics.get.fetchWaitTime}, stageInfo + " fetchWaitTime") + } + + stageInfo.taskInfos.foreach{case (taskInfo, taskMetrics) => + taskMetrics.resultSize should be > (0l) + if (isStage(stageInfo, Set(d2.name, d3.name), Set(d4.name))) { + taskMetrics.shuffleWriteMetrics should be ('defined) + taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l) + } + if (stageInfo.stage.rdd.name == d4.name) { + taskMetrics.shuffleReadMetrics should be ('defined) + val sm = taskMetrics.shuffleReadMetrics.get + sm.totalBlocksFetched should be > (0) + sm.shuffleReadMillis should be > (0l) + sm.localBlocksFetched should be > (0) + sm.remoteBlocksFetched should be (0) + sm.remoteBytesRead should be (0l) + sm.remoteFetchTime should be (0l) + } + } + } + } + + def checkNonZeroAvg(m: Traversable[Long], msg: String) { + assert(m.sum / m.size.toDouble > 0.0, msg) + } + + def isStage(stageInfo: StageInfo, rddNames: Set[String], excludedNames: Set[String]) = { + val names = Set(stageInfo.stage.rdd.name) ++ stageInfo.stage.rdd.dependencies.map{_.rdd.name} + !names.intersect(rddNames).isEmpty && names.intersect(excludedNames).isEmpty + } + + class SaveStageInfo extends SparkListener { + val stageInfos = mutable.Buffer[StageInfo]() + def onStageCompleted(stage: StageCompleted) { + stageInfos += stage.stageInfo + } + } + +} -- cgit v1.2.3 From 9f97f2f9d86f542e818e2991588be958c1ffede1 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 10 Mar 2013 22:30:18 -0700 Subject: add a small wait to one task to make sure some task runtime really is non-zero --- .../test/scala/spark/scheduler/SparkListenerSuite.scala | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) (limited to 'core') diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala index dd9f2d7e91..a45a5efbb4 100644 --- a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala @@ -17,16 +17,22 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc val listener = new SaveStageInfo sc.addSparkListener(listener) sc.addSparkListener(new StatsReportListener) + //just to make sure some of the tasks take a noticeable amount of time + val w = {i:Int => + if (i == 0) + Thread.sleep(25) + i + } - val d = sc.parallelize(1 to 1e4.toInt, 64) + val d = sc.parallelize(1 to 1e4.toInt, 64).map{i => w(i)} d.count listener.stageInfos.size should be (1) - val d2 = d.map{i => i -> i * 2}.setName("shuffle input 1") + val d2 = d.map{i => w(i) -> i * 2}.setName("shuffle input 1") - val d3 = d.map{i => i -> (0 to (i % 5))}.setName("shuffle input 2") + val d3 = d.map{i => w(i) -> (0 to (i % 5))}.setName("shuffle input 2") - val d4 = d2.cogroup(d3, 64).map{case(k,(v1,v2)) => k -> (v1.size, v2.size)} + val d4 = d2.cogroup(d3, 64).map{case(k,(v1,v2)) => w(k) -> (v1.size, v2.size)} d4.setName("A Cogroup") d4.collectAsMap -- cgit v1.2.3 From 8a11ac3dc7fe5dad4cb3767bb2dd8769d632402b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 10 Mar 2013 22:31:44 -0700 Subject: increase sleep time --- core/src/test/scala/spark/scheduler/SparkListenerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core') diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala index a45a5efbb4..2f5af10e69 100644 --- a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala @@ -20,7 +20,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc //just to make sure some of the tasks take a noticeable amount of time val w = {i:Int => if (i == 0) - Thread.sleep(25) + Thread.sleep(100) i } -- cgit v1.2.3 From 769d399674f47caf840739ebfc9e7a5b5cb9aef6 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Mon, 11 Mar 2013 14:17:05 -0700 Subject: Send block sizes as longs. --- core/src/main/scala/spark/storage/BlockManagerMessages.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index 1494f90103..cff48d9909 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -49,16 +49,16 @@ class UpdateBlockInfo( blockManagerId.writeExternal(out) out.writeUTF(blockId) storageLevel.writeExternal(out) - out.writeInt(memSize.toInt) - out.writeInt(diskSize.toInt) + out.writeLong(memSize) + out.writeLong(diskSize) } override def readExternal(in: ObjectInput) { blockManagerId = BlockManagerId(in) blockId = in.readUTF() storageLevel = StorageLevel(in) - memSize = in.readInt() - diskSize = in.readInt() + memSize = in.readLong() + diskSize = in.readLong() } } -- cgit v1.2.3 From 9e68f4862556995ff4a02251eac3583542c11ad8 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Mon, 11 Mar 2013 23:59:17 -0500 Subject: More quickly call close in HadoopRDD. This also refactors out the common "gotNext" iterator pattern into a shared utility class. --- core/src/main/scala/spark/rdd/HadoopRDD.scala | 31 +++--------- .../main/scala/spark/serializer/Serializer.scala | 32 ++---------- core/src/main/scala/spark/util/NextIterator.scala | 58 ++++++++++++++++++++++ .../streaming/dstream/SocketInputDStream.scala | 44 ++++------------ 4 files changed, 78 insertions(+), 87 deletions(-) create mode 100644 core/src/main/scala/spark/util/NextIterator.scala (limited to 'core') diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index 78097502bc..43c6749ddc 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -16,6 +16,7 @@ import org.apache.hadoop.mapred.Reporter import org.apache.hadoop.util.ReflectionUtils import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext} +import spark.util.NextIterator /** @@ -62,7 +63,7 @@ class HadoopRDD[K, V]( .asInstanceOf[InputFormat[K, V]] } - override def compute(theSplit: Partition, context: TaskContext) = new Iterator[(K, V)] { + override def compute(theSplit: Partition, context: TaskContext) = new NextIterator[(K, V)] { val split = theSplit.asInstanceOf[HadoopPartition] var reader: RecordReader[K, V] = null @@ -75,34 +76,18 @@ class HadoopRDD[K, V]( val key: K = reader.createKey() val value: V = reader.createValue() - var gotNext = false - var finished = false - - override def hasNext: Boolean = { - if (!gotNext) { - try { - finished = !reader.next(key, value) - } catch { - case eof: EOFException => - finished = true - } - gotNext = true - } - !finished - } - override def next: (K, V) = { - if (!gotNext) { + override def getNext() = { + try { finished = !reader.next(key, value) + } catch { + case eof: EOFException => + finished = true } - if (finished) { - throw new NoSuchElementException("End of stream") - } - gotNext = false (key, value) } - private def close() { + override def close() { try { reader.close() } catch { diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/spark/serializer/Serializer.scala index 50b086125a..d94ffa78f7 100644 --- a/core/src/main/scala/spark/serializer/Serializer.scala +++ b/core/src/main/scala/spark/serializer/Serializer.scala @@ -72,40 +72,14 @@ trait DeserializationStream { * Read the elements of this stream through an iterator. This can only be called once, as * reading each element will consume data from the input source. */ - def asIterator: Iterator[Any] = new Iterator[Any] { - var gotNext = false - var finished = false - var nextValue: Any = null - - private def getNext() { + def asIterator: Iterator[Any] = new spark.util.NextIterator[Any] { + override protected def getNext() = { try { - nextValue = readObject[Any]() + readObject[Any]() } catch { case eof: EOFException => finished = true } - gotNext = true - } - - override def hasNext: Boolean = { - if (!gotNext) { - getNext() - } - if (finished) { - close() - } - !finished - } - - override def next(): Any = { - if (!gotNext) { - getNext() - } - if (finished) { - throw new NoSuchElementException("End of stream") - } - gotNext = false - nextValue } } } diff --git a/core/src/main/scala/spark/util/NextIterator.scala b/core/src/main/scala/spark/util/NextIterator.scala new file mode 100644 index 0000000000..32fae42e73 --- /dev/null +++ b/core/src/main/scala/spark/util/NextIterator.scala @@ -0,0 +1,58 @@ +package spark.util + +/** Provides a basic/boilerplate Iterator implementation. */ +private[spark] abstract class NextIterator[U] extends Iterator[U] { + + private var gotNext = false + private var nextValue: U = _ + protected var finished = false + + /** + * Method for subclasses to implement to provide the next element. + * + * If no next element is available, the subclass should set `finished` + * to `true` and may return any value (it will be ignored). + * + * This convention is required because `null` may be a valid value, + * and using `Option` seems like it might create unnecessary Some/None + * instances, given some iterators might be called in a tight loop. + * + * @return U, or set 'finished' when done + */ + protected def getNext(): U + + /** + * Method for subclasses to optionally implement when all elements + * have been successfully iterated, and the iteration is done. + * + * Note: `NextIterator` cannot guarantee that `close` will be + * called because it has no control over what happens when an exception + * happens in the user code that is calling hasNext/next. + * + * Ideally you should have another try/catch, as in HadoopRDD, that + * ensures any resources are closed should iteration fail. + */ + protected def close() { + } + + override def hasNext: Boolean = { + if (!finished) { + if (!gotNext) { + nextValue = getNext() + if (finished) { + close() + } + gotNext = true + } + } + !finished + } + + override def next(): U = { + if (!hasNext) { + throw new NoSuchElementException("End of stream") + } + gotNext = false + nextValue + } +} \ No newline at end of file diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala index 4af839ad7f..38239b054a 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala @@ -2,6 +2,7 @@ package spark.streaming.dstream import spark.streaming.StreamingContext import spark.storage.StorageLevel +import spark.util.NextIterator import java.io._ import java.net.Socket @@ -59,45 +60,18 @@ object SocketReceiver { */ def bytesToLines(inputStream: InputStream): Iterator[String] = { val dataInputStream = new BufferedReader(new InputStreamReader(inputStream, "UTF-8")) - - val iterator = new Iterator[String] { - var gotNext = false - var finished = false - var nextValue: String = null - - private def getNext() { - try { - nextValue = dataInputStream.readLine() - if (nextValue == null) { - finished = true - } - } - gotNext = true - } - - override def hasNext: Boolean = { - if (!finished) { - if (!gotNext) { - getNext() - if (finished) { - dataInputStream.close() - } - } + new NextIterator[String] { + protected override def getNext() { + val nextValue = dataInputStream.readLine() + if (nextValue == null) { + finished = true } - !finished + nextValue } - override def next(): String = { - if (finished) { - throw new NoSuchElementException("End of stream") - } - if (!gotNext) { - getNext() - } - gotNext = false - nextValue + protected override def close() { + dataInputStream.close() } } - iterator } } -- cgit v1.2.3 From 8f00d23598bc3d96f1e270fd0c652b1602efb18e Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 12 Mar 2013 12:30:10 -0500 Subject: Remove NextIterator.close default implementation. --- core/src/main/scala/spark/serializer/Serializer.scala | 4 ++++ core/src/main/scala/spark/util/NextIterator.scala | 7 +++---- 2 files changed, 7 insertions(+), 4 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/spark/serializer/Serializer.scala index d94ffa78f7..aca86ab6f0 100644 --- a/core/src/main/scala/spark/serializer/Serializer.scala +++ b/core/src/main/scala/spark/serializer/Serializer.scala @@ -81,5 +81,9 @@ trait DeserializationStream { finished = true } } + + override protected def close() { + DeserializationStream.this.close() + } } } diff --git a/core/src/main/scala/spark/util/NextIterator.scala b/core/src/main/scala/spark/util/NextIterator.scala index 32fae42e73..da76b5f6d0 100644 --- a/core/src/main/scala/spark/util/NextIterator.scala +++ b/core/src/main/scala/spark/util/NextIterator.scala @@ -22,8 +22,8 @@ private[spark] abstract class NextIterator[U] extends Iterator[U] { protected def getNext(): U /** - * Method for subclasses to optionally implement when all elements - * have been successfully iterated, and the iteration is done. + * Method for subclasses to implement when all elements have been successfully + * iterated, and the iteration is done. * * Note: `NextIterator` cannot guarantee that `close` will be * called because it has no control over what happens when an exception @@ -32,8 +32,7 @@ private[spark] abstract class NextIterator[U] extends Iterator[U] { * Ideally you should have another try/catch, as in HadoopRDD, that * ensures any resources are closed should iteration fail. */ - protected def close() { - } + protected def close() override def hasNext: Boolean = { if (!finished) { -- cgit v1.2.3 From 1a175d13b967b0080cfa4b5d8d1c278e0e61565a Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Wed, 13 Mar 2013 10:17:39 -0500 Subject: Add NextIterator.closeIfNeeded. --- core/src/main/scala/spark/rdd/HadoopRDD.scala | 2 +- core/src/main/scala/spark/util/NextIterator.scala | 16 +++++++++++++++- 2 files changed, 16 insertions(+), 2 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index 43c6749ddc..a6322dc58d 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -72,7 +72,7 @@ class HadoopRDD[K, V]( reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback{ () => close() } + context.addOnCompleteCallback{ () => closeIfNeeded() } val key: K = reader.createKey() val value: V = reader.createValue() diff --git a/core/src/main/scala/spark/util/NextIterator.scala b/core/src/main/scala/spark/util/NextIterator.scala index da76b5f6d0..48b5018ddd 100644 --- a/core/src/main/scala/spark/util/NextIterator.scala +++ b/core/src/main/scala/spark/util/NextIterator.scala @@ -5,6 +5,7 @@ private[spark] abstract class NextIterator[U] extends Iterator[U] { private var gotNext = false private var nextValue: U = _ + private var closed = false protected var finished = false /** @@ -34,12 +35,25 @@ private[spark] abstract class NextIterator[U] extends Iterator[U] { */ protected def close() + /** + * Calls the subclass-defined close method, but only once. + * + * Usually calling `close` multiple times should be fine, but historically + * there have been issues with some InputFormats throwing exceptions. + */ + def closeIfNeeded() { + if (!closed) { + close() + closed = true + } + } + override def hasNext: Boolean = { if (!finished) { if (!gotNext) { nextValue = getNext() if (finished) { - close() + closeIfNeeded() } gotNext = true } -- cgit v1.2.3 From e7f1a69c6b8acffffb41fcb66a89e2b624c83da7 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Wed, 13 Mar 2013 10:46:33 -0500 Subject: Add a test for NextIterator. --- .../test/scala/spark/util/NextIteratorSuite.scala | 68 ++++++++++++++++++++++ 1 file changed, 68 insertions(+) create mode 100644 core/src/test/scala/spark/util/NextIteratorSuite.scala (limited to 'core') diff --git a/core/src/test/scala/spark/util/NextIteratorSuite.scala b/core/src/test/scala/spark/util/NextIteratorSuite.scala new file mode 100644 index 0000000000..ed5b36da73 --- /dev/null +++ b/core/src/test/scala/spark/util/NextIteratorSuite.scala @@ -0,0 +1,68 @@ +package spark.util + +import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers +import scala.collection.mutable.Buffer +import java.util.NoSuchElementException + +class NextIteratorSuite extends FunSuite with ShouldMatchers { + test("one iteration") { + val i = new StubIterator(Buffer(1)) + i.hasNext should be === true + i.next should be === 1 + i.hasNext should be === false + intercept[NoSuchElementException] { i.next() } + } + + test("two iterations") { + val i = new StubIterator(Buffer(1, 2)) + i.hasNext should be === true + i.next should be === 1 + i.hasNext should be === true + i.next should be === 2 + i.hasNext should be === false + intercept[NoSuchElementException] { i.next() } + } + + test("empty iteration") { + val i = new StubIterator(Buffer()) + i.hasNext should be === false + intercept[NoSuchElementException] { i.next() } + } + + test("close is called once for empty iterations") { + val i = new StubIterator(Buffer()) + i.hasNext should be === false + i.hasNext should be === false + i.closeCalled should be === 1 + } + + test("close is called once for non-empty iterations") { + val i = new StubIterator(Buffer(1, 2)) + i.next should be === 1 + i.next should be === 2 + // close isn't called until we check for the next element + i.closeCalled should be === 0 + i.hasNext should be === false + i.closeCalled should be === 1 + i.hasNext should be === false + i.closeCalled should be === 1 + } + + class StubIterator(ints: Buffer[Int]) extends NextIterator[Int] { + var closeCalled = 0 + + override def getNext() = { + if (ints.size == 0) { + finished = true + 0 + } else { + ints.remove(0) + } + } + + override def close() { + closeCalled += 1 + } + } +} -- cgit v1.2.3 From 63fe22558791e6a511eb1f48efb88e2afdf77659 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Wed, 13 Mar 2013 17:17:34 -0500 Subject: Simplify SubtractedRDD in preparation from subtractByKey. --- core/src/main/scala/spark/PairRDDFunctions.scala | 2 + core/src/main/scala/spark/RDD.scala | 24 ++++++- core/src/main/scala/spark/rdd/SubtractedRDD.scala | 78 ++++++++++------------- 3 files changed, 58 insertions(+), 46 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index e7408e4352..1bd1741a71 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -639,6 +639,8 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( } }, true) } + + // def subtractByKey(other: RDD[K]): RDD[(K,V)] = subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.size))) } private[spark] diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 584efa8adf..3451136fd4 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -408,8 +408,24 @@ abstract class RDD[T: ClassManifest]( * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ - def subtract(other: RDD[T]): RDD[T] = - subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.size))) + def subtract(other: RDD[T]): RDD[T] = { + // If we do have a partitioner, our T is really (K, V), and we'll need to + // unwrap the (T, null) that subtract does to get back to the K + val rdd = subtract(other, partitioner match { + case None => new HashPartitioner(partitions.size) + case Some(p) => new Partitioner() { + override def numPartitions = p.numPartitions + override def getPartition(k: Any) = p.getPartition(k.asInstanceOf[(Any, _)]._1) + } + }) + // Hacky, but if we did have a partitioner, we can keep using it + new RDD[T](rdd) { + override def getPartitions = rdd.partitions + override def getDependencies = rdd.dependencies + override def compute(split: Partition, context: TaskContext) = rdd.compute(split, context) + override val partitioner = RDD.this.partitioner + } + } /** * Return an RDD with the elements from `this` that are not in `other`. @@ -420,7 +436,9 @@ abstract class RDD[T: ClassManifest]( /** * Return an RDD with the elements from `this` that are not in `other`. */ - def subtract(other: RDD[T], p: Partitioner): RDD[T] = new SubtractedRDD[T](this, other, p) + def subtract(other: RDD[T], p: Partitioner): RDD[T] = { + new SubtractedRDD[T, Any](this.map((_, null)), other.map((_, null)), p).keys + } /** * Reduces the elements of this RDD using the specified commutative and associative binary operator. diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 43ec90cac5..1bc84f7e1e 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -1,7 +1,8 @@ package spark.rdd -import java.util.{HashSet => JHashSet} +import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer import spark.RDD import spark.Partitioner import spark.Dependency @@ -27,39 +28,20 @@ import spark.OneToOneDependency * you can use `rdd1`'s partitioner/partition size and not worry about running * out of memory because of the size of `rdd2`. */ -private[spark] class SubtractedRDD[T: ClassManifest]( - @transient var rdd1: RDD[T], - @transient var rdd2: RDD[T], - part: Partitioner) extends RDD[T](rdd1.context, Nil) { +private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest]( + @transient var rdd1: RDD[(K, V)], + @transient var rdd2: RDD[(K, V)], + part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) { override def getDependencies: Seq[Dependency[_]] = { Seq(rdd1, rdd2).map { rdd => - if (rdd.partitioner == Some(part)) { - logInfo("Adding one-to-one dependency with " + rdd) - new OneToOneDependency(rdd) - } else { - logInfo("Adding shuffle dependency with " + rdd) - val mapSideCombinedRDD = rdd.mapPartitions(i => { - val set = new JHashSet[T]() - while (i.hasNext) { - set.add(i.next) - } - set.iterator - }, true) - // ShuffleDependency requires a tuple (k, v), which it will partition by k. - // We need this to partition to map to the same place as the k for - // OneToOneDependency, which means: - // - for already-tupled RDD[(A, B)], into getPartition(a) - // - for non-tupled RDD[C], into getPartition(c) - val part2 = new Partitioner() { - def numPartitions = part.numPartitions - def getPartition(key: Any) = key match { - case (k, v) => part.getPartition(k) - case k => part.getPartition(k) - } - } - new ShuffleDependency(mapSideCombinedRDD.map((_, null)), part2) - } + if (rdd.partitioner == Some(part)) { + logInfo("Adding one-to-one dependency with " + rdd) + new OneToOneDependency(rdd) + } else { + logInfo("Adding shuffle dependency with " + rdd) + new ShuffleDependency(rdd, part) + } } } @@ -81,22 +63,32 @@ private[spark] class SubtractedRDD[T: ClassManifest]( override val partitioner = Some(part) - override def compute(p: Partition, context: TaskContext): Iterator[T] = { + override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = { val partition = p.asInstanceOf[CoGroupPartition] - val set = new JHashSet[T] - def integrate(dep: CoGroupSplitDep, op: T => Unit) = dep match { + val map = new JHashMap[K, ArrayBuffer[V]] + def getSeq(k: K): ArrayBuffer[V] = { + val seq = map.get(k) + if (seq != null) { + seq + } else { + val seq = new ArrayBuffer[V]() + map.put(k, seq) + seq + } + } + def integrate(dep: CoGroupSplitDep, op: ((K, V)) => Unit) = dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => - for (k <- rdd.iterator(itsSplit, context)) - op(k.asInstanceOf[T]) + for (t <- rdd.iterator(itsSplit, context)) + op(t.asInstanceOf[(K, V)]) case ShuffleCoGroupSplitDep(shuffleId) => - for ((k, _) <- SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index, context.taskMetrics)) - op(k.asInstanceOf[T]) + for (t <- SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index, context.taskMetrics)) + op(t.asInstanceOf[(K, V)]) } - // the first dep is rdd1; add all keys to the set - integrate(partition.deps(0), set.add) - // the second dep is rdd2; remove all of its keys from the set - integrate(partition.deps(1), set.remove) - set.iterator + // the first dep is rdd1; add all values to the map + integrate(partition.deps(0), t => getSeq(t._1) += t._2) + // the second dep is rdd2; remove all of its keys + integrate(partition.deps(1), t => map.remove(t._1) ) + map.iterator.map { t => t._2.iterator.map { (t._1, _) } }.flatten } override def clearDependencies() { -- cgit v1.2.3 From 4632c45af16dbdbf9e959fb9cac7f5f4a8d44357 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Thu, 14 Mar 2013 10:35:34 -0500 Subject: Finished subtractByKeys. --- core/src/main/scala/spark/PairRDDFunctions.scala | 19 ++++++++++-- core/src/main/scala/spark/RDD.scala | 36 +++++++++++------------ core/src/main/scala/spark/rdd/SubtractedRDD.scala | 2 +- core/src/test/scala/spark/ShuffleSuite.scala | 28 +++++++++++++++++- 4 files changed, 62 insertions(+), 23 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 1bd1741a71..47b9c6962f 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -440,6 +440,23 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } + /** + * Return an RDD with the pairs from `this` whose keys are not in `other`. + * + * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting + * RDD will be <= us. + */ + def subtractByKey(other: RDD[(K, V)]): RDD[(K, V)] = + subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.size))) + + /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ + def subtractByKey(other: RDD[(K, V)], numPartitions: Int): RDD[(K, V)] = + subtractByKey(other, new HashPartitioner(numPartitions)) + + /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ + def subtractByKey(other: RDD[(K, V)], p: Partitioner): RDD[(K, V)] = + new SubtractedRDD[K, V](self, other, p) + /** * Return the list of values in the RDD for key `key`. This operation is done efficiently if the * RDD has a known partitioner by only searching the partition that the key maps to. @@ -639,8 +656,6 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( } }, true) } - - // def subtractByKey(other: RDD[K]): RDD[(K,V)] = subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.size))) } private[spark] diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 3451136fd4..9bd8a0f98d 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -408,24 +408,8 @@ abstract class RDD[T: ClassManifest]( * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ - def subtract(other: RDD[T]): RDD[T] = { - // If we do have a partitioner, our T is really (K, V), and we'll need to - // unwrap the (T, null) that subtract does to get back to the K - val rdd = subtract(other, partitioner match { - case None => new HashPartitioner(partitions.size) - case Some(p) => new Partitioner() { - override def numPartitions = p.numPartitions - override def getPartition(k: Any) = p.getPartition(k.asInstanceOf[(Any, _)]._1) - } - }) - // Hacky, but if we did have a partitioner, we can keep using it - new RDD[T](rdd) { - override def getPartitions = rdd.partitions - override def getDependencies = rdd.dependencies - override def compute(split: Partition, context: TaskContext) = rdd.compute(split, context) - override val partitioner = RDD.this.partitioner - } - } + def subtract(other: RDD[T]): RDD[T] = + subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.size))) /** * Return an RDD with the elements from `this` that are not in `other`. @@ -437,7 +421,21 @@ abstract class RDD[T: ClassManifest]( * Return an RDD with the elements from `this` that are not in `other`. */ def subtract(other: RDD[T], p: Partitioner): RDD[T] = { - new SubtractedRDD[T, Any](this.map((_, null)), other.map((_, null)), p).keys + if (partitioner == Some(p)) { + // Our partitioner knows how to handle T (which, since we have a partitioner, is + // really (K, V)) so make a new Partitioner that will de-tuple our fake tuples + val p2 = new Partitioner() { + override def numPartitions = p.numPartitions + override def getPartition(k: Any) = p.getPartition(k.asInstanceOf[(Any, _)]._1) + } + // Unfortunately, since we're making a new p2, we'll get ShuffleDependencies + // anyway, and when calling .keys, will not have a partitioner set, even though + // the SubtractedRDD will, thanks to p2's de-tupled partitioning, already be + // partitioned by the right/real keys (e.g. p). + this.map(x => (x, null)).subtractByKey(other.map((_, null)), p2).keys + } else { + this.map(x => (x, null)).subtractByKey(other.map((_, null)), p).keys + } } /** diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 1bc84f7e1e..90488f13cc 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -87,7 +87,7 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest]( // the first dep is rdd1; add all values to the map integrate(partition.deps(0), t => getSeq(t._1) += t._2) // the second dep is rdd2; remove all of its keys - integrate(partition.deps(1), t => map.remove(t._1) ) + integrate(partition.deps(1), t => map.remove(t._1)) map.iterator.map { t => t._2.iterator.map { (t._1, _) } }.flatten } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 8411291b2c..731c45cca2 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -272,13 +272,39 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { } // partitionBy so we have a narrow dependency val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) - println(sc.runJob(a, (i: Iterator[(Int, String)]) => i.toList).toList) // more partitions/no partitioner so a shuffle dependency val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) val c = a.subtract(b) assert(c.collect().toSet === Set((1, "a"), (3, "c"))) + // Ideally we could keep the original partitioner... + assert(c.partitioner === None) + } + + test("subtractByKey") { + sc = new SparkContext("local", "test") + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) + val b = sc.parallelize(Array((2, "bb"), (3, "cc"), (4, "dd")), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + assert(c.partitions.size === a.partitions.size) + } + + test("subtractByKey with narrow dependency") { + sc = new SparkContext("local", "test") + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + // more partitions/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) assert(c.partitioner.get === p) } + } object ShuffleSuite { -- cgit v1.2.3 From 7d8bb4df3a5f8078cd4e86cef5e3b0b728afd2bc Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Thu, 14 Mar 2013 14:44:15 -0500 Subject: Allow subtractByKey's other argument to have a different value type. --- core/src/main/scala/spark/PairRDDFunctions.scala | 8 ++++---- core/src/main/scala/spark/rdd/SubtractedRDD.scala | 6 +++--- core/src/test/scala/spark/ShuffleSuite.scala | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 47b9c6962f..3d1b1ca268 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -446,16 +446,16 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ - def subtractByKey(other: RDD[(K, V)]): RDD[(K, V)] = + def subtractByKey[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, V)] = subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.size))) /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ - def subtractByKey(other: RDD[(K, V)], numPartitions: Int): RDD[(K, V)] = + def subtractByKey[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] = subtractByKey(other, new HashPartitioner(numPartitions)) /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ - def subtractByKey(other: RDD[(K, V)], p: Partitioner): RDD[(K, V)] = - new SubtractedRDD[K, V](self, other, p) + def subtractByKey[W: ClassManifest](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] = + new SubtractedRDD[K, V, W](self, other, p) /** * Return the list of values in the RDD for key `key`. This operation is done efficiently if the diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 90488f13cc..2f8ff9bb34 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -28,9 +28,9 @@ import spark.OneToOneDependency * you can use `rdd1`'s partitioner/partition size and not worry about running * out of memory because of the size of `rdd2`. */ -private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest]( +private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassManifest]( @transient var rdd1: RDD[(K, V)], - @transient var rdd2: RDD[(K, V)], + @transient var rdd2: RDD[(K, W)], part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) { override def getDependencies: Seq[Dependency[_]] = { @@ -40,7 +40,7 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest]( new OneToOneDependency(rdd) } else { logInfo("Adding shuffle dependency with " + rdd) - new ShuffleDependency(rdd, part) + new ShuffleDependency(rdd.asInstanceOf[RDD[(K, Any)]], part) } } } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 731c45cca2..2b2a90defa 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -283,7 +283,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { test("subtractByKey") { sc = new SparkContext("local", "test") val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) - val b = sc.parallelize(Array((2, "bb"), (3, "cc"), (4, "dd")), 4) + val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) val c = a.subtractByKey(b) assert(c.collect().toSet === Set((1, "a"), (1, "a"))) assert(c.partitions.size === a.partitions.size) -- cgit v1.2.3 From 7786881f47699d8e463d911468ed9d43079948a8 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Thu, 14 Mar 2013 14:57:12 -0500 Subject: Fix tabs that snuck in. --- core/src/main/scala/spark/rdd/SubtractedRDD.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 2f8ff9bb34..0a02561062 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -35,13 +35,13 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM override def getDependencies: Seq[Dependency[_]] = { Seq(rdd1, rdd2).map { rdd => - if (rdd.partitioner == Some(part)) { - logInfo("Adding one-to-one dependency with " + rdd) - new OneToOneDependency(rdd) - } else { - logInfo("Adding shuffle dependency with " + rdd) - new ShuffleDependency(rdd.asInstanceOf[RDD[(K, Any)]], part) - } + if (rdd.partitioner == Some(part)) { + logInfo("Adding one-to-one dependency with " + rdd) + new OneToOneDependency(rdd) + } else { + logInfo("Adding shuffle dependency with " + rdd) + new ShuffleDependency(rdd.asInstanceOf[RDD[(K, Any)]], part) + } } } @@ -97,4 +97,4 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM rdd2 = null } -} \ No newline at end of file +} -- cgit v1.2.3 From b1422cbdd59569261c9df84034a5a03833ec3996 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 14 Mar 2013 12:59:58 -0700 Subject: added foldByKey --- core/src/main/scala/spark/PairRDDFunctions.scala | 7 +++++++ 1 file changed, 7 insertions(+) (limited to 'core') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index e7408e4352..a6e00c3a84 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -88,6 +88,13 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) } + /** + * Merge the values for each key using an associative function and a neutral "zero value". + */ + def foldByKey[V1 >: V](zeroValue: V1)(op: (V1, V1) => V1): RDD[(K, V1)] = { + groupByKey.mapValues(seq => seq.fold(zeroValue)(op)) + } + /** * Merge the values for each key using an associative reduce function. This will also perform * the merging locally on each mapper before sending results to a reducer, similarly to a -- cgit v1.2.3 From 16a4ca45373cd4b75032f88668610d9b693fb4b3 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 14 Mar 2013 13:58:37 -0700 Subject: restrict V type of foldByKey in order to retain ClassManifest; added foldByKey to Java API and test --- core/src/main/scala/spark/PairRDDFunctions.scala | 4 ++-- .../main/scala/spark/api/java/JavaPairRDD.scala | 6 ++++++ core/src/test/scala/spark/JavaAPISuite.java | 24 +++++++++++++++++++++- 3 files changed, 31 insertions(+), 3 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index a6e00c3a84..0fde902261 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -91,8 +91,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Merge the values for each key using an associative function and a neutral "zero value". */ - def foldByKey[V1 >: V](zeroValue: V1)(op: (V1, V1) => V1): RDD[(K, V1)] = { - groupByKey.mapValues(seq => seq.fold(zeroValue)(op)) + def foldByKey(zeroValue: V)(op: (V, V) => V): RDD[(K, V)] = { + groupByKey.mapValues(seq => seq.fold[V](zeroValue)(op)) } /** diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index c1bd13c49a..1e1c910202 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -160,6 +160,12 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif : PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) + /** + * Merge the values for each key using an associative function and a neutral "zero value". + */ + def foldByKey(zeroValue: V, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = + fromRDD(rdd.foldByKey(zeroValue)(func)) + /** * Merge the values for each key using an associative reduce function. This will also perform * the merging locally on each mapper before sending results to a reducer, similarly to a diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 26e3ab72c0..b83076b929 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -196,7 +196,29 @@ public class JavaAPISuite implements Serializable { Assert.assertEquals(33, sum); } - @Test + @Test + public void foldByKey() { + List> pairs = Arrays.asList( + new Tuple2(2, 1), + new Tuple2(2, 1), + new Tuple2(1, 1), + new Tuple2(3, 2), + new Tuple2(3, 1) + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + JavaPairRDD sums = rdd.foldByKey(0, + new Function2() { + @Override + public Integer call(Integer a, Integer b) { + return a + b; + } + }); + Assert.assertEquals(1, sums.lookup(1).get(0).intValue()); + Assert.assertEquals(2, sums.lookup(2).get(0).intValue()); + Assert.assertEquals(3, sums.lookup(3).get(0).intValue()); + } + + @Test public void reduceByKey() { List> pairs = Arrays.asList( new Tuple2(2, 1), -- cgit v1.2.3 From 857010392b4281fb16078bc97c571b7068c7e822 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 15 Mar 2013 10:56:05 -0700 Subject: Fuller implementation of foldByKey --- core/src/main/scala/spark/PairRDDFunctions.scala | 19 +++++++++++++++++-- core/src/main/scala/spark/api/java/JavaPairRDD.scala | 12 ++++++++++++ 2 files changed, 29 insertions(+), 2 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 0fde902261..1eb1a7e14a 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -91,8 +91,23 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Merge the values for each key using an associative function and a neutral "zero value". */ - def foldByKey(zeroValue: V)(op: (V, V) => V): RDD[(K, V)] = { - groupByKey.mapValues(seq => seq.fold[V](zeroValue)(op)) + def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = { + groupByKey(partitioner).mapValues(seq => seq.fold[V](zeroValue)(func)) + } + + + /** + * Merge the values for each key using an associative function and a neutral "zero value". + */ + def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): RDD[(K, V)] = { + foldByKey(zeroValue, new HashPartitioner(numPartitions))(func) + } + + /** + * Merge the values for each key using an associative function and a neutral "zero value". + */ + def foldByKey(zeroValue: V)(func: (V, V) => V): RDD[(K, V)] = { + foldByKey(zeroValue, defaultPartitioner(self))(func) } /** diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 1e1c910202..1dc73a3584 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -160,6 +160,18 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif : PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) + /** + * Merge the values for each key using an associative function and a neutral "zero value". + */ + def foldByKey(zeroValue: V, partitioner: Partitioner, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = + fromRDD(rdd.foldByKey(zeroValue, partitioner)(func)) + + /** + * Merge the values for each key using an associative function and a neutral "zero value". + */ + def foldByKey(zeroValue: V, numPartitions: Int, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = + fromRDD(rdd.foldByKey(zeroValue, numPartitions)(func)) + /** * Merge the values for each key using an associative function and a neutral "zero value". */ -- cgit v1.2.3 From 1a4070477d40ac4d7f3d8718161f705f7f0b70c3 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 15 Mar 2013 11:28:28 -0700 Subject: whitespace cleanup --- core/src/test/scala/spark/JavaAPISuite.java | 44 ++++++++++++++--------------- 1 file changed, 22 insertions(+), 22 deletions(-) (limited to 'core') diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index b83076b929..d3dcd3bbeb 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -196,29 +196,29 @@ public class JavaAPISuite implements Serializable { Assert.assertEquals(33, sum); } - @Test - public void foldByKey() { - List> pairs = Arrays.asList( - new Tuple2(2, 1), - new Tuple2(2, 1), - new Tuple2(1, 1), - new Tuple2(3, 2), - new Tuple2(3, 1) - ); - JavaPairRDD rdd = sc.parallelizePairs(pairs); - JavaPairRDD sums = rdd.foldByKey(0, - new Function2() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }); - Assert.assertEquals(1, sums.lookup(1).get(0).intValue()); - Assert.assertEquals(2, sums.lookup(2).get(0).intValue()); - Assert.assertEquals(3, sums.lookup(3).get(0).intValue()); - } + @Test + public void foldByKey() { + List> pairs = Arrays.asList( + new Tuple2(2, 1), + new Tuple2(2, 1), + new Tuple2(1, 1), + new Tuple2(3, 2), + new Tuple2(3, 1) + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + JavaPairRDD sums = rdd.foldByKey(0, + new Function2() { + @Override + public Integer call(Integer a, Integer b) { + return a + b; + } + }); + Assert.assertEquals(1, sums.lookup(1).get(0).intValue()); + Assert.assertEquals(2, sums.lookup(2).get(0).intValue()); + Assert.assertEquals(3, sums.lookup(3).get(0).intValue()); + } - @Test + @Test public void reduceByKey() { List> pairs = Arrays.asList( new Tuple2(2, 1), -- cgit v1.2.3 From 7fd2708edaa863701ed8032e395e255df399d898 Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Fri, 15 Mar 2013 11:41:51 -0700 Subject: Add a log4j compile dependency to fix build in IntelliJ Also rename parent project to spark-parent (otherwise it shows up as "parent" in IntelliJ, which is very confusing). --- bagel/pom.xml | 2 +- core/pom.xml | 6 +++++- examples/pom.xml | 2 +- pom.xml | 9 ++++++++- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 18 insertions(+), 7 deletions(-) (limited to 'core') diff --git a/bagel/pom.xml b/bagel/pom.xml index 667d28c1a2..510cff4669 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 9d46d94c1c..fe9c803728 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.1-SNAPSHOT ../pom.xml @@ -87,6 +87,10 @@ org.apache.mesos mesos + + log4j + log4j + org.scalatest diff --git a/examples/pom.xml b/examples/pom.xml index 2adeec8786..39cc47c709 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 09ad903e6e..08d1fc12e0 100644 --- a/pom.xml +++ b/pom.xml @@ -2,7 +2,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.1-SNAPSHOT pom Spark Project Parent POM @@ -58,6 +58,7 @@ 1.1.1 1.6.1 4.1.2 + 1.2.17 @@ -267,6 +268,12 @@ ${scala.version} + + log4j + log4j + ${log4j.version} + + org.scalatest scalatest_${scala.version} diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index a60028bb53..dd720e2291 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.1-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index a1b3ccece8..a3e4606edc 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.1-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index d1a766aeac..ec077e8089 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.1-SNAPSHOT ../pom.xml -- cgit v1.2.3 From ca9f81e8fcf46bceb032f3bd0efb9f6ed2c25656 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Sat, 16 Mar 2013 13:31:01 -0700 Subject: refactor foldByKey to use combineByKey --- core/src/main/scala/spark/PairRDDFunctions.scala | 14 ++++++++++---- core/src/main/scala/spark/api/java/JavaPairRDD.scala | 12 +++++++++--- 2 files changed, 19 insertions(+), 7 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index ffa6336ceb..e1dde1b497 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -89,22 +89,28 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( } /** - * Merge the values for each key using an associative function and a neutral "zero value". + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = { - groupByKey(partitioner).mapValues(seq => seq.fold[V](zeroValue)(func)) + combineByKey[V]({v: V => func(zeroValue, v)}, func, func, partitioner) } /** - * Merge the values for each key using an associative function and a neutral "zero value". + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): RDD[(K, V)] = { foldByKey(zeroValue, new HashPartitioner(numPartitions))(func) } /** - * Merge the values for each key using an associative function and a neutral "zero value". + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V)(func: (V, V) => V): RDD[(K, V)] = { foldByKey(zeroValue, defaultPartitioner(self))(func) diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 1dc73a3584..49aaabf835 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -161,19 +161,25 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) /** - * Merge the values for each key using an associative function and a neutral "zero value". + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V, partitioner: Partitioner, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = fromRDD(rdd.foldByKey(zeroValue, partitioner)(func)) /** - * Merge the values for each key using an associative function and a neutral "zero value". + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V, numPartitions: Int, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = fromRDD(rdd.foldByKey(zeroValue, numPartitions)(func)) /** - * Merge the values for each key using an associative function and a neutral "zero value". + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = fromRDD(rdd.foldByKey(zeroValue)(func)) -- cgit v1.2.3 From 32979b5e7dddd7f42d3eb1f34881cc052f79fbdf Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Sat, 16 Mar 2013 13:36:46 -0700 Subject: whitespace --- core/src/main/scala/spark/PairRDDFunctions.scala | 1 - 1 file changed, 1 deletion(-) (limited to 'core') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index e1dde1b497..07efba9e8d 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -97,7 +97,6 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( combineByKey[V]({v: V => func(zeroValue, v)}, func, func, partitioner) } - /** * Merge the values for each key using an associative function and a neutral "zero value" which may * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for -- cgit v1.2.3 From fb34967815bd990a612a69dc5b428cf776f9e368 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Mon, 18 Mar 2013 01:55:50 -0500 Subject: Remove try/catch block that can't be hit. --- core/src/main/scala/spark/deploy/worker/Worker.scala | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 2bbc931316..da3f4f636c 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -74,16 +74,10 @@ private[spark] class Worker( def connectToMaster() { logInfo("Connecting to master " + masterUrl) - try { - master = context.actorFor(Master.toAkkaUrl(masterUrl)) - master ! RegisterWorker(workerId, ip, port, cores, memory, webUiPort, publicAddress) - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) - context.watch(master) // Doesn't work with remote actors, but useful for testing - } catch { - case e: Exception => - logError("Failed to connect to master", e) - System.exit(1) - } + master = context.actorFor(Master.toAkkaUrl(masterUrl)) + master ! RegisterWorker(workerId, ip, port, cores, memory, webUiPort, publicAddress) + context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.watch(master) // Doesn't work with remote actors, but useful for testing } def startWebUi() { -- cgit v1.2.3 From aceae029f7c82f87ed5fc2775190cad1bd6cc326 Mon Sep 17 00:00:00 2001 From: Giovanni Delussu Date: Tue, 19 Mar 2013 11:25:45 +0100 Subject: CoalescedRDD changed to work with a big number of partitions both in the original and the new coalesced RDD. The limitation was in the range that Scala.Int can represent. --- core/src/main/scala/spark/rdd/CoalescedRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 0d16cf6e85..6d862c0c28 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -37,8 +37,8 @@ class CoalescedRDD[T: ClassManifest]( prevSplits.map(_.index).map{idx => new CoalescedRDDPartition(idx, prev, Array(idx)) } } else { (0 until maxPartitions).map { i => - val rangeStart = (i * prevSplits.length) / maxPartitions - val rangeEnd = ((i + 1) * prevSplits.length) / maxPartitions + val rangeStart = ((i.toLong * prevSplits.length) / maxPartitions).toInt + val rangeEnd = (((i.toLong + 1) * prevSplits.length) / maxPartitions).toInt new CoalescedRDDPartition(i, prev, (rangeStart until rangeEnd).toArray) }.toArray } -- cgit v1.2.3 From bd167f83b0adbff43b6cd910893103001679317b Mon Sep 17 00:00:00 2001 From: Andrey Kouznetsov Date: Tue, 19 Mar 2013 17:15:15 +0400 Subject: call setConf from input format if it is Configurable --- core/src/main/scala/spark/rdd/HadoopRDD.scala | 7 +++++++ core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 8 +++++++- 2 files changed, 14 insertions(+), 1 deletion(-) (limited to 'core') diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index a6322dc58d..cbf5512e24 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -17,6 +17,7 @@ import org.apache.hadoop.util.ReflectionUtils import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext} import spark.util.NextIterator +import org.apache.hadoop.conf.Configurable /** @@ -50,6 +51,9 @@ class HadoopRDD[K, V]( override def getPartitions: Array[Partition] = { val inputFormat = createInputFormat(conf) + if (inputFormat.isInstanceOf[Configurable]) { + inputFormat.asInstanceOf[Configurable].setConf(conf) + } val inputSplits = inputFormat.getSplits(conf, minSplits) val array = new Array[Partition](inputSplits.size) for (i <- 0 until inputSplits.size) { @@ -69,6 +73,9 @@ class HadoopRDD[K, V]( val conf = confBroadcast.value.value val fmt = createInputFormat(conf) + if (fmt.isInstanceOf[Configurable]) { + fmt.asInstanceOf[Configurable].setConf(conf) + } reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index df2361025c..bdd974590a 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -3,7 +3,7 @@ package spark.rdd import java.text.SimpleDateFormat import java.util.Date -import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ @@ -42,6 +42,9 @@ class NewHadoopRDD[K, V]( override def getPartitions: Array[Partition] = { val inputFormat = inputFormatClass.newInstance + if (inputFormat.isInstanceOf[Configurable]) { + inputFormat.asInstanceOf[Configurable].setConf(conf) + } val jobContext = newJobContext(conf, jobId) val rawSplits = inputFormat.getSplits(jobContext).toArray val result = new Array[Partition](rawSplits.size) @@ -57,6 +60,9 @@ class NewHadoopRDD[K, V]( val attemptId = new TaskAttemptID(jobtrackerId, id, true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) val format = inputFormatClass.newInstance + if (format.isInstanceOf[Configurable]) { + format.asInstanceOf[Configurable].setConf(conf) + } val reader = format.createRecordReader( split.serializableHadoopSplit.value, hadoopAttemptContext) reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) -- cgit v1.2.3