From 163302d26af6ab4b780e7047c417d6199f6c3020 Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 5 May 2014 13:21:05 +0800 Subject: [PATCH 01/88] minor fix Signed-off-by: lirui --- .../scala/org/apache/spark/storage/ShuffleBlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 35910e552fe8..a935f2b6ce6a 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -143,7 +143,7 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { val filename = physicalFileName(shuffleId, bucketId, fileId) blockManager.diskBlockManager.getFile(filename) } - val fileGroup = new ShuffleFileGroup(fileId, shuffleId, files) + val fileGroup = new ShuffleFileGroup(shuffleId, fileId, files) shuffleState.allFileGroups.add(fileGroup) fileGroup } From 3124380ccfd19a38878e05f0af29f80c279a897b Mon Sep 17 00:00:00 2001 From: lirui Date: Thu, 8 May 2014 15:06:57 +0800 Subject: [PATCH 02/88] try to locate the point to remove the barrier --- .../org/apache/spark/scheduler/DAGScheduler.scala | 10 ++++++++++ .../cluster/CoarseGrainedSchedulerBackend.scala | 6 ++++++ 2 files changed, 16 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ff411e24a3d8..68966dd622b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -39,6 +39,7 @@ import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId} import org.apache.spark.util.Utils +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of @@ -890,6 +891,15 @@ class DAGScheduler( submitMissingTasks(stage, jobId) } } + } else { + //ShuffleMap stage not finished yet. Maybe we can remove the stage barrier here. + if (taskScheduler.isInstanceOf[TaskSchedulerImpl] && taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.isInstanceOf[CoarseGrainedSchedulerBackend]) { + val backend = taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.asInstanceOf[CoarseGrainedSchedulerBackend] + if (backend.freeCoreCount.get() > 0) { + logInfo("We have "+backend.totalCoreCount.get()+" CPUs. "+pendingTasks(stage).size+" tasks are running/pending. "+ + backend.freeCoreCount.get()+" cores are free. "+waitingStages.size+" stages are waiting to be submitted. ---lirui") + } + } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index a6d6b3d26a3c..15b8f297f7dd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -46,6 +46,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) + //Also track total free cores + var freeCoreCount = new AtomicInteger(0) val conf = scheduler.sc.conf private val timeout = AkkaUtils.askTimeout(conf) @@ -82,6 +84,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A executorAddress(executorId) = sender.path.address addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) + freeCoreCount.addAndGet(cores) makeOffers() } @@ -90,6 +93,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A if (TaskState.isFinished(state)) { if (executorActor.contains(executorId)) { freeCores(executorId) += scheduler.CPUS_PER_TASK + freeCoreCount.addAndGet(scheduler.CPUS_PER_TASK) makeOffers(executorId) } else { // Ignoring the update since we don't know about the executor. @@ -141,6 +145,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A def launchTasks(tasks: Seq[Seq[TaskDescription]]) { for (task <- tasks.flatten) { freeCores(task.executorId) -= scheduler.CPUS_PER_TASK + freeCoreCount.addAndGet(-scheduler.CPUS_PER_TASK) executorActor(task.executorId) ! LaunchTask(task) } } @@ -157,6 +162,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A totalCores -= executorId freeCores -= executorId totalCoreCount.addAndGet(-numCores) + freeCoreCount.addAndGet(-numCores) scheduler.executorLost(executorId, SlaveLost(reason)) } } From 8e625c0c87a4f5f3d35433f2eb1aca5d1cf09549 Mon Sep 17 00:00:00 2001 From: lirui Date: Thu, 8 May 2014 15:57:52 +0800 Subject: [PATCH 03/88] apply upstream hot fix --- .../examples => examples/streaming}/JavaCustomReceiver.java | 6 ++---- .../examples => examples/streaming}/CustomReceiver.scala | 4 ++-- 2 files changed, 4 insertions(+), 6 deletions(-) rename examples/src/main/java/org/apache/spark/{streaming/examples => examples/streaming}/JavaCustomReceiver.java (98%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/CustomReceiver.scala (97%) diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java similarity index 98% rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java index e36c7800be23..7f558f3ee713 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples; +package org.apache.spark.examples.streaming; import com.google.common.collect.Lists; @@ -48,7 +48,7 @@ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run org.apache.spark.streaming.examples.JavaCustomReceiver local[2] localhost 9999` + * `$ ./run org.apache.spark.examples.streaming.JavaCustomReceiver local[2] localhost 9999` */ public class JavaCustomReceiver extends Receiver { @@ -149,5 +149,3 @@ private void receive() { } } } - - diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala similarity index 97% rename from examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index eebffd824983..e317e2d36ae4 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import java.io.{InputStreamReader, BufferedReader, InputStream} import java.net.Socket @@ -37,7 +37,7 @@ import org.apache.spark.streaming.receiver.Receiver * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run org.apache.spark.streaming.examples.CustomReceiver local[2] localhost 9999` + * `$ ./run org.apache.spark.examples.streaming.CustomReceiver local[2] localhost 9999` */ object CustomReceiver { def main(args: Array[String]) { From 1d5d0f0be263a98387ba763b2298a8eccf3e2c65 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 9 May 2014 21:33:36 +0800 Subject: [PATCH 04/88] RemoveStageBarrier: support partial map outputs --- .../spark/BlockStoreShuffleFetcher.scala | 30 +++++- .../org/apache/spark/MapOutputTracker.scala | 97 +++++++++++-------- .../apache/spark/scheduler/DAGScheduler.scala | 16 ++- 3 files changed, 99 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala index a67392441ed2..519a1cb6a831 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala @@ -39,13 +39,19 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin val blockManager = SparkEnv.get.blockManager val startTime = System.currentTimeMillis - val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, reduceId) + var 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)]] + //track the map outputs we're missing + val missingMapOutputs= new ArrayBuffer[Int]() for (((address, size), index) <- statuses.zipWithIndex) { - splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size)) + if (address != null) { + splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size)) + } else { + missingMapOutputs += index + } } val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = splitsByAddress.toSeq.map { @@ -73,7 +79,25 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } } - val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer) + var blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer) + if (!missingMapOutputs.isEmpty) { + logInfo("We're missing outputs of " + missingMapOutputs.size + " map tasks.Wait for them to finish. ---lirui") + do { + Thread.sleep(8000) + logInfo("Trying to get updated map outputs for reduceId " + reduceId + " ---lirui") + statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, reduceId) + logInfo("Still missing " + statuses.filter(_._1 == null).size + " outputs for reduceId " + reduceId + " ---lirui") + } while (statuses.exists(_._1 == null)) + val missingSplitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] + for (index <- missingMapOutputs) { + missingSplitsByAddress.getOrElseUpdate(statuses(index)._1, ArrayBuffer()) += ((index, statuses(index)._2)) + } + val missingBlocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = missingSplitsByAddress.toSeq.map { + case (address, splits) => + (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) + } + blockFetcherItr = blockFetcherItr ++ blockManager.getMultiple(missingBlocksByAddress, serializer) + } val itr = blockFetcherItr.flatMap(unpackBlock) val completionIter = CompletionIterator[T, Iterator[T]](itr, { diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index ee82d9fa7874..745c10062f9c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -28,6 +28,7 @@ import akka.pattern.ask import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId import org.apache.spark.util._ +import scala.collection.mutable private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int) @@ -83,6 +84,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * On the workers, it simply serves as a cache, in which a miss triggers a fetch from the * master's corresponding HashMap. */ + //TODO: we should also record if the output for a shuffle is partial protected val mapStatuses: Map[Int, Array[MapStatus]] /** @@ -121,6 +123,57 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * a given shuffle. */ def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = { + val statuses = getMapStatusesForShuffle(shuffleId, reduceId) + statuses.synchronized { + MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses) + } + } + + /** Called to get current epoch number. */ + def getEpoch: Long = { + epochLock.synchronized { + return epoch + } + } + + /** + * Called from executors to update the epoch number, potentially clearing old outputs + * because of a fetch failure. Each worker task calls this with the latest epoch + * number on the master at the time it was created. + */ + def updateEpoch(newEpoch: Long) { + epochLock.synchronized { + if (newEpoch > epoch) { + logInfo("Updating epoch to " + newEpoch + " and clearing cache") + epoch = newEpoch + mapStatuses.clear() + } + } + } + + /** Unregister shuffle data. */ + def unregisterShuffle(shuffleId: Int) { + mapStatuses.remove(shuffleId) + } + + /** Stop the tracker. */ + def stop() { } + + //check if the map output for a shuffle is partial + def partialOutputForShuffle(shuffleId: Int) = { + if (mapStatuses.get(shuffleId).isDefined) { + mapStatuses.get(shuffleId).get.exists(_ == null) + } else { + false + } + } + + //get map statuses for a shuffle + def getMapStatusesForShuffle(shuffleId: Int, reduceId: Int): Array[MapStatus]={ + //we remove the previously fetched outputs if it's partial + if (partialOutputForShuffle(shuffleId)) { + mapStatuses -= shuffleId + } val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") @@ -164,49 +217,15 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } } if (fetchedStatuses != null) { - fetchedStatuses.synchronized { - return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, fetchedStatuses) - } + fetchedStatuses } else { throw new FetchFailedException(null, shuffleId, -1, reduceId, new Exception("Missing all output locations for shuffle " + shuffleId)) } } else { - statuses.synchronized { - return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses) - } - } - } - - /** Called to get current epoch number. */ - def getEpoch: Long = { - epochLock.synchronized { - return epoch + statuses } } - - /** - * Called from executors to update the epoch number, potentially clearing old outputs - * because of a fetch failure. Each worker task calls this with the latest epoch - * number on the master at the time it was created. - */ - def updateEpoch(newEpoch: Long) { - epochLock.synchronized { - if (newEpoch > epoch) { - logInfo("Updating epoch to " + newEpoch + " and clearing cache") - epoch = newEpoch - mapStatuses.clear() - } - } - } - - /** Unregister shuffle data. */ - def unregisterShuffle(shuffleId: Int) { - mapStatuses.remove(shuffleId) - } - - /** Stop the tracker. */ - def stop() { } } /** @@ -371,8 +390,10 @@ private[spark] object MapOutputTracker { statuses.map { status => if (status == null) { - throw new FetchFailedException(null, shuffleId, -1, reduceId, - new Exception("Missing an output location for shuffle " + shuffleId)) + //TODO: need to distinguish whether this is due to failed map tasks or partial outputs + (null, 0) +// throw new FetchFailedException(null, shuffleId, -1, reduceId, +// new Exception("Missing an output location for shuffle " + shuffleId)) } else { (status.location, decompressSize(status.compressedSizes(reduceId))) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 68966dd622b8..38b489104e02 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -895,9 +895,19 @@ class DAGScheduler( //ShuffleMap stage not finished yet. Maybe we can remove the stage barrier here. if (taskScheduler.isInstanceOf[TaskSchedulerImpl] && taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.isInstanceOf[CoarseGrainedSchedulerBackend]) { val backend = taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.asInstanceOf[CoarseGrainedSchedulerBackend] - if (backend.freeCoreCount.get() > 0) { - logInfo("We have "+backend.totalCoreCount.get()+" CPUs. "+pendingTasks(stage).size+" tasks are running/pending. "+ - backend.freeCoreCount.get()+" cores are free. "+waitingStages.size+" stages are waiting to be submitted. ---lirui") + //there are free cores and waiting stages + if (backend.freeCoreCount.get() > 0 && waitingStages.size > 0 && stage.shuffleDep.isDefined) { + logInfo("We have " + backend.totalCoreCount.get() + " CPUs. " + pendingTasks(stage).size + " tasks are running/pending. " + + backend.freeCoreCount.get() + " cores are free. " + waitingStages.size + " stages are waiting to be submitted. ---lirui") + //TODO: find a waiting stage that depends on the current "stage" + val preStartedStage = waitingStages.head + val shuffleId = stage.shuffleDep.get.shuffleId + logInfo("Register partial map outputs for shuffleId " + shuffleId + " ---lirui") + mapOutputTracker.registerMapOutputs(shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray) + logInfo("Pre-start stage " + preStartedStage.id + " ---lirui") + waitingStages -= preStartedStage + runningStages += preStartedStage + submitMissingTasks(preStartedStage, activeJobForStage(preStartedStage)) } } } From c4f405446739280a3e02e70854d88f08498d8447 Mon Sep 17 00:00:00 2001 From: lirui Date: Sun, 11 May 2014 14:50:40 +0800 Subject: [PATCH 05/88] RemoveStageBarrier: build fix --- .../org/apache/spark/BlockStoreShuffleFetcher.scala | 9 +++++---- .../main/scala/org/apache/spark/MapOutputTracker.scala | 2 +- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 ++- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala index 519a1cb6a831..22726584018d 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.HashMap import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.serializer.Serializer -import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId} +import org.apache.spark.storage.{BlockFetcherIterator, BlockId, BlockManagerId, ShuffleBlockId} import org.apache.spark.util.CompletionIterator private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { @@ -79,7 +79,8 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } } - var blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer) + val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer) + var itr = blockFetcherItr.flatMap(unpackBlock) if (!missingMapOutputs.isEmpty) { logInfo("We're missing outputs of " + missingMapOutputs.size + " map tasks.Wait for them to finish. ---lirui") do { @@ -96,9 +97,9 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin case (address, splits) => (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) } - blockFetcherItr = blockFetcherItr ++ blockManager.getMultiple(missingBlocksByAddress, serializer) + val missingBlockFetcherItr = blockManager.getMultiple(missingBlocksByAddress, serializer) + itr = itr ++ missingBlockFetcherItr.flatMap(unpackBlock) } - val itr = blockFetcherItr.flatMap(unpackBlock) val completionIter = CompletionIterator[T, Iterator[T]](itr, { val shuffleMetrics = new ShuffleReadMetrics diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 745c10062f9c..b5edc57105a9 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -391,7 +391,7 @@ private[spark] object MapOutputTracker { status => if (status == null) { //TODO: need to distinguish whether this is due to failed map tasks or partial outputs - (null, 0) + (null, 0.toLong) // throw new FetchFailedException(null, shuffleId, -1, reduceId, // new Exception("Missing an output location for shuffle " + shuffleId)) } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 38b489104e02..8f7913a2ee52 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -893,6 +893,7 @@ class DAGScheduler( } } else { //ShuffleMap stage not finished yet. Maybe we can remove the stage barrier here. + //TODO: need a better way to get the number of free CPUs if (taskScheduler.isInstanceOf[TaskSchedulerImpl] && taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.isInstanceOf[CoarseGrainedSchedulerBackend]) { val backend = taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.asInstanceOf[CoarseGrainedSchedulerBackend] //there are free cores and waiting stages @@ -907,7 +908,7 @@ class DAGScheduler( logInfo("Pre-start stage " + preStartedStage.id + " ---lirui") waitingStages -= preStartedStage runningStages += preStartedStage - submitMissingTasks(preStartedStage, activeJobForStage(preStartedStage)) + submitMissingTasks(preStartedStage, activeJobForStage(preStartedStage).get) } } } From 444d2d96de39edc5399276fa2efec57638f10462 Mon Sep 17 00:00:00 2001 From: lirui Date: Sun, 11 May 2014 21:54:54 +0800 Subject: [PATCH 06/88] RemoveStageBarrier: register map outputs progressively --- .../spark/BlockStoreShuffleFetcher.scala | 43 +++++++++---------- .../apache/spark/scheduler/DAGScheduler.scala | 42 +++++++++++------- 2 files changed, 47 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala index 22726584018d..23118fd6f820 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala @@ -44,15 +44,12 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin shuffleId, reduceId, System.currentTimeMillis - startTime)) val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] - //track the map outputs we're missing - val missingMapOutputs= new ArrayBuffer[Int]() - for (((address, size), index) <- statuses.zipWithIndex) { - if (address != null) { - splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size)) - } else { - missingMapOutputs += index - } + + for (((address, size), index) <- statuses.zipWithIndex if address != null) { + splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size)) } + //track the map outputs we're missing + var missingMapOutputs = statuses.zipWithIndex.filter(_._1._1 == null).map(_._2) val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = splitsByAddress.toSeq.map { case (address, splits) => @@ -81,24 +78,26 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer) var itr = blockFetcherItr.flatMap(unpackBlock) - if (!missingMapOutputs.isEmpty) { - logInfo("We're missing outputs of " + missingMapOutputs.size + " map tasks.Wait for them to finish. ---lirui") - do { - Thread.sleep(8000) - logInfo("Trying to get updated map outputs for reduceId " + reduceId + " ---lirui") - statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, reduceId) - logInfo("Still missing " + statuses.filter(_._1 == null).size + " outputs for reduceId " + reduceId + " ---lirui") - } while (statuses.exists(_._1 == null)) + + while(!missingMapOutputs.isEmpty){ + logInfo("Still missing "+missingMapOutputs.size+" outputs for reduceId "+reduceId+" ---lirui") + Thread.sleep(8000) + logInfo("Trying to update map output statues for reduceId "+reduceId+" ---lirui") + statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, reduceId) val missingSplitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] - for (index <- missingMapOutputs) { + for (index <- missingMapOutputs if statuses(index)._1 != null) { missingSplitsByAddress.getOrElseUpdate(statuses(index)._1, ArrayBuffer()) += ((index, statuses(index)._2)) } - val missingBlocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = missingSplitsByAddress.toSeq.map { - case (address, splits) => - (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) + //we have new outputs ready for this reduce + if(!missingSplitsByAddress.isEmpty){ + val missingBlocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = missingSplitsByAddress.toSeq.map { + case (address, splits) => + (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) + } + val missingBlockFetcherItr = blockManager.getMultiple(missingBlocksByAddress, serializer) + itr = itr ++ missingBlockFetcherItr.flatMap(unpackBlock) } - val missingBlockFetcherItr = blockManager.getMultiple(missingBlocksByAddress, serializer) - itr = itr ++ missingBlockFetcherItr.flatMap(unpackBlock) + missingMapOutputs = statuses.zipWithIndex.filter(_._1._1 == null).map(_._2) } val completionIter = CompletionIterator[T, Iterator[T]](itr, { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 8f7913a2ee52..78c52616b832 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -121,6 +121,9 @@ class DAGScheduler( private[scheduler] var eventProcessActor: ActorRef = _ + //whether to enable remove stage barrier + val removeStageBarrier = env.conf.getBoolean("spark.schedule.removeStageBarrier", true) + private def initializeEventProcessActor() { // blocking the thread until supervisor is started, which ensures eventProcessActor is // not null before any job is submitted @@ -847,6 +850,11 @@ class DAGScheduler( } else { stage.addOutputLoc(smt.partitionId, status) } + //we need to register map outputs progressively if remove stage barrier is enabled + if (removeStageBarrier && stage.shuffleDep.isDefined) { + logInfo("Register output progressively: Map task "+smt.partitionId+" ---lirui") + mapOutputTracker.registerMapOutput(stage.shuffleDep.get.shuffleId, smt.partitionId, status) + } if (runningStages.contains(stage) && pendingTasks(stage).isEmpty) { markStageAsFinished(stage) logInfo("looking for newly runnable stages") @@ -893,22 +901,24 @@ class DAGScheduler( } } else { //ShuffleMap stage not finished yet. Maybe we can remove the stage barrier here. - //TODO: need a better way to get the number of free CPUs - if (taskScheduler.isInstanceOf[TaskSchedulerImpl] && taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.isInstanceOf[CoarseGrainedSchedulerBackend]) { - val backend = taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.asInstanceOf[CoarseGrainedSchedulerBackend] - //there are free cores and waiting stages - if (backend.freeCoreCount.get() > 0 && waitingStages.size > 0 && stage.shuffleDep.isDefined) { - logInfo("We have " + backend.totalCoreCount.get() + " CPUs. " + pendingTasks(stage).size + " tasks are running/pending. " + - backend.freeCoreCount.get() + " cores are free. " + waitingStages.size + " stages are waiting to be submitted. ---lirui") - //TODO: find a waiting stage that depends on the current "stage" - val preStartedStage = waitingStages.head - val shuffleId = stage.shuffleDep.get.shuffleId - logInfo("Register partial map outputs for shuffleId " + shuffleId + " ---lirui") - mapOutputTracker.registerMapOutputs(shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray) - logInfo("Pre-start stage " + preStartedStage.id + " ---lirui") - waitingStages -= preStartedStage - runningStages += preStartedStage - submitMissingTasks(preStartedStage, activeJobForStage(preStartedStage).get) + if(removeStageBarrier){ + //TODO: need a better way to get the number of free CPUs + if (taskScheduler.isInstanceOf[TaskSchedulerImpl] && taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.isInstanceOf[CoarseGrainedSchedulerBackend]) { + val backend = taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.asInstanceOf[CoarseGrainedSchedulerBackend] + //there are free cores and waiting stages + if (backend.freeCoreCount.get() > 0 && waitingStages.size > 0 && stage.shuffleDep.isDefined) { + logInfo("We have " + backend.totalCoreCount.get() + " CPUs. " + pendingTasks(stage).size + " tasks are running/pending. " + + backend.freeCoreCount.get() + " cores are free. " + waitingStages.size + " stages are waiting to be submitted. ---lirui") + //TODO: find a waiting stage that depends on the current "stage" + val preStartedStage = waitingStages.head + val shuffleId = stage.shuffleDep.get.shuffleId + logInfo("Register partial map outputs for shuffleId " + shuffleId + " ---lirui") + mapOutputTracker.registerMapOutputs(shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray) + logInfo("Pre-start stage " + preStartedStage.id + " ---lirui") + waitingStages -= preStartedStage + runningStages += preStartedStage + submitMissingTasks(preStartedStage, activeJobForStage(preStartedStage).get) + } } } } From 2df1d4e5cab7ba36443425a6f7b54a6ed06f519f Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 12 May 2014 10:30:39 +0800 Subject: [PATCH 07/88] RemoveStageBarrier: increment epoch for progressive registration --- .../scala/org/apache/spark/MapOutputTracker.scala | 11 +++++++---- .../org/apache/spark/scheduler/DAGScheduler.scala | 9 ++++++--- 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index b5edc57105a9..a31bbb6e04f3 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -160,7 +160,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging def stop() { } //check if the map output for a shuffle is partial - def partialOutputForShuffle(shuffleId: Int) = { + private def partialOutputForShuffle(shuffleId: Int) = { if (mapStatuses.get(shuffleId).isDefined) { mapStatuses.get(shuffleId).get.exists(_ == null) } else { @@ -170,9 +170,12 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging //get map statuses for a shuffle def getMapStatusesForShuffle(shuffleId: Int, reduceId: Int): Array[MapStatus]={ - //we remove the previously fetched outputs if it's partial - if (partialOutputForShuffle(shuffleId)) { - mapStatuses -= shuffleId + mapStatuses.synchronized { + //we remove the previously fetched outputs if it's partial + if (partialOutputForShuffle(shuffleId) && !this.isInstanceOf[MapOutputTrackerMaster]) { + logInfo("We've cached partial map output statuses, will clear the cache. ---lirui") + mapStatuses -= shuffleId + } } val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 78c52616b832..88536c775bb5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -854,6 +854,8 @@ class DAGScheduler( if (removeStageBarrier && stage.shuffleDep.isDefined) { logInfo("Register output progressively: Map task "+smt.partitionId+" ---lirui") mapOutputTracker.registerMapOutput(stage.shuffleDep.get.shuffleId, smt.partitionId, status) + //need to increment the mapoutputtrackermaster's epoch so that it will clear the cache + mapOutputTracker.incrementEpoch() } if (runningStages.contains(stage) && pendingTasks(stage).isEmpty) { markStageAsFinished(stage) @@ -911,9 +913,10 @@ class DAGScheduler( backend.freeCoreCount.get() + " cores are free. " + waitingStages.size + " stages are waiting to be submitted. ---lirui") //TODO: find a waiting stage that depends on the current "stage" val preStartedStage = waitingStages.head - val shuffleId = stage.shuffleDep.get.shuffleId - logInfo("Register partial map outputs for shuffleId " + shuffleId + " ---lirui") - mapOutputTracker.registerMapOutputs(shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray) + //map outputs should have been registered progressively +// val shuffleId = stage.shuffleDep.get.shuffleId +// logInfo("Register partial map outputs for shuffleId " + shuffleId + " ---lirui") +// mapOutputTracker.registerMapOutputs(shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray) logInfo("Pre-start stage " + preStartedStage.id + " ---lirui") waitingStages -= preStartedStage runningStages += preStartedStage From 9f18dc74a25d34dfda923e3f1064cc339110e9a9 Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 12 May 2014 16:39:10 +0800 Subject: [PATCH 08/88] RemoveStageBarrier: fix check free CPUs --- .../org/apache/spark/scheduler/DAGScheduler.scala | 11 +++++++---- .../cluster/CoarseGrainedSchedulerBackend.scala | 6 ------ .../apache/spark/storage/BlockFetcherIterator.scala | 1 + 3 files changed, 8 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 88536c775bb5..12268cf9dc32 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -907,10 +907,13 @@ class DAGScheduler( //TODO: need a better way to get the number of free CPUs if (taskScheduler.isInstanceOf[TaskSchedulerImpl] && taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.isInstanceOf[CoarseGrainedSchedulerBackend]) { val backend = taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.asInstanceOf[CoarseGrainedSchedulerBackend] - //there are free cores and waiting stages - if (backend.freeCoreCount.get() > 0 && waitingStages.size > 0 && stage.shuffleDep.isDefined) { - logInfo("We have " + backend.totalCoreCount.get() + " CPUs. " + pendingTasks(stage).size + " tasks are running/pending. " + - backend.freeCoreCount.get() + " cores are free. " + waitingStages.size + " stages are waiting to be submitted. ---lirui") + //check CPU usage + val totalCores = backend.totalCoreCount.get() + val waitingStageNum = waitingStages.size + val pendingTaskNum = (for(taskSet <- pendingTasks.values) yield taskSet.size).sum + if (pendingTaskNum < totalCores && waitingStageNum > 0 && stage.shuffleDep.isDefined) { + logInfo("We have " + totalCores + " CPUs. " + pendingTaskNum + " tasks are running/pending. " + + waitingStageNum + " stages are waiting to be submitted. ---lirui") //TODO: find a waiting stage that depends on the current "stage" val preStartedStage = waitingStages.head //map outputs should have been registered progressively diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 15b8f297f7dd..a6d6b3d26a3c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -46,8 +46,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) - //Also track total free cores - var freeCoreCount = new AtomicInteger(0) val conf = scheduler.sc.conf private val timeout = AkkaUtils.askTimeout(conf) @@ -84,7 +82,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A executorAddress(executorId) = sender.path.address addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) - freeCoreCount.addAndGet(cores) makeOffers() } @@ -93,7 +90,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A if (TaskState.isFinished(state)) { if (executorActor.contains(executorId)) { freeCores(executorId) += scheduler.CPUS_PER_TASK - freeCoreCount.addAndGet(scheduler.CPUS_PER_TASK) makeOffers(executorId) } else { // Ignoring the update since we don't know about the executor. @@ -145,7 +141,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A def launchTasks(tasks: Seq[Seq[TaskDescription]]) { for (task <- tasks.flatten) { freeCores(task.executorId) -= scheduler.CPUS_PER_TASK - freeCoreCount.addAndGet(-scheduler.CPUS_PER_TASK) executorActor(task.executorId) ! LaunchTask(task) } } @@ -162,7 +157,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A totalCores -= executorId freeCores -= executorId totalCoreCount.addAndGet(-numCores) - freeCoreCount.addAndGet(-numCores) scheduler.executorLost(executorId, SlaveLost(reason)) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 408a79708805..94ff94a6f12a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -135,6 +135,7 @@ object BlockFetcherIterator { _remoteBytesRead += networkSize logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } + logInfo("Got "+blockMessageArray.length+" remote blocks in "+Utils.getUsedTimeMs(startTime)+" ---lirui") } case None => { logError("Could not get block(s) from " + cmId) From 7af23c0fa134ac329d2ee4fa8813e1deb13b1ddd Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 13 May 2014 16:30:41 +0800 Subject: [PATCH 09/88] RemoveStageBarrier: make reducers refresh map outputs less often --- .../apache/spark/BlockStoreShuffleFetcher.scala | 10 ++++++++-- .../org/apache/spark/MapOutputTracker.scala | 17 +++++++++++++---- .../apache/spark/scheduler/DAGScheduler.scala | 8 +++++--- 3 files changed, 26 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala index 23118fd6f820..b489f17770dc 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala @@ -79,9 +79,13 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer) var itr = blockFetcherItr.flatMap(unpackBlock) + //time interval(in second) the thread should sleep + var sleepInterval = 8.toFloat while(!missingMapOutputs.isEmpty){ - logInfo("Still missing "+missingMapOutputs.size+" outputs for reduceId "+reduceId+" ---lirui") - Thread.sleep(8000) + val oldMissingNum = missingMapOutputs.size + logInfo("Still missing " + oldMissingNum + " outputs for reduceId " + reduceId + + ". Sleep " + sleepInterval + "s. ---lirui") + Thread.sleep((sleepInterval * 1000).toLong) logInfo("Trying to update map output statues for reduceId "+reduceId+" ---lirui") statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, reduceId) val missingSplitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] @@ -98,6 +102,8 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin itr = itr ++ missingBlockFetcherItr.flatMap(unpackBlock) } missingMapOutputs = statuses.zipWithIndex.filter(_._1._1 == null).map(_._2) + val fillingUpSpeed = (oldMissingNum - missingMapOutputs.size).toFloat / sleepInterval + sleepInterval = if (fillingUpSpeed > 0.01) math.max(10.toFloat, missingMapOutputs.size.toFloat / 5) / fillingUpSpeed else sleepInterval * 2 } val completionIter = CompletionIterator[T, Iterator[T]](itr, { diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index a31bbb6e04f3..4830496ee83a 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -34,6 +34,7 @@ private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage +private[spark] case object GetMasterEpoch extends MapOutputTrackerMessage /** Actor class for MapOutputTrackerMaster */ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster, conf: SparkConf) @@ -63,6 +64,9 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster logInfo("MapOutputTrackerActor stopped!") sender ! true context.stop(self) + + case GetMasterEpoch => + sender ! tracker.getEpoch } } @@ -144,7 +148,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging def updateEpoch(newEpoch: Long) { epochLock.synchronized { if (newEpoch > epoch) { - logInfo("Updating epoch to " + newEpoch + " and clearing cache") + logInfo("Updating epoch from "+epoch+" to " + newEpoch + " and clearing cache") epoch = newEpoch mapStatuses.clear() } @@ -171,10 +175,15 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging //get map statuses for a shuffle def getMapStatusesForShuffle(shuffleId: Int, reduceId: Int): Array[MapStatus]={ mapStatuses.synchronized { - //we remove the previously fetched outputs if it's partial + //we cached partial map outputs, the master may have updates for us if (partialOutputForShuffle(shuffleId) && !this.isInstanceOf[MapOutputTrackerMaster]) { - logInfo("We've cached partial map output statuses, will clear the cache. ---lirui") - mapStatuses -= shuffleId + val masterEpoch = askTracker(GetMasterEpoch).asInstanceOf[Long] + //we don't want to clear the local cache too often + if (masterEpoch - epoch >= math.min(10, mapStatuses.get(shuffleId).get.filter(_ == null).size)) { + logInfo("Master's epoch is " + masterEpoch + ", local epoch is " + epoch + ". Clear local cache. ---lirui") + mapStatuses -= shuffleId + epoch = masterEpoch + } } } val statuses = mapStatuses.get(shuffleId).orNull diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 12268cf9dc32..d8d2ba8b4721 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -904,14 +904,16 @@ class DAGScheduler( } else { //ShuffleMap stage not finished yet. Maybe we can remove the stage barrier here. if(removeStageBarrier){ - //TODO: need a better way to get the number of free CPUs + //TODO: need a better way to get the number of total CPUs if (taskScheduler.isInstanceOf[TaskSchedulerImpl] && taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.isInstanceOf[CoarseGrainedSchedulerBackend]) { val backend = taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.asInstanceOf[CoarseGrainedSchedulerBackend] //check CPU usage val totalCores = backend.totalCoreCount.get() + val pendingTaskNum = (for (taskSet <- pendingTasks.values) yield taskSet.size).sum + val freeCores = totalCores - pendingTaskNum val waitingStageNum = waitingStages.size - val pendingTaskNum = (for(taskSet <- pendingTasks.values) yield taskSet.size).sum - if (pendingTaskNum < totalCores && waitingStageNum > 0 && stage.shuffleDep.isDefined) { + //we pre-start the waiting stage only if there're enough free cores, e.g. 25% is free + if (4 * freeCores >= totalCores && waitingStageNum > 0 && stage.shuffleDep.isDefined) { logInfo("We have " + totalCores + " CPUs. " + pendingTaskNum + " tasks are running/pending. " + waitingStageNum + " stages are waiting to be submitted. ---lirui") //TODO: find a waiting stage that depends on the current "stage" From 9a32a17d0f2620ef807aa3d1ed26df39038bf3af Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 13 May 2014 21:00:20 +0800 Subject: [PATCH 10/88] RemoveStageBarrier: start reducers earlier --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index d8d2ba8b4721..06de00ca2419 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -912,16 +912,11 @@ class DAGScheduler( val pendingTaskNum = (for (taskSet <- pendingTasks.values) yield taskSet.size).sum val freeCores = totalCores - pendingTaskNum val waitingStageNum = waitingStages.size - //we pre-start the waiting stage only if there're enough free cores, e.g. 25% is free - if (4 * freeCores >= totalCores && waitingStageNum > 0 && stage.shuffleDep.isDefined) { + if (freeCores > 0 && waitingStageNum > 0) { logInfo("We have " + totalCores + " CPUs. " + pendingTaskNum + " tasks are running/pending. " + waitingStageNum + " stages are waiting to be submitted. ---lirui") //TODO: find a waiting stage that depends on the current "stage" val preStartedStage = waitingStages.head - //map outputs should have been registered progressively -// val shuffleId = stage.shuffleDep.get.shuffleId -// logInfo("Register partial map outputs for shuffleId " + shuffleId + " ---lirui") -// mapOutputTracker.registerMapOutputs(shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray) logInfo("Pre-start stage " + preStartedStage.id + " ---lirui") waitingStages -= preStartedStage runningStages += preStartedStage From 9ffb208e0c32990f7919589c091124d41de51f7e Mon Sep 17 00:00:00 2001 From: lirui Date: Wed, 14 May 2014 11:21:59 +0800 Subject: [PATCH 11/88] RemoveStageBarrier: add log info --- .../spark/BlockStoreShuffleFetcher.scala | 8 +++-- .../org/apache/spark/MapOutputTracker.scala | 32 +++++++++++-------- .../apache/spark/scheduler/DAGScheduler.scala | 11 +++++-- .../spark/storage/BlockFetcherIterator.scala | 8 ++++- 4 files changed, 41 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala index b489f17770dc..745623a65b61 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala @@ -37,9 +37,10 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) val blockManager = SparkEnv.get.blockManager + val mapOutputTracker = SparkEnv.get.mapOutputTracker val startTime = System.currentTimeMillis - var statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, reduceId) + var statuses = mapOutputTracker.getServerStatuses(shuffleId, reduceId) logDebug("Fetching map output location for shuffle %d, reduce %d took %d ms".format( shuffleId, reduceId, System.currentTimeMillis - startTime)) @@ -87,7 +88,8 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin ". Sleep " + sleepInterval + "s. ---lirui") Thread.sleep((sleepInterval * 1000).toLong) logInfo("Trying to update map output statues for reduceId "+reduceId+" ---lirui") - statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, reduceId) + mapOutputTracker.updateMapStatusesForShuffle(shuffleId) + statuses = mapOutputTracker.getServerStatuses(shuffleId, reduceId) val missingSplitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] for (index <- missingMapOutputs if statuses(index)._1 != null) { missingSplitsByAddress.getOrElseUpdate(statuses(index)._1, ArrayBuffer()) += ((index, statuses(index)._2)) @@ -100,6 +102,8 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } val missingBlockFetcherItr = blockManager.getMultiple(missingBlocksByAddress, serializer) itr = itr ++ missingBlockFetcherItr.flatMap(unpackBlock) + } else { + logInfo("No updates in the previous interval "+sleepInterval+"s, sleep longer. ---lirui") } missingMapOutputs = statuses.zipWithIndex.filter(_._1._1 == null).map(_._2) val fillingUpSpeed = (oldMissingNum - missingMapOutputs.size).toFloat / sleepInterval diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 4830496ee83a..66d03d98934c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -173,19 +173,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } //get map statuses for a shuffle - def getMapStatusesForShuffle(shuffleId: Int, reduceId: Int): Array[MapStatus]={ - mapStatuses.synchronized { - //we cached partial map outputs, the master may have updates for us - if (partialOutputForShuffle(shuffleId) && !this.isInstanceOf[MapOutputTrackerMaster]) { - val masterEpoch = askTracker(GetMasterEpoch).asInstanceOf[Long] - //we don't want to clear the local cache too often - if (masterEpoch - epoch >= math.min(10, mapStatuses.get(shuffleId).get.filter(_ == null).size)) { - logInfo("Master's epoch is " + masterEpoch + ", local epoch is " + epoch + ". Clear local cache. ---lirui") - mapStatuses -= shuffleId - epoch = masterEpoch - } - } - } + private def getMapStatusesForShuffle(shuffleId: Int, reduceId: Int): Array[MapStatus]={ val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") @@ -238,6 +226,24 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging statuses } } + + //update partial map outputs for a shuffle + def updateMapStatusesForShuffle(shuffleId: Int){ + mapStatuses.synchronized { + //we cached partial map outputs, the master may have updates for us + if (partialOutputForShuffle(shuffleId) && !this.isInstanceOf[MapOutputTrackerMaster]) { + val masterEpoch = askTracker(GetMasterEpoch).asInstanceOf[Long] + //we don't want to clear the local cache too often + if (masterEpoch - epoch >= math.min(10, mapStatuses.get(shuffleId).get.filter(_ == null).size)) { + logInfo("Master's epoch is " + masterEpoch + ", local epoch is " + epoch + ". Clear local cache. ---lirui") + mapStatuses -= shuffleId + epoch = masterEpoch + } + } else { + logInfo("The map output is not partial. ---lirui") + } + } + } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 06de00ca2419..b2b3f825e8da 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -40,6 +40,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId} import org.apache.spark.util.Utils import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import scala.collection.mutable /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of @@ -123,6 +124,8 @@ class DAGScheduler( //whether to enable remove stage barrier val removeStageBarrier = env.conf.getBoolean("spark.schedule.removeStageBarrier", true) + //whether there's pre-started stages depending on this stage + val dependantStagePreStarted = new mutable.HashSet[Stage]() private def initializeEventProcessActor() { // blocking the thread until supervisor is started, which ensures eventProcessActor is @@ -851,13 +854,14 @@ class DAGScheduler( stage.addOutputLoc(smt.partitionId, status) } //we need to register map outputs progressively if remove stage barrier is enabled - if (removeStageBarrier && stage.shuffleDep.isDefined) { + if (removeStageBarrier && dependantStagePreStarted.contains(stage) && stage.shuffleDep.isDefined) { logInfo("Register output progressively: Map task "+smt.partitionId+" ---lirui") mapOutputTracker.registerMapOutput(stage.shuffleDep.get.shuffleId, smt.partitionId, status) //need to increment the mapoutputtrackermaster's epoch so that it will clear the cache mapOutputTracker.incrementEpoch() } if (runningStages.contains(stage) && pendingTasks(stage).isEmpty) { + dependantStagePreStarted -= stage markStageAsFinished(stage) logInfo("looking for newly runnable stages") logInfo("running: " + runningStages) @@ -912,14 +916,17 @@ class DAGScheduler( val pendingTaskNum = (for (taskSet <- pendingTasks.values) yield taskSet.size).sum val freeCores = totalCores - pendingTaskNum val waitingStageNum = waitingStages.size - if (freeCores > 0 && waitingStageNum > 0) { + if (freeCores > 0 && waitingStageNum > 0 && stage.shuffleDep.isDefined) { logInfo("We have " + totalCores + " CPUs. " + pendingTaskNum + " tasks are running/pending. " + waitingStageNum + " stages are waiting to be submitted. ---lirui") //TODO: find a waiting stage that depends on the current "stage" val preStartedStage = waitingStages.head logInfo("Pre-start stage " + preStartedStage.id + " ---lirui") + //register map output finished so far + mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, true) waitingStages -= preStartedStage runningStages += preStartedStage + dependantStagePreStarted += stage submitMissingTasks(preStartedStage, activeJobForStage(preStartedStage).get) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 94ff94a6f12a..5836d4e8ee32 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -192,7 +192,7 @@ object BlockFetcherIterator { } } } - logInfo("Getting " + _numBlocksToFetch + " non-empty blocks out of " + + logDebug("Getting " + _numBlocksToFetch + " non-empty blocks out of " + totalBlocks + " blocks") remoteRequests } @@ -221,12 +221,18 @@ object BlockFetcherIterator { // Add the remote requests into our queue in a random order fetchRequests ++= Utils.randomize(remoteRequests) + //how many remote blocks we're fetching during initialization + var initialRemoteBlocks = 0 + // Send out initial requests for blocks, up to our maxBytesInFlight while (!fetchRequests.isEmpty && (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + initialRemoteBlocks += fetchRequests.front.blocks.size sendRequest(fetchRequests.dequeue()) } + logInfo("Fetching "+initialRemoteBlocks+" out of "+remoteBlocksToFetch.size+" remote blocks for initialization. ---lirui") + val numFetches = remoteRequests.size - fetchRequests.size logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) From ef3b04323bcdeadab58d02b9076a14b08e130436 Mon Sep 17 00:00:00 2001 From: lirui Date: Wed, 14 May 2014 16:45:03 +0800 Subject: [PATCH 12/88] RemoveStageBarrier: adjust sleep interval --- .../org/apache/spark/BlockStoreShuffleFetcher.scala | 4 ++-- .../main/scala/org/apache/spark/MapOutputTracker.scala | 9 +++------ 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala index 745623a65b61..8d42aa9128e2 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala @@ -87,7 +87,7 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin logInfo("Still missing " + oldMissingNum + " outputs for reduceId " + reduceId + ". Sleep " + sleepInterval + "s. ---lirui") Thread.sleep((sleepInterval * 1000).toLong) - logInfo("Trying to update map output statues for reduceId "+reduceId+" ---lirui") + logInfo("Trying to update map output statuses for reduceId "+reduceId+" ---lirui") mapOutputTracker.updateMapStatusesForShuffle(shuffleId) statuses = mapOutputTracker.getServerStatuses(shuffleId, reduceId) val missingSplitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] @@ -107,7 +107,7 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } missingMapOutputs = statuses.zipWithIndex.filter(_._1._1 == null).map(_._2) val fillingUpSpeed = (oldMissingNum - missingMapOutputs.size).toFloat / sleepInterval - sleepInterval = if (fillingUpSpeed > 0.01) math.max(10.toFloat, missingMapOutputs.size.toFloat / 5) / fillingUpSpeed else sleepInterval * 2 + sleepInterval = if (fillingUpSpeed > 0.01) math.max(10.toFloat, missingMapOutputs.size.toFloat / 2) / fillingUpSpeed else sleepInterval * 4 } val completionIter = CompletionIterator[T, Iterator[T]](itr, { diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 66d03d98934c..57ea626381d7 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -230,17 +230,14 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging //update partial map outputs for a shuffle def updateMapStatusesForShuffle(shuffleId: Int){ mapStatuses.synchronized { - //we cached partial map outputs, the master may have updates for us - if (partialOutputForShuffle(shuffleId) && !this.isInstanceOf[MapOutputTrackerMaster]) { + //we may have cached partial map outputs, the master may have updates for us + if (mapStatuses.get(shuffleId).isDefined) { val masterEpoch = askTracker(GetMasterEpoch).asInstanceOf[Long] - //we don't want to clear the local cache too often - if (masterEpoch - epoch >= math.min(10, mapStatuses.get(shuffleId).get.filter(_ == null).size)) { + if (masterEpoch > epoch) { logInfo("Master's epoch is " + masterEpoch + ", local epoch is " + epoch + ". Clear local cache. ---lirui") mapStatuses -= shuffleId epoch = masterEpoch } - } else { - logInfo("The map output is not partial. ---lirui") } } } From 4213d63be930a7f39fca5667855b2862e52d3487 Mon Sep 17 00:00:00 2001 From: lirui Date: Thu, 15 May 2014 19:24:25 +0800 Subject: [PATCH 13/88] RemoveStageBarrier: add a new iterator to manage partial map outputs --- .../spark/BlockStoreShuffleFetcher.scala | 84 ++++++++--------- .../org/apache/spark/MapOutputTracker.scala | 3 +- .../spark/storage/BlockFetcherIterator.scala | 89 ++++++++++++++++++- .../apache/spark/storage/BlockManager.scala | 11 +++ 4 files changed, 140 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala index 8d42aa9128e2..c77f0286a5a5 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala @@ -40,21 +40,22 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin val mapOutputTracker = SparkEnv.get.mapOutputTracker val startTime = System.currentTimeMillis - var statuses = mapOutputTracker.getServerStatuses(shuffleId, reduceId) + val statuses = 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 if address != null) { - splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size)) - } - //track the map outputs we're missing - var missingMapOutputs = statuses.zipWithIndex.filter(_._1._1 == null).map(_._2) - - val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = splitsByAddress.toSeq.map { - case (address, splits) => - (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) + val blockFetcherItr = if (statuses.exists(_._1 == null)) { + blockManager.getPartial(statuses, mapOutputTracker, serializer, shuffleId, reduceId) + } else { + val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] + for (((address, size), index) <- statuses.zipWithIndex) { + splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size)) + } + val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = splitsByAddress.toSeq.map { + case (address, splits) => + (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) + } + blockManager.getMultiple(blocksByAddress, serializer) } def unpackBlock(blockPair: (BlockId, Option[Iterator[Any]])) : Iterator[T] = { @@ -77,38 +78,37 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } } - val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer) - var itr = blockFetcherItr.flatMap(unpackBlock) + val itr = blockFetcherItr.flatMap(unpackBlock) //time interval(in second) the thread should sleep - var sleepInterval = 8.toFloat - while(!missingMapOutputs.isEmpty){ - val oldMissingNum = missingMapOutputs.size - logInfo("Still missing " + oldMissingNum + " outputs for reduceId " + reduceId + - ". Sleep " + sleepInterval + "s. ---lirui") - Thread.sleep((sleepInterval * 1000).toLong) - logInfo("Trying to update map output statuses for reduceId "+reduceId+" ---lirui") - mapOutputTracker.updateMapStatusesForShuffle(shuffleId) - statuses = mapOutputTracker.getServerStatuses(shuffleId, reduceId) - val missingSplitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] - for (index <- missingMapOutputs if statuses(index)._1 != null) { - missingSplitsByAddress.getOrElseUpdate(statuses(index)._1, ArrayBuffer()) += ((index, statuses(index)._2)) - } - //we have new outputs ready for this reduce - if(!missingSplitsByAddress.isEmpty){ - val missingBlocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = missingSplitsByAddress.toSeq.map { - case (address, splits) => - (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) - } - val missingBlockFetcherItr = blockManager.getMultiple(missingBlocksByAddress, serializer) - itr = itr ++ missingBlockFetcherItr.flatMap(unpackBlock) - } else { - logInfo("No updates in the previous interval "+sleepInterval+"s, sleep longer. ---lirui") - } - missingMapOutputs = statuses.zipWithIndex.filter(_._1._1 == null).map(_._2) - val fillingUpSpeed = (oldMissingNum - missingMapOutputs.size).toFloat / sleepInterval - sleepInterval = if (fillingUpSpeed > 0.01) math.max(10.toFloat, missingMapOutputs.size.toFloat / 2) / fillingUpSpeed else sleepInterval * 4 - } +// var sleepInterval = 8.toFloat +// while(!missingMapOutputs.isEmpty){ +// val oldMissingNum = missingMapOutputs.size +// logInfo("Still missing " + oldMissingNum + " outputs for reduceId " + reduceId + +// ". Sleep " + sleepInterval + "s. ---lirui") +// Thread.sleep((sleepInterval * 1000).toLong) +// logInfo("Trying to update map output statuses for reduceId "+reduceId+" ---lirui") +// mapOutputTracker.updateMapStatusesForShuffle(shuffleId) +// statuses = mapOutputTracker.getServerStatuses(shuffleId, reduceId) +// val missingSplitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] +// for (index <- missingMapOutputs if statuses(index)._1 != null) { +// missingSplitsByAddress.getOrElseUpdate(statuses(index)._1, ArrayBuffer()) += ((index, statuses(index)._2)) +// } +// //we have new outputs ready for this reduce +// if(!missingSplitsByAddress.isEmpty){ +// val missingBlocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = missingSplitsByAddress.toSeq.map { +// case (address, splits) => +// (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) +// } +// val missingBlockFetcherItr = blockManager.getMultiple(missingBlocksByAddress, serializer) +// itr = itr ++ missingBlockFetcherItr.flatMap(unpackBlock) +// } else { +// logInfo("No updates in the previous interval "+sleepInterval+"s, sleep longer. ---lirui") +// } +// missingMapOutputs = statuses.zipWithIndex.filter(_._1._1 == null).map(_._2) +// val fillingUpSpeed = (oldMissingNum - missingMapOutputs.size).toFloat / sleepInterval +// sleepInterval = if (fillingUpSpeed > 0.01) math.max(10.toFloat, missingMapOutputs.size.toFloat / 2) / fillingUpSpeed else sleepInterval * 4 +// } val completionIter = CompletionIterator[T, Iterator[T]](itr, { val shuffleMetrics = new ShuffleReadMetrics diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 57ea626381d7..d30af58a041a 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -174,6 +174,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging //get map statuses for a shuffle private def getMapStatusesForShuffle(shuffleId: Int, reduceId: Int): Array[MapStatus]={ + updateMapStatusesForShuffle(shuffleId) val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") @@ -228,7 +229,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } //update partial map outputs for a shuffle - def updateMapStatusesForShuffle(shuffleId: Int){ + private def updateMapStatusesForShuffle(shuffleId: Int){ mapStatuses.synchronized { //we may have cached partial map outputs, the master may have updates for us if (mapStatuses.get(shuffleId).isDefined) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 5836d4e8ee32..0ce3becf3a8c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -19,13 +19,11 @@ package org.apache.spark.storage import java.util.concurrent.LinkedBlockingQueue -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashSet -import scala.collection.mutable.Queue +import scala.collection.mutable.{HashMap, ArrayBuffer, HashSet, Queue} import io.netty.buffer.ByteBuf -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.{MapOutputTracker, Logging, SparkException} import org.apache.spark.network.BufferMessage import org.apache.spark.network.ConnectionManagerId import org.apache.spark.network.netty.ShuffleCopier @@ -351,4 +349,87 @@ object BlockFetcherIterator { } } // End of NettyBlockFetcherIterator + + class PartialBlockFetcherIterator( + private val blockManager: BlockManager, + private var statuses: Array[(BlockManagerId, Long)], + private val mapOutputTracker: MapOutputTracker, + serializer: Serializer, + shuffleId: Int, + reduceId: Int) + extends BlockFetcherIterator { + private val iterators=new ArrayBuffer[BlockFetcherIterator]() + + //track the map outputs we've delegated + private val delegatedStatuses = new HashSet[Int]() + + //check if the map output is partial + private def isPartial = statuses.exists(_._1 == null) + + //get the updated map output + private def updateStatuses() { + logInfo("Trying to update map statuses for reduceId "+reduceId+" ---lirui") + statuses = mapOutputTracker.getServerStatuses(shuffleId, reduceId) + } + + private def readyStatuses = (0 until statuses.size).filter(statuses(_)._1 != null) + + //check if there's new map outputs ready to collect + private def newStatusesReady = readyStatuses.exists(!delegatedStatuses.contains(_)) + + private def getIterator() = { + if (isPartial) { + logInfo("Still missing "+statuses.filter(_._1==null).size+" map outputs for reduceId "+reduceId+" ---lirui") + updateStatuses() + } + val maxTrialCount = 8 + var trialCount = 0 + while (!newStatusesReady && trialCount < maxTrialCount) { + logInfo("Waiting for new map outputs for reduceId " + reduceId + " ---lirui") + Thread.sleep(5000) + updateStatuses() + trialCount += 1 + } + if (trialCount >= maxTrialCount) { + throw new SparkException("Failed to get new iterator: no update in last " + trialCount + " trials.") + } + val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] + for (index <- readyStatuses if !delegatedStatuses.contains(index)) { + splitsByAddress.getOrElseUpdate(statuses(index)._1, ArrayBuffer()) += ((index, statuses(index)._2)) + delegatedStatuses += index + } + val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = splitsByAddress.toSeq.map { + case (address, splits) => + (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) + } + logInfo("Delegating " + blocksByAddress.map(_._2.size).sum + " blocks to a new iterator. ---lirui") + blockManager.getMultiple(blocksByAddress, serializer) + } + + override def initialize(){ + iterators += getIterator() + } + + override def hasNext: Boolean = iterators.exists(_.hasNext) || delegatedStatuses.size < statuses.size + + override def next(): (BlockId, Option[Iterator[Any]]) = { + //firstly try to get a block from the iterators we've created + for (itr <- iterators if itr.hasNext) { + return itr.next() + } + //TODO: need to take care of empty blocks here + iterators += getIterator() + next() + } + + override def totalBlocks = iterators.map(_.totalBlocks).sum + + override def numLocalBlocks = iterators.map(_.numLocalBlocks).sum + + override def numRemoteBlocks = iterators.map(_.numRemoteBlocks).sum + + override def fetchWaitTime = iterators.map(_.fetchWaitTime).sum + + override def remoteBytesRead = iterators.map(_.remoteBytesRead).sum + } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 6d7d4f922e1f..f06c67799b32 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -538,6 +538,17 @@ private[spark] class BlockManager( iter } + def getPartial( + statuses: Array[(BlockManagerId, Long)], + mapOutputTracker: MapOutputTracker, + serializer: Serializer, + shuffleId: Int, + reduceId: Int) = { + val iter = new BlockFetcherIterator.PartialBlockFetcherIterator(this, statuses, mapOutputTracker, serializer, shuffleId, reduceId) + iter.initialize() + iter + } + def put( blockId: BlockId, values: Iterator[Any], From 376230a20822e797558530e67ee9df3619476d78 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 16 May 2014 13:13:08 +0800 Subject: [PATCH 14/88] RemoveStageBarrier: minor fixes --- .../scala/org/apache/spark/MapOutputTracker.scala | 9 --------- .../org/apache/spark/scheduler/DAGScheduler.scala | 14 +++++++------- .../spark/storage/BlockFetcherIterator.scala | 10 +++++----- 3 files changed, 12 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index d30af58a041a..c13db855bad6 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -163,15 +163,6 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging /** Stop the tracker. */ def stop() { } - //check if the map output for a shuffle is partial - private def partialOutputForShuffle(shuffleId: Int) = { - if (mapStatuses.get(shuffleId).isDefined) { - mapStatuses.get(shuffleId).get.exists(_ == null) - } else { - false - } - } - //get map statuses for a shuffle private def getMapStatusesForShuffle(shuffleId: Int, reduceId: Int): Array[MapStatus]={ updateMapStatusesForShuffle(shuffleId) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b2b3f825e8da..e79ad318fa55 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -852,13 +852,13 @@ class DAGScheduler( logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId) } else { stage.addOutputLoc(smt.partitionId, status) - } - //we need to register map outputs progressively if remove stage barrier is enabled - if (removeStageBarrier && dependantStagePreStarted.contains(stage) && stage.shuffleDep.isDefined) { - logInfo("Register output progressively: Map task "+smt.partitionId+" ---lirui") - mapOutputTracker.registerMapOutput(stage.shuffleDep.get.shuffleId, smt.partitionId, status) - //need to increment the mapoutputtrackermaster's epoch so that it will clear the cache - mapOutputTracker.incrementEpoch() + //we need to register map outputs progressively if remove stage barrier is enabled + if (removeStageBarrier && dependantStagePreStarted.contains(stage) && stage.shuffleDep.isDefined) { + logInfo("Register output progressively: Map task "+smt.partitionId+" ---lirui") + mapOutputTracker.registerMapOutput(stage.shuffleDep.get.shuffleId, smt.partitionId, status) + //need to increment the mapoutputtrackermaster's epoch so that it will clear the cache + mapOutputTracker.incrementEpoch() + } } if (runningStages.contains(stage) && pendingTasks(stage).isEmpty) { dependantStagePreStarted -= stage diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 0ce3becf3a8c..cdda96649592 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -377,8 +377,8 @@ object BlockFetcherIterator { //check if there's new map outputs ready to collect private def newStatusesReady = readyStatuses.exists(!delegatedStatuses.contains(_)) - private def getIterator() = { - if (isPartial) { + private def getIterator(initial: Boolean = false) = { + if (isPartial && !initial) { logInfo("Still missing "+statuses.filter(_._1==null).size+" map outputs for reduceId "+reduceId+" ---lirui") updateStatuses() } @@ -402,15 +402,15 @@ object BlockFetcherIterator { case (address, splits) => (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) } - logInfo("Delegating " + blocksByAddress.map(_._2.size).sum + " blocks to a new iterator. ---lirui") + logInfo("Delegating " + blocksByAddress.map(_._2.size).sum + " blocks to a new iterator for reduceId "+reduceId+" ---lirui") blockManager.getMultiple(blocksByAddress, serializer) } override def initialize(){ - iterators += getIterator() + iterators += getIterator(true) } - override def hasNext: Boolean = iterators.exists(_.hasNext) || delegatedStatuses.size < statuses.size + override def hasNext: Boolean = delegatedStatuses.size < statuses.size || iterators.exists(_.hasNext) override def next(): (BlockId, Option[Iterator[Any]]) = { //firstly try to get a block from the iterators we've created From efd31efb006ddb3275c42375a069efcfa8d72244 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 16 May 2014 14:20:16 +0800 Subject: [PATCH 15/88] RemoveStageBarrier: fix: reducers may fail due to very slow mappers --- .../apache/spark/storage/BlockFetcherIterator.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index cdda96649592..c01f61ccc244 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -382,17 +382,17 @@ object BlockFetcherIterator { logInfo("Still missing "+statuses.filter(_._1==null).size+" map outputs for reduceId "+reduceId+" ---lirui") updateStatuses() } - val maxTrialCount = 8 var trialCount = 0 - while (!newStatusesReady && trialCount < maxTrialCount) { + while (!newStatusesReady) { + if (!isPartial && delegatedStatuses.size >= statuses.size) { + //shouldn't get here, could be due to empty blocks though + throw new SparkException("No more blocks to fetch for reduceId " + reduceId) + } logInfo("Waiting for new map outputs for reduceId " + reduceId + " ---lirui") - Thread.sleep(5000) + Thread.sleep(5000 + 2000 * trialCount) updateStatuses() trialCount += 1 } - if (trialCount >= maxTrialCount) { - throw new SparkException("Failed to get new iterator: no update in last " + trialCount + " trials.") - } val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] for (index <- readyStatuses if !delegatedStatuses.contains(index)) { splitsByAddress.getOrElseUpdate(statuses(index)._1, ArrayBuffer()) += ((index, statuses(index)._2)) From 3cb944c9cdbc48ba9a152a9f425d3b09ab810390 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 16 May 2014 16:29:09 +0800 Subject: [PATCH 16/88] RemoveStageBarrier: add some log info --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 3 +-- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 1 + .../scala/org/apache/spark/storage/BlockFetcherIterator.scala | 1 + 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index c13db855bad6..de9acdf897c1 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -165,7 +165,6 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging //get map statuses for a shuffle private def getMapStatusesForShuffle(shuffleId: Int, reduceId: Int): Array[MapStatus]={ - updateMapStatusesForShuffle(shuffleId) val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") @@ -220,7 +219,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } //update partial map outputs for a shuffle - private def updateMapStatusesForShuffle(shuffleId: Int){ + def updateMapStatusesForShuffle(shuffleId: Int){ mapStatuses.synchronized { //we may have cached partial map outputs, the master may have updates for us if (mapStatuses.get(shuffleId).isDefined) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 5a68f38bc584..945f9173cef3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -227,6 +227,7 @@ private[spark] class TaskSchedulerImpl( logDebug("parentName: %s, name: %s, runningTasks: %s".format( taskSet.parent.name, taskSet.name, taskSet.runningTasks)) } + logInfo("After sorting, the first TaskSet is (StageID:"+sortedTaskSets.head.stageId+", Priority:"+sortedTaskSets.head.priority+") ---lirui") // Take each TaskSet in our scheduling order, and then offer it each node in increasing order // of locality levels so that it gets a chance to launch local tasks on all of them. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index c01f61ccc244..e73e53393b39 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -369,6 +369,7 @@ object BlockFetcherIterator { //get the updated map output private def updateStatuses() { logInfo("Trying to update map statuses for reduceId "+reduceId+" ---lirui") + mapOutputTracker.updateMapStatusesForShuffle(shuffleId) statuses = mapOutputTracker.getServerStatuses(shuffleId, reduceId) } From 641715e4a27e1b303a0cb13dc16e5544a8710cc7 Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 19 May 2014 09:28:32 +0800 Subject: [PATCH 17/88] RemoveStageBarrier: stage with a bigger ID should take precedence --- .../scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala | 2 +- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala index 5e62c8468f00..393b483d695b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala @@ -34,7 +34,7 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { if (res == 0) { val stageId1 = s1.stageId val stageId2 = s2.stageId - res = math.signum(stageId1 - stageId2) + res = math.signum(stageId2 - stageId1) } if (res < 0) { true diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 945f9173cef3..5a68f38bc584 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -227,7 +227,6 @@ private[spark] class TaskSchedulerImpl( logDebug("parentName: %s, name: %s, runningTasks: %s".format( taskSet.parent.name, taskSet.name, taskSet.runningTasks)) } - logInfo("After sorting, the first TaskSet is (StageID:"+sortedTaskSets.head.stageId+", Priority:"+sortedTaskSets.head.priority+") ---lirui") // Take each TaskSet in our scheduling order, and then offer it each node in increasing order // of locality levels so that it gets a chance to launch local tasks on all of them. From b0c2df24909d5ecb92f6d926f4705dedaf96660f Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 23 May 2014 10:35:58 +0800 Subject: [PATCH 18/88] RemoveStageBarrier: track whether map output for a shuffle is partial in MapOutputTracker --- .../org/apache/spark/MapOutputTracker.scala | 57 ++++++++++++++----- .../apache/spark/scheduler/DAGScheduler.scala | 6 +- .../spark/storage/BlockFetcherIterator.scala | 2 +- .../apache/spark/storage/MemoryStore.scala | 2 +- .../spark/storage/ShuffleBlockManager.scala | 1 + 5 files changed, 49 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index de9acdf897c1..9fce4df44ffd 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -91,6 +91,9 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging //TODO: we should also record if the output for a shuffle is partial protected val mapStatuses: Map[Int, Array[MapStatus]] + //track if we have partial map outputs for a shuffle + protected val partialForShuffle = new mutable.HashSet[Int]() + /** * Incremented every time a fetch fails so that client nodes know to clear * their cache of map output locations if this happens. @@ -129,7 +132,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = { val statuses = getMapStatusesForShuffle(shuffleId, reduceId) statuses.synchronized { - MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses) + MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses, isPartial = partialForShuffle.contains(shuffleId)) } } @@ -158,6 +161,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging /** Unregister shuffle data. */ def unregisterShuffle(shuffleId: Int) { mapStatuses.remove(shuffleId) + partialForShuffle -= shuffleId } /** Stop the tracker. */ @@ -197,7 +201,15 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging try { val fetchedBytes = askTracker(GetMapOutputStatuses(shuffleId)).asInstanceOf[Array[Byte]] - fetchedStatuses = MapOutputTracker.deserializeMapStatuses(fetchedBytes) + val fetchedResults = MapOutputTracker.deserializeMapStatuses(fetchedBytes) + fetchedStatuses = fetchedResults._1 + if (fetchedResults._2) { + logInfo("Got partial map outputs from master for reduceId " + reduceId + ". ---lirui") + partialForShuffle += shuffleId + } else { + logInfo("Got complete map outputs from master for reduceId " + reduceId + ". ---lirui") + partialForShuffle -= shuffleId + } logInfo("Got the output locations") mapStatuses.put(shuffleId, fetchedStatuses) } finally { @@ -218,8 +230,9 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } } - //update partial map outputs for a shuffle - def updateMapStatusesForShuffle(shuffleId: Int){ + //clear outdated map outputs for a shuffle and sync the local epoch with master + //this method is intended to be followed by "getServerStatuses" + def clearOutdatedMapStatuses(shuffleId: Int){ mapStatuses.synchronized { //we may have cached partial map outputs, the master may have updates for us if (mapStatuses.get(shuffleId).isDefined) { @@ -227,6 +240,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging if (masterEpoch > epoch) { logInfo("Master's epoch is " + masterEpoch + ", local epoch is " + epoch + ". Clear local cache. ---lirui") mapStatuses -= shuffleId + partialForShuffle -= shuffleId epoch = masterEpoch } } @@ -270,11 +284,18 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } /** Register multiple map output information for the given shuffle */ - def registerMapOutputs(shuffleId: Int, statuses: Array[MapStatus], changeEpoch: Boolean = false) { + def registerMapOutputs(shuffleId: Int, statuses: Array[MapStatus], changeEpoch: Boolean = false, isPartial: Boolean = false) { mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses) if (changeEpoch) { incrementEpoch() } + if (isPartial) { + logInfo("Registered partial map outputs. ---lirui") + partialForShuffle += shuffleId + } else { + logInfo("Registered complete map outputs. ---lirui") + partialForShuffle -= shuffleId + } } /** Unregister map output information of the given shuffle, mapper and block manager */ @@ -295,7 +316,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) /** Unregister shuffle data */ override def unregisterShuffle(shuffleId: Int) { - mapStatuses.remove(shuffleId) + super.unregisterShuffle(shuffleId) cachedSerializedStatuses.remove(shuffleId) } @@ -329,7 +350,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } // If we got here, we failed to find the serialized locations in the cache, so we pulled // out a snapshot of the locations as "statuses"; let's serialize and return that - val bytes = MapOutputTracker.serializeMapStatuses(statuses) + val bytes = MapOutputTracker.serializeMapStatuses(statuses,isPartial = partialForShuffle.contains(shuffleId)) logInfo("Size of output statuses for shuffle %d is %d bytes".format(shuffleId, bytes.length)) // Add them into the table only if the epoch hasn't changed while we were working epochLock.synchronized { @@ -368,21 +389,24 @@ private[spark] object MapOutputTracker { // Serialize an array of map output locations into an efficient byte format so that we can send // it to reduce tasks. We do this by compressing the serialized bytes using GZIP. They will // generally be pretty compressible because many map outputs will be on the same hostname. - def serializeMapStatuses(statuses: Array[MapStatus]): Array[Byte] = { + def serializeMapStatuses(statuses: Array[MapStatus], isPartial: Boolean = false): Array[Byte] = { val out = new ByteArrayOutputStream val objOut = new ObjectOutputStream(new GZIPOutputStream(out)) // Since statuses can be modified in parallel, sync on it statuses.synchronized { objOut.writeObject(statuses) + objOut.writeBoolean(isPartial) } objOut.close() out.toByteArray } // Opposite of serializeMapStatuses. - def deserializeMapStatuses(bytes: Array[Byte]): Array[MapStatus] = { + def deserializeMapStatuses(bytes: Array[Byte]): (Array[MapStatus], Boolean) = { val objIn = new ObjectInputStream(new GZIPInputStream(new ByteArrayInputStream(bytes))) - objIn.readObject().asInstanceOf[Array[MapStatus]] + val mapStatuses = objIn.readObject().asInstanceOf[Array[MapStatus]] + val isPartial = objIn.readBoolean() + (mapStatuses, isPartial) } // Convert an array of MapStatuses to locations and sizes for a given reduce ID. If @@ -391,15 +415,18 @@ private[spark] object MapOutputTracker { private def convertMapStatuses( shuffleId: Int, reduceId: Int, - statuses: Array[MapStatus]): Array[(BlockManagerId, Long)] = { + statuses: Array[MapStatus], + isPartial: Boolean = false): Array[(BlockManagerId, Long)] = { assert (statuses != null) statuses.map { status => if (status == null) { - //TODO: need to distinguish whether this is due to failed map tasks or partial outputs - (null, 0.toLong) -// throw new FetchFailedException(null, shuffleId, -1, reduceId, -// new Exception("Missing an output location for shuffle " + shuffleId)) + if(isPartial){ + (null, 0.toLong) + } else { + throw new FetchFailedException(null, shuffleId, -1, reduceId, + new Exception("Missing an output location for shuffle " + shuffleId)) + } } else { (status.location, decompressSize(status.compressedSizes(reduceId))) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index e79ad318fa55..e29ee597df96 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -247,7 +247,7 @@ class DAGScheduler( val stage = newStage(rdd, numTasks, Some(shuffleDep), jobId, callSite) if (mapOutputTracker.containsShuffle(shuffleDep.shuffleId)) { val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId) - val locs = MapOutputTracker.deserializeMapStatuses(serLocs) + val locs = MapOutputTracker.deserializeMapStatuses(serLocs)._1 for (i <- 0 until locs.size) { stage.outputLocs(i) = Option(locs(i)).toList // locs(i) will be null if missing } @@ -923,7 +923,9 @@ class DAGScheduler( val preStartedStage = waitingStages.head logInfo("Pre-start stage " + preStartedStage.id + " ---lirui") //register map output finished so far - mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, true) + mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, + stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, + changeEpoch = true, isPartial = true) waitingStages -= preStartedStage runningStages += preStartedStage dependantStagePreStarted += stage diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index e73e53393b39..d4553853abe3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -369,7 +369,7 @@ object BlockFetcherIterator { //get the updated map output private def updateStatuses() { logInfo("Trying to update map statuses for reduceId "+reduceId+" ---lirui") - mapOutputTracker.updateMapStatusesForShuffle(shuffleId) + mapOutputTracker.clearOutdatedMapStatuses(shuffleId) statuses = mapOutputTracker.getServerStatuses(shuffleId, reduceId) } diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 488f1ea9628f..e3fcf95992de 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -154,7 +154,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) * size must also be passed by the caller. * * Lock on the object putLock to ensure that all the put requests and its associated block - * dropping is done by only on thread at a time. Otherwise while one thread is dropping + * dropping is done by only one thread at a time. Otherwise while one thread is dropping * blocks to free memory for one block, another thread may use up the freed space for * another block. * diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index a935f2b6ce6a..1b8de89e42c4 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -249,6 +249,7 @@ object ShuffleBlockManager { /** Returns the FileSegment associated with the given map task, or None if no entry exists. */ def getFileSegmentFor(mapId: Int, reducerId: Int): Option[FileSegment] = { + //TODO: potential issue here when map and reduce stages overlap val file = files(reducerId) val blockOffsets = blockOffsetsByReducer(reducerId) val index = mapIdToIndex.getOrElse(mapId, -1) From 75d27449e3b65382e5968842f5fa73c5f939c663 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 23 May 2014 15:27:02 +0800 Subject: [PATCH 19/88] RemoveStageBarrier: refine how we get the stage to pre-start --- .../org/apache/spark/MapOutputTracker.scala | 1 + .../apache/spark/scheduler/DAGScheduler.scala | 38 +++++++++++++------ 2 files changed, 28 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 9fce4df44ffd..9015ec8cf02b 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -154,6 +154,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging logInfo("Updating epoch from "+epoch+" to " + newEpoch + " and clearing cache") epoch = newEpoch mapStatuses.clear() + partialForShuffle.clear() } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index e29ee597df96..359393c5f3cb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -919,17 +919,20 @@ class DAGScheduler( if (freeCores > 0 && waitingStageNum > 0 && stage.shuffleDep.isDefined) { logInfo("We have " + totalCores + " CPUs. " + pendingTaskNum + " tasks are running/pending. " + waitingStageNum + " stages are waiting to be submitted. ---lirui") - //TODO: find a waiting stage that depends on the current "stage" - val preStartedStage = waitingStages.head - logInfo("Pre-start stage " + preStartedStage.id + " ---lirui") - //register map output finished so far - mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, - stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, - changeEpoch = true, isPartial = true) - waitingStages -= preStartedStage - runningStages += preStartedStage - dependantStagePreStarted += stage - submitMissingTasks(preStartedStage, activeJobForStage(preStartedStage).get) + val preStartableStage = getPreStartableStage(stage) + if (preStartableStage.isDefined) { + val preStartedStage = preStartableStage.get + logInfo("Pre-start stage " + preStartedStage.id + " ---lirui") + //register map output finished so far + mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, + stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, + changeEpoch = true, isPartial = true) + waitingStages -= preStartedStage + runningStages += preStartedStage + //inform parent stages that the depending stage has been pre-started + dependantStagePreStarted ++= getMissingParentStages(preStartedStage) + submitMissingTasks(preStartedStage, activeJobForStage(preStartedStage).get) + } } } } @@ -1181,6 +1184,19 @@ class DAGScheduler( dagSchedulerActorSupervisor ! PoisonPill taskScheduler.stop() } + + //select a waiting stage to pre-start + private def getPreStartableStage(stage: Stage): Option[Stage] = { + //select a stage not ready to run + for (waitingStage <- waitingStages) { + val missingParents = getMissingParentStages(waitingStage) + if (missingParents != Nil && missingParents.contains(stage) && + missingParents.forall(parent => !(waitingStages.contains(parent) || failedStages.contains(parent)))) { + return Some(waitingStage) + } + } + None + } } private[scheduler] class DAGSchedulerActorSupervisor(dagScheduler: DAGScheduler) From b7f1f844522d2c502c2e344f87ea2c729d53b4bc Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 23 May 2014 15:47:03 +0800 Subject: [PATCH 20/88] RemoveStageBarrier: indicate the output is partial for progressive registration --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 359393c5f3cb..69a052660b77 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -855,9 +855,9 @@ class DAGScheduler( //we need to register map outputs progressively if remove stage barrier is enabled if (removeStageBarrier && dependantStagePreStarted.contains(stage) && stage.shuffleDep.isDefined) { logInfo("Register output progressively: Map task "+smt.partitionId+" ---lirui") - mapOutputTracker.registerMapOutput(stage.shuffleDep.get.shuffleId, smt.partitionId, status) - //need to increment the mapoutputtrackermaster's epoch so that it will clear the cache - mapOutputTracker.incrementEpoch() + mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, + stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, + changeEpoch = true, isPartial = true) } } if (runningStages.contains(stage) && pendingTasks(stage).isEmpty) { From be474084f7a422873114cdd291c8a07da3f5ad2d Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 26 May 2014 13:03:34 +0800 Subject: [PATCH 21/88] add some debug info --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 +++ .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 4 ++++ 2 files changed, 7 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 69a052660b77..2b9256935fad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -742,6 +742,9 @@ class DAGScheduler( if (stage.isShuffleMap) { for (p <- 0 until stage.numPartitions if stage.outputLocs(p) == Nil) { val locs = getPreferredLocs(stage.rdd, p) + if(!locs.isEmpty){ + logInfo("Partition "+p+" has "+locs.size+" preferred locations. First is "+locs(0).host+". ---lirui") + } tasks += new ShuffleMapTask(stage.id, stage.rdd, stage.shuffleDep.get, p, locs) } } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index f3bd0797aa03..a856050bb96a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -151,6 +151,10 @@ private[spark] class TaskSetManager( for (i <- (0 until numTasks).reverse) { addPendingTask(i) } + //add some debug info here + for(tasks <- pendingTasksForExecutor){ + logInfo("Pending tasks size for executorId "+tasks._1+" is "+tasks._2.size+". ---lirui") + } // Figure out which locality levels we have in our TaskSet, so we can do delay scheduling val myLocalityLevels = computeValidLocalityLevels() From c88014b0ebcf261554bcbac537e60130b9276da5 Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 26 May 2014 14:37:04 +0800 Subject: [PATCH 22/88] add a new locality level for tasks with no preferred locations --- .../main/scala/org/apache/spark/scheduler/TaskLocality.scala | 2 +- .../main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index eb920ab0c0b6..1857078778c0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -22,7 +22,7 @@ import org.apache.spark.annotation.DeveloperApi @DeveloperApi object TaskLocality extends Enumeration { // Process local is expected to be used ONLY within TaskSetManager for now. - val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value + val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, NO_PREFER, ANY = Value type TaskLocality = Value diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index a856050bb96a..9a47d2230b00 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -366,7 +366,7 @@ private[spark] class TaskSetManager( // Look for no-pref tasks after rack-local tasks since they can run anywhere. for (index <- findTaskFromList(execId, pendingTasksWithNoPrefs)) { - return Some((index, TaskLocality.PROCESS_LOCAL)) + return Some((index, TaskLocality.NO_PREFER)) } if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { From 133a356d04e00a87a84e576b464b803d24a270ed Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 26 May 2014 20:34:16 +0800 Subject: [PATCH 23/88] re-compute pending list when new executor is added --- .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 6 ++++++ .../org/apache/spark/scheduler/TaskSetManager.scala | 10 ++++++++++ 2 files changed, 16 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 5a68f38bc584..5ed114c4aa32 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -209,11 +209,14 @@ private[spark] class TaskSchedulerImpl( SparkEnv.set(sc.env) // Mark each slave as alive and remember its hostname + //also track if new executor is added + var executorUpdated = false for (o <- offers) { executorIdToHost(o.executorId) = o.host if (!executorsByHost.contains(o.host)) { executorsByHost(o.host) = new HashSet[String]() executorAdded(o.executorId, o.host) + executorUpdated = true } } @@ -232,6 +235,9 @@ private[spark] class TaskSchedulerImpl( // of locality levels so that it gets a chance to launch local tasks on all of them. var launchedTask = false for (taskSet <- sortedTaskSets; maxLocality <- TaskLocality.values) { + if (executorUpdated) { + taskSet.reAssignTasksWithNoPrefs() + } do { launchedTask = false for (i <- 0 until shuffledOffers.size) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 9a47d2230b00..4296b62b68c4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -742,4 +742,14 @@ private[spark] class TaskSetManager( logDebug("Valid locality levels for " + taskSet + ": " + levels.mkString(", ")) levels.toArray } + + //Re-assign tasks in pendingTasksWithNoPrefs when there's new executor added + def reAssignTasksWithNoPrefs() { + val tasksClone = pendingTasksWithNoPrefs.clone() + pendingTasksWithNoPrefs.clear() + for (index <- tasksClone) { + addPendingTask(index) + } + logInfo("Moved " + (tasksClone.size - pendingTasksWithNoPrefs.size) + " tasks from NoPrefs to other pending list. ---lirui") + } } From 7d92f9a3d1a925701170c8ac9e459903073c09b8 Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 26 May 2014 22:10:33 +0800 Subject: [PATCH 24/88] pendingTasksWithNoPrefs should only contain tasks that really have no preferred locations --- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../apache/spark/scheduler/TaskSetManager.scala | 14 ++++++-------- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 5ed114c4aa32..a635224f9867 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -236,7 +236,7 @@ private[spark] class TaskSchedulerImpl( var launchedTask = false for (taskSet <- sortedTaskSets; maxLocality <- TaskLocality.values) { if (executorUpdated) { - taskSet.reAssignTasksWithNoPrefs() + taskSet.reAddPendingTasks() } do { launchedTask = false diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 4296b62b68c4..34efe67ac0b1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -199,7 +199,7 @@ private[spark] class TaskSetManager( } } - if (!hadAliveLocations) { + if (tasks(index).preferredLocations.isEmpty) { // Even though the task might've had preferred locations, all of those hosts or executors // are dead; put it in the no-prefs list so we can schedule it elsewhere right away. addTo(pendingTasksWithNoPrefs) @@ -743,13 +743,11 @@ private[spark] class TaskSetManager( levels.toArray } - //Re-assign tasks in pendingTasksWithNoPrefs when there's new executor added - def reAssignTasksWithNoPrefs() { - val tasksClone = pendingTasksWithNoPrefs.clone() - pendingTasksWithNoPrefs.clear() - for (index <- tasksClone) { - addPendingTask(index) + //Re-compute the pending lists. This should be called when new executor is added + def reAddPendingTasks() { + logInfo("Re-computing pending task lists.") + for (i <- (0 until numTasks).reverse.filter(index => copiesRunning(index) == 0 && !successful(index))) { + addPendingTask(i, readding = true) } - logInfo("Moved " + (tasksClone.size - pendingTasksWithNoPrefs.size) + " tasks from NoPrefs to other pending list. ---lirui") } } From c1de426c159c9a0670bfb30bae81d772195290d8 Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 27 May 2014 10:16:45 +0800 Subject: [PATCH 25/88] make the delay schedule configurable --- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index a635224f9867..673501f4df0c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -235,7 +235,7 @@ private[spark] class TaskSchedulerImpl( // of locality levels so that it gets a chance to launch local tasks on all of them. var launchedTask = false for (taskSet <- sortedTaskSets; maxLocality <- TaskLocality.values) { - if (executorUpdated) { + if (executorUpdated && conf.getBoolean("spark.schedule.delaySchedule", false)) { taskSet.reAddPendingTasks() } do { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 34efe67ac0b1..541e3482e4c6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -199,7 +199,8 @@ private[spark] class TaskSetManager( } } - if (tasks(index).preferredLocations.isEmpty) { + if (tasks(index).preferredLocations.isEmpty || + (!conf.getBoolean("spark.schedule.delaySchedule", false) && !hadAliveLocations)) { // Even though the task might've had preferred locations, all of those hosts or executors // are dead; put it in the no-prefs list so we can schedule it elsewhere right away. addTo(pendingTasksWithNoPrefs) From e57e081e847495b11289b48aeb894cd634ecca71 Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 27 May 2014 11:39:29 +0800 Subject: [PATCH 26/88] clean up --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 --- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 4 ---- 2 files changed, 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 2b9256935fad..69a052660b77 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -742,9 +742,6 @@ class DAGScheduler( if (stage.isShuffleMap) { for (p <- 0 until stage.numPartitions if stage.outputLocs(p) == Nil) { val locs = getPreferredLocs(stage.rdd, p) - if(!locs.isEmpty){ - logInfo("Partition "+p+" has "+locs.size+" preferred locations. First is "+locs(0).host+". ---lirui") - } tasks += new ShuffleMapTask(stage.id, stage.rdd, stage.shuffleDep.get, p, locs) } } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 541e3482e4c6..e8ff948be02f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -151,10 +151,6 @@ private[spark] class TaskSetManager( for (i <- (0 until numTasks).reverse) { addPendingTask(i) } - //add some debug info here - for(tasks <- pendingTasksForExecutor){ - logInfo("Pending tasks size for executorId "+tasks._1+" is "+tasks._2.size+". ---lirui") - } // Figure out which locality levels we have in our TaskSet, so we can do delay scheduling val myLocalityLevels = computeValidLocalityLevels() From fda0281490b52adf7cf20da20ffb36f5c34f7e7c Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 27 May 2014 13:54:13 +0800 Subject: [PATCH 27/88] do some refactor --- .../apache/spark/scheduler/TaskLocality.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 6 ----- .../spark/scheduler/TaskSetManager.scala | 25 ++++++++----------- 3 files changed, 11 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index 1857078778c0..eb920ab0c0b6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -22,7 +22,7 @@ import org.apache.spark.annotation.DeveloperApi @DeveloperApi object TaskLocality extends Enumeration { // Process local is expected to be used ONLY within TaskSetManager for now. - val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, NO_PREFER, ANY = Value + val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value type TaskLocality = Value diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 673501f4df0c..5a68f38bc584 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -209,14 +209,11 @@ private[spark] class TaskSchedulerImpl( SparkEnv.set(sc.env) // Mark each slave as alive and remember its hostname - //also track if new executor is added - var executorUpdated = false for (o <- offers) { executorIdToHost(o.executorId) = o.host if (!executorsByHost.contains(o.host)) { executorsByHost(o.host) = new HashSet[String]() executorAdded(o.executorId, o.host) - executorUpdated = true } } @@ -235,9 +232,6 @@ private[spark] class TaskSchedulerImpl( // of locality levels so that it gets a chance to launch local tasks on all of them. var launchedTask = false for (taskSet <- sortedTaskSets; maxLocality <- TaskLocality.values) { - if (executorUpdated && conf.getBoolean("spark.schedule.delaySchedule", false)) { - taskSet.reAddPendingTasks() - } do { launchedTask = false for (i <- 0 until shuffledOffers.size) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index e8ff948be02f..ed38ba755cc9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -148,8 +148,10 @@ private[spark] class TaskSetManager( // Add all our tasks to the pending lists. We do this in reverse order // of task index so that tasks with low indices get launched first. + val delaySchedule = conf.getBoolean("spark.schedule.delaySchedule", true) for (i <- (0 until numTasks).reverse) { - addPendingTask(i) + //if delay schedule is set, we shouldn't enforce check since executors may haven't registered yet + addPendingTask(i, enforceCheck = !delaySchedule) } // Figure out which locality levels we have in our TaskSet, so we can do delay scheduling @@ -169,8 +171,10 @@ private[spark] class TaskSetManager( /** * Add a task to all the pending-task lists that it should be on. If readding is set, we are * re-adding the task so only include it in each list if it's not already there. + * If enforceCheck is set, we'll check the availability of executors/hosts before adding a task + * to the pending list, otherwise, we simply add the task according to its preference. */ - private def addPendingTask(index: Int, readding: Boolean = false) { + private def addPendingTask(index: Int, readding: Boolean = false, enforceCheck: Boolean = true) { // Utility method that adds `index` to a list only if readding=false or it's not already there def addTo(list: ArrayBuffer[Int]) { if (!readding || !list.contains(index)) { @@ -181,12 +185,12 @@ private[spark] class TaskSetManager( var hadAliveLocations = false for (loc <- tasks(index).preferredLocations) { for (execId <- loc.executorId) { - if (sched.isExecutorAlive(execId)) { + if (!enforceCheck || sched.isExecutorAlive(execId)) { addTo(pendingTasksForExecutor.getOrElseUpdate(execId, new ArrayBuffer)) hadAliveLocations = true } } - if (sched.hasExecutorsAliveOnHost(loc.host)) { + if (!enforceCheck || sched.hasExecutorsAliveOnHost(loc.host)) { addTo(pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer)) for (rack <- sched.getRackForHost(loc.host)) { addTo(pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer)) @@ -195,8 +199,7 @@ private[spark] class TaskSetManager( } } - if (tasks(index).preferredLocations.isEmpty || - (!conf.getBoolean("spark.schedule.delaySchedule", false) && !hadAliveLocations)) { + if (!hadAliveLocations) { // Even though the task might've had preferred locations, all of those hosts or executors // are dead; put it in the no-prefs list so we can schedule it elsewhere right away. addTo(pendingTasksWithNoPrefs) @@ -363,7 +366,7 @@ private[spark] class TaskSetManager( // Look for no-pref tasks after rack-local tasks since they can run anywhere. for (index <- findTaskFromList(execId, pendingTasksWithNoPrefs)) { - return Some((index, TaskLocality.NO_PREFER)) + return Some((index, TaskLocality.PROCESS_LOCAL)) } if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { @@ -739,12 +742,4 @@ private[spark] class TaskSetManager( logDebug("Valid locality levels for " + taskSet + ": " + levels.mkString(", ")) levels.toArray } - - //Re-compute the pending lists. This should be called when new executor is added - def reAddPendingTasks() { - logInfo("Re-computing pending task lists.") - for (i <- (0 until numTasks).reverse.filter(index => copiesRunning(index) == 0 && !successful(index))) { - addPendingTask(i, readding = true) - } - } } From 781861dc432ab6fad134691050be4d71f8bad1eb Mon Sep 17 00:00:00 2001 From: lirui Date: Wed, 28 May 2014 16:31:33 +0800 Subject: [PATCH 28/88] RemoveStageBarrier: fix problem with consolidated shuffle file --- .../spark/storage/ShuffleBlockManager.scala | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 1b8de89e42c4..85bc6f139d99 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -123,7 +123,7 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { override def releaseWriters(success: Boolean) { if (consolidateShuffleFiles) { if (success) { - val offsets = writers.map(_.fileSegment().offset) + val offsets = writers.map(writer => writer.fileSegment().offset + writer.bytesWritten) fileGroup.recordMapOutput(mapId, offsets) } recycleFileGroup(fileGroup) @@ -231,9 +231,13 @@ object ShuffleBlockManager { * This ordering allows us to compute block lengths by examining the following block offset. * Note: mapIdToIndex(mapId) returns the index of the mapper into the vector for every * reducer. + * We also keep the offset of "one past the end" block, which is effectively the file length. + * Therefore when append new offsets, we're actually appending new file lengths */ private val blockOffsetsByReducer = Array.fill[PrimitiveVector[Long]](files.length) { - new PrimitiveVector[Long]() + val offsets = new PrimitiveVector[Long]() + offsets += 0 + offsets } def numBlocks = mapIdToIndex.size @@ -254,13 +258,15 @@ object ShuffleBlockManager { val blockOffsets = blockOffsetsByReducer(reducerId) val index = mapIdToIndex.getOrElse(mapId, -1) if (index >= 0) { + assert(index + 1 < blockOffsets.size, "Index is " + index + ", total size is " + blockOffsets.size) val offset = blockOffsets(index) - val length = - if (index + 1 < numBlocks) { - blockOffsets(index + 1) - offset - } else { - file.length() - offset - } + val length = blockOffsets(index + 1) - offset +// val length = +// if (index + 1 < numBlocks) { +// blockOffsets(index + 1) - offset +// } else { +// file.length() - offset +// } assert(length >= 0) Some(new FileSegment(file, offset, length)) } else { From 679813b180a68074e4dbadd16fc9a36fdaac9f37 Mon Sep 17 00:00:00 2001 From: lirui Date: Thu, 29 May 2014 11:39:34 +0800 Subject: [PATCH 29/88] RemoveStageBarrier: should fail the pre-started stages if the parent stage gets re-submitted --- .../apache/spark/scheduler/DAGScheduler.scala | 25 ++++++++++++++++--- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 69a052660b77..2c9b418630d2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -124,8 +124,8 @@ class DAGScheduler( //whether to enable remove stage barrier val removeStageBarrier = env.conf.getBoolean("spark.schedule.removeStageBarrier", true) - //whether there's pre-started stages depending on this stage - val dependantStagePreStarted = new mutable.HashSet[Stage]() + //track the pre-started stages depending on a stage (the key) + val dependantStagePreStarted = new mutable.HashMap[Stage, ArrayBuffer[Stage]]() private def initializeEventProcessActor() { // blocking the thread until supervisor is started, which ensures eventProcessActor is @@ -886,6 +886,21 @@ class DAGScheduler( logInfo("Resubmitting " + stage + " (" + stage.name + ") because some of its tasks had failed: " + stage.outputLocs.zipWithIndex.filter(_._1 == Nil).map(_._2).mkString(", ")) + //Pre-started dependant stages should fail + if (dependantStagePreStarted.contains(stage)) { + for (preStartedStage <- dependantStagePreStarted.get(stage).get) { + runningStages -= preStartedStage + // TODO: Cancel running tasks in the stage + logInfo("Marking " + preStartedStage + " (" + preStartedStage.name + + ") for resubmision due to parent stage resubmission") + if (failedStages.isEmpty && eventProcessActor != null) { + import env.actorSystem.dispatcher + env.actorSystem.scheduler.scheduleOnce( + RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) + } + failedStages += preStartedStage + } + } submitStage(stage) } else { val newlyRunnable = new ArrayBuffer[Stage] @@ -929,8 +944,10 @@ class DAGScheduler( changeEpoch = true, isPartial = true) waitingStages -= preStartedStage runningStages += preStartedStage - //inform parent stages that the depending stage has been pre-started - dependantStagePreStarted ++= getMissingParentStages(preStartedStage) + //inform parent stages that the dependant stage has been pre-started + for (parentStage <- getMissingParentStages(preStartedStage)) { + dependantStagePreStarted.getOrElseUpdate(parentStage, new ArrayBuffer[Stage]()) += preStartedStage + } submitMissingTasks(preStartedStage, activeJobForStage(preStartedStage).get) } } From 563d743f04038b0ea10e53877c36705bbdfea7d3 Mon Sep 17 00:00:00 2001 From: lirui Date: Thu, 29 May 2014 15:16:36 +0800 Subject: [PATCH 30/88] RemoveStageBarrier: fix issue with empty shuffle blocks --- .../spark/BlockStoreShuffleFetcher.scala | 30 ---------------- .../spark/storage/BlockFetcherIterator.scala | 36 ++++++++++++++----- 2 files changed, 27 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala index c77f0286a5a5..5d93bf5fbd7f 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala @@ -80,36 +80,6 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin val itr = blockFetcherItr.flatMap(unpackBlock) - //time interval(in second) the thread should sleep -// var sleepInterval = 8.toFloat -// while(!missingMapOutputs.isEmpty){ -// val oldMissingNum = missingMapOutputs.size -// logInfo("Still missing " + oldMissingNum + " outputs for reduceId " + reduceId + -// ". Sleep " + sleepInterval + "s. ---lirui") -// Thread.sleep((sleepInterval * 1000).toLong) -// logInfo("Trying to update map output statuses for reduceId "+reduceId+" ---lirui") -// mapOutputTracker.updateMapStatusesForShuffle(shuffleId) -// statuses = mapOutputTracker.getServerStatuses(shuffleId, reduceId) -// val missingSplitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] -// for (index <- missingMapOutputs if statuses(index)._1 != null) { -// missingSplitsByAddress.getOrElseUpdate(statuses(index)._1, ArrayBuffer()) += ((index, statuses(index)._2)) -// } -// //we have new outputs ready for this reduce -// if(!missingSplitsByAddress.isEmpty){ -// val missingBlocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = missingSplitsByAddress.toSeq.map { -// case (address, splits) => -// (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) -// } -// val missingBlockFetcherItr = blockManager.getMultiple(missingBlocksByAddress, serializer) -// itr = itr ++ missingBlockFetcherItr.flatMap(unpackBlock) -// } else { -// logInfo("No updates in the previous interval "+sleepInterval+"s, sleep longer. ---lirui") -// } -// missingMapOutputs = statuses.zipWithIndex.filter(_._1._1 == null).map(_._2) -// val fillingUpSpeed = (oldMissingNum - missingMapOutputs.size).toFloat / sleepInterval -// sleepInterval = if (fillingUpSpeed > 0.01) math.max(10.toFloat, missingMapOutputs.size.toFloat / 2) / fillingUpSpeed else sleepInterval * 4 -// } - val completionIter = CompletionIterator[T, Iterator[T]](itr, { val shuffleMetrics = new ShuffleReadMetrics shuffleMetrics.shuffleFinishTime = System.currentTimeMillis diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index d4553853abe3..9d9804ec693b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -380,14 +380,14 @@ object BlockFetcherIterator { private def getIterator(initial: Boolean = false) = { if (isPartial && !initial) { - logInfo("Still missing "+statuses.filter(_._1==null).size+" map outputs for reduceId "+reduceId+" ---lirui") + logInfo("Still missing " + statuses.filter(_._1 == null).size + " map outputs for reduceId " + reduceId + " ---lirui") updateStatuses() } var trialCount = 0 while (!newStatusesReady) { if (!isPartial && delegatedStatuses.size >= statuses.size) { - //shouldn't get here, could be due to empty blocks though - throw new SparkException("No more blocks to fetch for reduceId " + reduceId) + //shouldn't get here, just to avoid infinite loop + throw new SparkException("All blocks have been delegated for reduceId " + reduceId) } logInfo("Waiting for new map outputs for reduceId " + reduceId + " ---lirui") Thread.sleep(5000 + 2000 * trialCount) @@ -408,19 +408,36 @@ object BlockFetcherIterator { } override def initialize(){ - iterators += getIterator(true) + iterators += getIterator(initial = true) } - override def hasNext: Boolean = delegatedStatuses.size < statuses.size || iterators.exists(_.hasNext) + override def hasNext: Boolean = { + //firstly see if the delegated iterators have more blocks for us + if (iterators.exists(_.hasNext)) { + return true + } + //If we have blocks not delegated yet, try to delegate them to a new iterator + //and depend on the iterator to tell us if there are valid blocks. + while (delegatedStatuses.size < statuses.size) { + try { + iterators += getIterator() + } catch { + case e: SparkException => return false + } + if (iterators.exists(_.hasNext)) { + return true + } + } + false + } override def next(): (BlockId, Option[Iterator[Any]]) = { - //firstly try to get a block from the iterators we've created + //try to get a block from the iterators we've created for (itr <- iterators if itr.hasNext) { return itr.next() } - //TODO: need to take care of empty blocks here - iterators += getIterator() - next() + //we rely on the iterators for "hasNext", shouldn't get here + throw new SparkException("No more blocks to fetch for reduceId " + reduceId) } override def totalBlocks = iterators.map(_.totalBlocks).sum @@ -433,4 +450,5 @@ object BlockFetcherIterator { override def remoteBytesRead = iterators.map(_.remoteBytesRead).sum } + // End of PartialBlockFetcherIterator } From 2ab311ea945b2f41953ef8d020c7e4451b2d2525 Mon Sep 17 00:00:00 2001 From: lirui Date: Thu, 29 May 2014 20:26:56 +0800 Subject: [PATCH 31/88] RemoveStageBarrier: allow partial map output by default --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 9015ec8cf02b..4f1794d93fe9 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -236,7 +236,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging def clearOutdatedMapStatuses(shuffleId: Int){ mapStatuses.synchronized { //we may have cached partial map outputs, the master may have updates for us - if (mapStatuses.get(shuffleId).isDefined) { + if (mapStatuses.get(shuffleId).isDefined && partialForShuffle.contains(shuffleId)) { val masterEpoch = askTracker(GetMasterEpoch).asInstanceOf[Long] if (masterEpoch > epoch) { logInfo("Master's epoch is " + masterEpoch + ", local epoch is " + epoch + ". Clear local cache. ---lirui") @@ -275,6 +275,8 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) if (mapStatuses.put(shuffleId, new Array[MapStatus](numMaps)).isDefined) { throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice") } + //we allow partial output by default. should be later properly set when register map outputs + partialForShuffle += shuffleId } def registerMapOutput(shuffleId: Int, mapId: Int, status: MapStatus) { From 46da965cccaeed8d2c67b671d3fbac4dbaa2dc85 Mon Sep 17 00:00:00 2001 From: lirui Date: Thu, 29 May 2014 20:49:09 +0800 Subject: [PATCH 32/88] RemoveStageBarrier: make sure the feature is enabled before we use partial fetcher iterator --- .../scala/org/apache/spark/BlockStoreShuffleFetcher.scala | 4 ++++ core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 1 - .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala index 5d93bf5fbd7f..395182f66881 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala @@ -45,6 +45,10 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin shuffleId, reduceId, System.currentTimeMillis - startTime)) val blockFetcherItr = if (statuses.exists(_._1 == null)) { + if (!blockManager.conf.getBoolean("spark.schedule.removeStageBarrier", false)) { + throw new FetchFailedException(null, shuffleId, -1, reduceId, + new Exception("Map statuses contain null, while stage overlap is not enabled.")) + } blockManager.getPartial(statuses, mapOutputTracker, serializer, shuffleId, reduceId) } else { val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 4f1794d93fe9..73de38a262ce 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -241,7 +241,6 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging if (masterEpoch > epoch) { logInfo("Master's epoch is " + masterEpoch + ", local epoch is " + epoch + ". Clear local cache. ---lirui") mapStatuses -= shuffleId - partialForShuffle -= shuffleId epoch = masterEpoch } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 2c9b418630d2..ddda783987cf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -123,7 +123,7 @@ class DAGScheduler( private[scheduler] var eventProcessActor: ActorRef = _ //whether to enable remove stage barrier - val removeStageBarrier = env.conf.getBoolean("spark.schedule.removeStageBarrier", true) + val removeStageBarrier = env.conf.getBoolean("spark.schedule.removeStageBarrier", false) //track the pre-started stages depending on a stage (the key) val dependantStagePreStarted = new mutable.HashMap[Stage, ArrayBuffer[Stage]]() From a89c93f7decba17c5333b8a9950f6679dbe0e554 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 30 May 2014 10:48:54 +0800 Subject: [PATCH 33/88] RemoveStageBarrier: partialForShuffle may cause infinite loop --- .../scala/org/apache/spark/MapOutputTracker.scala | 12 +++++------- .../org/apache/spark/scheduler/DAGScheduler.scala | 1 - 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 73de38a262ce..77f8bb4c672e 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -154,7 +154,6 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging logInfo("Updating epoch from "+epoch+" to " + newEpoch + " and clearing cache") epoch = newEpoch mapStatuses.clear() - partialForShuffle.clear() } } } @@ -162,7 +161,6 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging /** Unregister shuffle data. */ def unregisterShuffle(shuffleId: Int) { mapStatuses.remove(shuffleId) - partialForShuffle -= shuffleId } /** Stop the tracker. */ @@ -205,10 +203,10 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging val fetchedResults = MapOutputTracker.deserializeMapStatuses(fetchedBytes) fetchedStatuses = fetchedResults._1 if (fetchedResults._2) { - logInfo("Got partial map outputs from master for reduceId " + reduceId + ". ---lirui") + logInfo("Got partial map outputs from master for shuffleId " + shuffleId + ". ---lirui") partialForShuffle += shuffleId } else { - logInfo("Got complete map outputs from master for reduceId " + reduceId + ". ---lirui") + logInfo("Got complete map outputs from master for shuffleId " + shuffleId + ". ---lirui") partialForShuffle -= shuffleId } logInfo("Got the output locations") @@ -236,7 +234,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging def clearOutdatedMapStatuses(shuffleId: Int){ mapStatuses.synchronized { //we may have cached partial map outputs, the master may have updates for us - if (mapStatuses.get(shuffleId).isDefined && partialForShuffle.contains(shuffleId)) { + if (mapStatuses.get(shuffleId).isDefined) { val masterEpoch = askTracker(GetMasterEpoch).asInstanceOf[Long] if (masterEpoch > epoch) { logInfo("Master's epoch is " + masterEpoch + ", local epoch is " + epoch + ". Clear local cache. ---lirui") @@ -292,10 +290,10 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) incrementEpoch() } if (isPartial) { - logInfo("Registered partial map outputs. ---lirui") + logInfo("Registered partial map outputs for shuffleId "+shuffleId+". ---lirui") partialForShuffle += shuffleId } else { - logInfo("Registered complete map outputs. ---lirui") + logInfo("Registered complete map outputs for shuffleId "+shuffleId+". ---lirui") partialForShuffle -= shuffleId } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ddda783987cf..2dc3a5e058c8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -854,7 +854,6 @@ class DAGScheduler( stage.addOutputLoc(smt.partitionId, status) //we need to register map outputs progressively if remove stage barrier is enabled if (removeStageBarrier && dependantStagePreStarted.contains(stage) && stage.shuffleDep.isDefined) { - logInfo("Register output progressively: Map task "+smt.partitionId+" ---lirui") mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, changeEpoch = true, isPartial = true) From 5cfbae8426725ee5ef20a5d99b36d6629cc44b79 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 30 May 2014 14:07:37 +0800 Subject: [PATCH 34/88] RemoveStageBarrier: cannot only depend on epoch to determine if the local map statuses are outdated --- .../org/apache/spark/MapOutputTracker.scala | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 77f8bb4c672e..1cfefc2d7f91 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -34,7 +34,7 @@ private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage -private[spark] case object GetMasterEpoch extends MapOutputTrackerMessage +private[spark] case class GetShuffleStatus(shuffleId: Int) extends MapOutputTrackerMessage /** Actor class for MapOutputTrackerMaster */ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster, conf: SparkConf) @@ -65,8 +65,8 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster sender ! true context.stop(self) - case GetMasterEpoch => - sender ! tracker.getEpoch + case GetShuffleStatus(shuffleId: Int) => + sender !(tracker.getEpoch, tracker.isShufflePartial(shuffleId)) } } @@ -235,11 +235,11 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging mapStatuses.synchronized { //we may have cached partial map outputs, the master may have updates for us if (mapStatuses.get(shuffleId).isDefined) { - val masterEpoch = askTracker(GetMasterEpoch).asInstanceOf[Long] - if (masterEpoch > epoch) { - logInfo("Master's epoch is " + masterEpoch + ", local epoch is " + epoch + ". Clear local cache. ---lirui") + val masterShuffleStatus = askTracker(GetShuffleStatus).asInstanceOf[(Long, Boolean)] + if (masterShuffleStatus._1 > epoch || (!masterShuffleStatus._2 && partialForShuffle.contains(shuffleId))) { + logInfo("Master's epoch is " + masterShuffleStatus + ", local epoch is " + epoch + ". Clear local cache. ---lirui") mapStatuses -= shuffleId - epoch = masterEpoch + epoch = masterShuffleStatus._1 } } } @@ -373,6 +373,10 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) mapStatuses.clearOldValues(cleanupTime) cachedSerializedStatuses.clearOldValues(cleanupTime) } + + def isShufflePartial(shuffleId: Int) = { + partialForShuffle.contains(shuffleId) + } } /** From 2df59390b4b04cf9ba14b3cc23228a04c62d3d3d Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 30 May 2014 15:22:01 +0800 Subject: [PATCH 35/88] RemoveStageBarrier: fix bug --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 1cfefc2d7f91..503b2280fe33 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -235,7 +235,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging mapStatuses.synchronized { //we may have cached partial map outputs, the master may have updates for us if (mapStatuses.get(shuffleId).isDefined) { - val masterShuffleStatus = askTracker(GetShuffleStatus).asInstanceOf[(Long, Boolean)] + val masterShuffleStatus = askTracker(GetShuffleStatus(shuffleId)).asInstanceOf[(Long, Boolean)] if (masterShuffleStatus._1 > epoch || (!masterShuffleStatus._2 && partialForShuffle.contains(shuffleId))) { logInfo("Master's epoch is " + masterShuffleStatus + ", local epoch is " + epoch + ". Clear local cache. ---lirui") mapStatuses -= shuffleId From 6891d58be90899b3f8c9a4b41965209e68572d8a Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 3 Jun 2014 21:45:15 +0800 Subject: [PATCH 36/88] RemoveStageBarrier: adjust fetching order of CoGroupedRDD --- .../org/apache/spark/rdd/CoGroupedRDD.scala | 17 +++++++++++++---- .../apache/spark/scheduler/DAGScheduler.scala | 1 + 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 9ff76892aed3..c4c7469d528a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -120,6 +120,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: // A list of (rdd iterator, dependency number) pairs val rddIterators = new ArrayBuffer[(Iterator[Product2[K, Any]], Int)] + //record the shuffle IDs we depend on + val dependedShuffle = new ArrayBuffer[(Int, Int)]() for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => // Read them from the parent @@ -128,10 +130,17 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: case ShuffleCoGroupSplitDep(shuffleId) => // Read map outputs of shuffle - val fetcher = SparkEnv.get.shuffleFetcher - val ser = Serializer.getSerializer(serializer) - val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser) - rddIterators += ((it, depNum)) +// val fetcher = SparkEnv.get.shuffleFetcher +// val ser = Serializer.getSerializer(serializer) +// val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser) +// rddIterators += ((it, depNum)) + dependedShuffle +=((shuffleId, depNum)) + } + for ((shuffleId, depNum) <- dependedShuffle.sortBy(_._1).reverse) { + val fetcher = SparkEnv.get.shuffleFetcher + val ser = Serializer.getSerializer(serializer) + val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser) + rddIterators += ((it, depNum)) } if (!externalSorting) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 2dc3a5e058c8..c43b4bbf0f7c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -930,6 +930,7 @@ class DAGScheduler( val pendingTaskNum = (for (taskSet <- pendingTasks.values) yield taskSet.size).sum val freeCores = totalCores - pendingTaskNum val waitingStageNum = waitingStages.size + //TODO: compare free cores with "spark.task.cpus" if (freeCores > 0 && waitingStageNum > 0 && stage.shuffleDep.isDefined) { logInfo("We have " + totalCores + " CPUs. " + pendingTaskNum + " tasks are running/pending. " + waitingStageNum + " stages are waiting to be submitted. ---lirui") From 5dd28dc841dd5655128e5b42bfc1dc528e671ca1 Mon Sep 17 00:00:00 2001 From: lirui Date: Wed, 4 Jun 2014 20:54:02 +0800 Subject: [PATCH 37/88] RemoveStageBarrier: make sure pre-started stage has lower priority to run --- .../org/apache/spark/scheduler/DAGScheduler.scala | 15 ++++++++++++++- .../scala/org/apache/spark/scheduler/Pool.scala | 3 ++- .../spark/scheduler/SchedulingAlgorithm.scala | 2 +- .../apache/spark/scheduler/TaskSetManager.scala | 3 +++ 4 files changed, 20 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 6041b6613ca8..82febe927614 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1206,7 +1206,7 @@ class DAGScheduler( taskScheduler.stop() } - //select a waiting stage to pre-start + // Select a waiting stage to pre-start private def getPreStartableStage(stage: Stage): Option[Stage] = { //select a stage not ready to run for (waitingStage <- waitingStages) { @@ -1218,6 +1218,19 @@ class DAGScheduler( } None } + + // Check if the given stageId is a pre-started stage + def isPreStartStage(stageId: Int): Boolean = { + if (stageIdToStage.contains(stageId)) { + val stage = stageIdToStage(stageId) + for (preStartedStages <- dependantStagePreStarted.values) { + if (preStartedStages.contains(stage)) { + return true + } + } + } + false + } } private[scheduler] class DAGSchedulerActorSupervisor(dagScheduler: DAGScheduler) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 174b73221afc..3ccb1597b1dc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -102,7 +102,8 @@ private[spark] class Pool( for (schedulable <- sortedSchedulableQueue) { sortedTaskSetQueue ++= schedulable.getSortedTaskSetQueue } - sortedTaskSetQueue + val partitionedTaskSets = sortedTaskSetQueue.partition(!_.isPreStart()) + partitionedTaskSets._1 ++ partitionedTaskSets._2 } def increaseRunningTasks(taskNum: Int) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala index 393b483d695b..5e62c8468f00 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala @@ -34,7 +34,7 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { if (res == 0) { val stageId1 = s1.stageId val stageId2 = s2.stageId - res = math.signum(stageId2 - stageId1) + res = math.signum(stageId1 - stageId2) } if (res < 0) { true diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index ed38ba755cc9..8b905ad24535 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -742,4 +742,7 @@ private[spark] class TaskSetManager( logDebug("Valid locality levels for " + taskSet + ": " + levels.mkString(", ")) levels.toArray } + + // Test if this stage is in pre-start state + def isPreStart() = sched.dagScheduler.isPreStartStage(stageId) } From 6cdf2a3fcc85be5a91537cd845db1f0e39929eec Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 9 Jun 2014 09:16:50 +0800 Subject: [PATCH 38/88] RemoveStageBarrier: revert previous changes to CoGroupedRDD --- .../org/apache/spark/rdd/CoGroupedRDD.scala | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index c4c7469d528a..9ff76892aed3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -120,8 +120,6 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: // A list of (rdd iterator, dependency number) pairs val rddIterators = new ArrayBuffer[(Iterator[Product2[K, Any]], Int)] - //record the shuffle IDs we depend on - val dependedShuffle = new ArrayBuffer[(Int, Int)]() for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => // Read them from the parent @@ -130,17 +128,10 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: case ShuffleCoGroupSplitDep(shuffleId) => // Read map outputs of shuffle -// val fetcher = SparkEnv.get.shuffleFetcher -// val ser = Serializer.getSerializer(serializer) -// val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser) -// rddIterators += ((it, depNum)) - dependedShuffle +=((shuffleId, depNum)) - } - for ((shuffleId, depNum) <- dependedShuffle.sortBy(_._1).reverse) { - val fetcher = SparkEnv.get.shuffleFetcher - val ser = Serializer.getSerializer(serializer) - val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser) - rddIterators += ((it, depNum)) + val fetcher = SparkEnv.get.shuffleFetcher + val ser = Serializer.getSerializer(serializer) + val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser) + rddIterators += ((it, depNum)) } if (!externalSorting) { From af000f7f3427ffd0b49ab68c0961bb3dbb37f222 Mon Sep 17 00:00:00 2001 From: lirui Date: Wed, 11 Jun 2014 15:35:42 +0800 Subject: [PATCH 39/88] RemoveStageBarrier: sleep less waiting for new map outputs --- .../scala/org/apache/spark/BlockStoreShuffleFetcher.scala | 2 +- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- .../scala/org/apache/spark/storage/BlockFetcherIterator.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala index 395182f66881..f219e46d4f2b 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala @@ -45,7 +45,7 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin shuffleId, reduceId, System.currentTimeMillis - startTime)) val blockFetcherItr = if (statuses.exists(_._1 == null)) { - if (!blockManager.conf.getBoolean("spark.schedule.removeStageBarrier", false)) { + if (!blockManager.conf.getBoolean("spark.scheduler.removeStageBarrier", false)) { throw new FetchFailedException(null, shuffleId, -1, reduceId, new Exception("Map statuses contain null, while stage overlap is not enabled.")) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 82febe927614..e8b50bf4fc71 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -122,8 +122,8 @@ class DAGScheduler( private[scheduler] var eventProcessActor: ActorRef = _ - //whether to enable remove stage barrier - val removeStageBarrier = env.conf.getBoolean("spark.schedule.removeStageBarrier", false) + // Whether to enable remove stage barrier + val removeStageBarrier = env.conf.getBoolean("spark.scheduler.removeStageBarrier", false) //track the pre-started stages depending on a stage (the key) val dependantStagePreStarted = new mutable.HashMap[Stage, ArrayBuffer[Stage]]() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 9d9804ec693b..76bf33a4f75f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -390,7 +390,7 @@ object BlockFetcherIterator { throw new SparkException("All blocks have been delegated for reduceId " + reduceId) } logInfo("Waiting for new map outputs for reduceId " + reduceId + " ---lirui") - Thread.sleep(5000 + 2000 * trialCount) + Thread.sleep(2000 + 1000 * trialCount) updateStatuses() trialCount += 1 } From 83494247064c2709a4f64a655ea11b05ce687192 Mon Sep 17 00:00:00 2001 From: lirui Date: Thu, 12 Jun 2014 17:05:24 +0800 Subject: [PATCH 40/88] RemoveStageBarrier: don't rely on epoch for updated map statuses --- .../org/apache/spark/MapOutputTracker.scala | 36 +++++++++++-------- .../spark/storage/BlockFetcherIterator.scala | 4 +-- 2 files changed, 22 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 503b2280fe33..b7f79e361939 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -66,7 +66,7 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster context.stop(self) case GetShuffleStatus(shuffleId: Int) => - sender !(tracker.getEpoch, tracker.isShufflePartial(shuffleId)) + sender ! tracker.completenessForShuffle(shuffleId) } } @@ -229,21 +229,29 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } } - //clear outdated map outputs for a shuffle and sync the local epoch with master - //this method is intended to be followed by "getServerStatuses" - def clearOutdatedMapStatuses(shuffleId: Int){ + // Clear outdated map outputs for a shuffle + def clearOutdatedMapStatuses(shuffleId: Int) { mapStatuses.synchronized { - //we may have cached partial map outputs, the master may have updates for us if (mapStatuses.get(shuffleId).isDefined) { - val masterShuffleStatus = askTracker(GetShuffleStatus(shuffleId)).asInstanceOf[(Long, Boolean)] - if (masterShuffleStatus._1 > epoch || (!masterShuffleStatus._2 && partialForShuffle.contains(shuffleId))) { - logInfo("Master's epoch is " + masterShuffleStatus + ", local epoch is " + epoch + ". Clear local cache. ---lirui") + val masterCompleteness = askTracker(GetShuffleStatus(shuffleId)).asInstanceOf[Int] + val diff = masterCompleteness - completenessForShuffle(shuffleId) + if (diff > 0) { + logInfo("Master is " + diff + " map statuses ahead of us. Clear local cache. ---lirui") mapStatuses -= shuffleId - epoch = masterShuffleStatus._1 } } } } + + // Compute the completeness of a shuffle + def completenessForShuffle(shuffleId: Int): Int = { + mapStatuses.synchronized { + if (mapStatuses.get(shuffleId).isDefined) { + return mapStatuses.get(shuffleId).get.count(_ != null) + } + } + 0 + } } /** @@ -272,7 +280,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) if (mapStatuses.put(shuffleId, new Array[MapStatus](numMaps)).isDefined) { throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice") } - //we allow partial output by default. should be later properly set when register map outputs + // We allow partial output by default. Should be later properly set when register map outputs partialForShuffle += shuffleId } @@ -285,7 +293,9 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) /** Register multiple map output information for the given shuffle */ def registerMapOutputs(shuffleId: Int, statuses: Array[MapStatus], changeEpoch: Boolean = false, isPartial: Boolean = false) { - mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses) + mapStatuses.synchronized{ + mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses) + } if (changeEpoch) { incrementEpoch() } @@ -373,10 +383,6 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) mapStatuses.clearOldValues(cleanupTime) cachedSerializedStatuses.clearOldValues(cleanupTime) } - - def isShufflePartial(shuffleId: Int) = { - partialForShuffle.contains(shuffleId) - } } /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 76bf33a4f75f..1e6d8e6b7c4a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -383,16 +383,14 @@ object BlockFetcherIterator { logInfo("Still missing " + statuses.filter(_._1 == null).size + " map outputs for reduceId " + reduceId + " ---lirui") updateStatuses() } - var trialCount = 0 while (!newStatusesReady) { if (!isPartial && delegatedStatuses.size >= statuses.size) { //shouldn't get here, just to avoid infinite loop throw new SparkException("All blocks have been delegated for reduceId " + reduceId) } logInfo("Waiting for new map outputs for reduceId " + reduceId + " ---lirui") - Thread.sleep(2000 + 1000 * trialCount) + Thread.sleep(2000) updateStatuses() - trialCount += 1 } val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] for (index <- readyStatuses if !delegatedStatuses.contains(index)) { From 28679b997c4b449f85813e40e5067125a236c2bd Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 13 Jun 2014 14:01:55 +0800 Subject: [PATCH 41/88] RemoveStageBarrier: add a proxy to update partial map outputs periodically --- .../org/apache/spark/MapOutputTracker.scala | 28 ++++++++++++++++--- .../spark/storage/BlockFetcherIterator.scala | 5 ++-- 2 files changed, 26 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index b7f79e361939..7b888f2b872c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -19,9 +19,12 @@ package org.apache.spark import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} +import java.util.concurrent._ +import java.util.Collections import scala.collection.mutable.{HashSet, HashMap, Map} import scala.concurrent.Await +import scala.collection.JavaConversions._ import akka.actor._ import akka.pattern.ask @@ -91,8 +94,9 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging //TODO: we should also record if the output for a shuffle is partial protected val mapStatuses: Map[Int, Array[MapStatus]] - //track if we have partial map outputs for a shuffle - protected val partialForShuffle = new mutable.HashSet[Int]() + // Track if we have partial map outputs for a shuffle + protected val partialForShuffle = + Collections.newSetFromMap[Int](new ConcurrentHashMap[Int, java.lang.Boolean]()) /** * Incremented every time a fetch fails so that client nodes know to clear @@ -204,7 +208,10 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging fetchedStatuses = fetchedResults._1 if (fetchedResults._2) { logInfo("Got partial map outputs from master for shuffleId " + shuffleId + ". ---lirui") - partialForShuffle += shuffleId + if(!partialForShuffle.contains(shuffleId)){ + partialForShuffle += shuffleId + new Thread(new MapStatusUpdater(shuffleId)).start() + } } else { logInfo("Got complete map outputs from master for shuffleId " + shuffleId + ". ---lirui") partialForShuffle -= shuffleId @@ -230,7 +237,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } // Clear outdated map outputs for a shuffle - def clearOutdatedMapStatuses(shuffleId: Int) { + private def clearOutdatedMapStatuses(shuffleId: Int) { mapStatuses.synchronized { if (mapStatuses.get(shuffleId).isDefined) { val masterCompleteness = askTracker(GetShuffleStatus(shuffleId)).asInstanceOf[Int] @@ -252,6 +259,19 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } 0 } + + // A proxy to update partial map statuses periodically + class MapStatusUpdater(shuffleId: Int) extends Runnable { + override def run() { + logInfo("Updater started for shuffleId "+shuffleId+". ---lirui") + while (partialForShuffle.contains(shuffleId)) { + Thread.sleep(1000) + clearOutdatedMapStatuses(shuffleId) + getMapStatusesForShuffle(shuffleId, -1) + } + logInfo("Map status for shuffleId "+shuffleId+" is now complete. Updater terminated. ---lirui") + } + } } /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 1e6d8e6b7c4a..1fd8cfe62838 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -366,10 +366,9 @@ object BlockFetcherIterator { //check if the map output is partial private def isPartial = statuses.exists(_._1 == null) - //get the updated map output + // Get the updated map output private def updateStatuses() { logInfo("Trying to update map statuses for reduceId "+reduceId+" ---lirui") - mapOutputTracker.clearOutdatedMapStatuses(shuffleId) statuses = mapOutputTracker.getServerStatuses(shuffleId, reduceId) } @@ -389,7 +388,7 @@ object BlockFetcherIterator { throw new SparkException("All blocks have been delegated for reduceId " + reduceId) } logInfo("Waiting for new map outputs for reduceId " + reduceId + " ---lirui") - Thread.sleep(2000) + Thread.sleep(1000) updateStatuses() } val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] From 71b87e50df6e52c1f989276b7571faef276f20d2 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 13 Jun 2014 15:38:57 +0800 Subject: [PATCH 42/88] RemoveStageBarrier: remove verbose logs --- .../src/main/scala/org/apache/spark/MapOutputTracker.scala | 6 ++---- .../org/apache/spark/storage/BlockFetcherIterator.scala | 7 ------- 2 files changed, 2 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 7b888f2b872c..1a4601677b4c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -170,7 +170,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging /** Stop the tracker. */ def stop() { } - //get map statuses for a shuffle + // Get map statuses for a shuffle private def getMapStatusesForShuffle(shuffleId: Int, reduceId: Int): Array[MapStatus]={ val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { @@ -243,7 +243,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging val masterCompleteness = askTracker(GetShuffleStatus(shuffleId)).asInstanceOf[Int] val diff = masterCompleteness - completenessForShuffle(shuffleId) if (diff > 0) { - logInfo("Master is " + diff + " map statuses ahead of us. Clear local cache. ---lirui") + logInfo("Master is " + diff + " map statuses ahead of us for shuffleId "+shuffleId+". Clear local cache. ---lirui") mapStatuses -= shuffleId } } @@ -320,10 +320,8 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) incrementEpoch() } if (isPartial) { - logInfo("Registered partial map outputs for shuffleId "+shuffleId+". ---lirui") partialForShuffle += shuffleId } else { - logInfo("Registered complete map outputs for shuffleId "+shuffleId+". ---lirui") partialForShuffle -= shuffleId } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 1fd8cfe62838..b95bb01b45a7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -133,7 +133,6 @@ object BlockFetcherIterator { _remoteBytesRead += networkSize logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } - logInfo("Got "+blockMessageArray.length+" remote blocks in "+Utils.getUsedTimeMs(startTime)+" ---lirui") } case None => { logError("Could not get block(s) from " + cmId) @@ -219,18 +218,12 @@ object BlockFetcherIterator { // Add the remote requests into our queue in a random order fetchRequests ++= Utils.randomize(remoteRequests) - //how many remote blocks we're fetching during initialization - var initialRemoteBlocks = 0 - // Send out initial requests for blocks, up to our maxBytesInFlight while (!fetchRequests.isEmpty && (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - initialRemoteBlocks += fetchRequests.front.blocks.size sendRequest(fetchRequests.dequeue()) } - logInfo("Fetching "+initialRemoteBlocks+" out of "+remoteBlocksToFetch.size+" remote blocks for initialization. ---lirui") - val numFetches = remoteRequests.size - fetchRequests.size logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) From 04f17e890513dc58fd094700f1710f1054ec06a4 Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 16 Jun 2014 14:22:07 +0800 Subject: [PATCH 43/88] RemoveStageBarrier: don't increase epoch for partial map output registration --- .../org/apache/spark/MapOutputTracker.scala | 3 ++- .../apache/spark/scheduler/DAGScheduler.scala | 10 ++++---- .../spark/storage/BlockFetcherIterator.scala | 25 ++++++++++--------- 3 files changed, 20 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 1a4601677b4c..2e91ff429fdc 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -364,7 +364,8 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) var statuses: Array[MapStatus] = null var epochGotten: Long = -1 epochLock.synchronized { - if (epoch > cacheEpoch) { + // Don't use the cached version if outputs are partial + if (epoch > cacheEpoch || partialForShuffle.contains(shuffleId)) { cachedSerializedStatuses.clear() cacheEpoch = epoch } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index e8b50bf4fc71..4290c3edcecd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -856,11 +856,11 @@ class DAGScheduler( logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId) } else { stage.addOutputLoc(smt.partitionId, status) - //we need to register map outputs progressively if remove stage barrier is enabled + // Need to register map outputs progressively if remove stage barrier is enabled if (removeStageBarrier && dependantStagePreStarted.contains(stage) && stage.shuffleDep.isDefined) { mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, - changeEpoch = true, isPartial = true) + changeEpoch = false, isPartial = true) } } if (runningStages.contains(stage) && pendingTasks(stage).isEmpty) { @@ -889,7 +889,7 @@ class DAGScheduler( logInfo("Resubmitting " + stage + " (" + stage.name + ") because some of its tasks had failed: " + stage.outputLocs.zipWithIndex.filter(_._1 == Nil).map(_._2).mkString(", ")) - //Pre-started dependant stages should fail + // Pre-started dependant stages should fail if (dependantStagePreStarted.contains(stage)) { for (preStartedStage <- dependantStagePreStarted.get(stage).get) { runningStages -= preStartedStage @@ -924,7 +924,7 @@ class DAGScheduler( } } } else { - //ShuffleMap stage not finished yet. Maybe we can remove the stage barrier here. + // ShuffleMap stage not finished yet. Maybe we can remove the stage barrier here. if(removeStageBarrier){ //TODO: need a better way to get the number of total CPUs if (taskScheduler.isInstanceOf[TaskSchedulerImpl] && taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.isInstanceOf[CoarseGrainedSchedulerBackend]) { @@ -945,7 +945,7 @@ class DAGScheduler( //register map output finished so far mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, - changeEpoch = true, isPartial = true) + changeEpoch = false, isPartial = true) waitingStages -= preStartedStage runningStages += preStartedStage //inform parent stages that the dependant stage has been pre-started diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index b95bb01b45a7..cbeadf546d1b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -353,15 +353,15 @@ object BlockFetcherIterator { extends BlockFetcherIterator { private val iterators=new ArrayBuffer[BlockFetcherIterator]() - //track the map outputs we've delegated + // Track the map outputs we've delegated private val delegatedStatuses = new HashSet[Int]() - //check if the map output is partial + // Check if the map output is partial private def isPartial = statuses.exists(_._1 == null) // Get the updated map output private def updateStatuses() { - logInfo("Trying to update map statuses for reduceId "+reduceId+" ---lirui") + logInfo("Updating map statuses for reduceId "+reduceId+" ---lirui") statuses = mapOutputTracker.getServerStatuses(shuffleId, reduceId) } @@ -402,31 +402,32 @@ object BlockFetcherIterator { } override def hasNext: Boolean = { - //firstly see if the delegated iterators have more blocks for us + // Firstly see if the delegated iterators have more blocks for us if (iterators.exists(_.hasNext)) { return true } - //If we have blocks not delegated yet, try to delegate them to a new iterator - //and depend on the iterator to tell us if there are valid blocks. + // If we have blocks not delegated yet, try to delegate them to a new iterator + // and depend on the iterator to tell us if there are valid blocks. while (delegatedStatuses.size < statuses.size) { try { - iterators += getIterator() + val newItr = getIterator() + iterators += newItr + if (newItr.hasNext) { + return true + } } catch { case e: SparkException => return false } - if (iterators.exists(_.hasNext)) { - return true - } } false } override def next(): (BlockId, Option[Iterator[Any]]) = { - //try to get a block from the iterators we've created + // Try to get a block from the iterators we've created for (itr <- iterators if itr.hasNext) { return itr.next() } - //we rely on the iterators for "hasNext", shouldn't get here + // We rely on the iterators for "hasNext", shouldn't get here throw new SparkException("No more blocks to fetch for reduceId " + reduceId) } From eafa47638269e44dbe34820f61ae314553da4330 Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 16 Jun 2014 14:30:10 +0800 Subject: [PATCH 44/88] RemoveStageBarrier: don't put partial outputs in cache --- .../main/scala/org/apache/spark/MapOutputTracker.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 2e91ff429fdc..8e2a087e47d8 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -363,9 +363,9 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) def getSerializedMapOutputStatuses(shuffleId: Int): Array[Byte] = { var statuses: Array[MapStatus] = null var epochGotten: Long = -1 + val partial = partialForShuffle.contains(shuffleId) epochLock.synchronized { - // Don't use the cached version if outputs are partial - if (epoch > cacheEpoch || partialForShuffle.contains(shuffleId)) { + if (epoch > cacheEpoch) { cachedSerializedStatuses.clear() cacheEpoch = epoch } @@ -379,11 +379,12 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } // If we got here, we failed to find the serialized locations in the cache, so we pulled // out a snapshot of the locations as "statuses"; let's serialize and return that - val bytes = MapOutputTracker.serializeMapStatuses(statuses,isPartial = partialForShuffle.contains(shuffleId)) + val bytes = MapOutputTracker.serializeMapStatuses(statuses,isPartial = partial) logInfo("Size of output statuses for shuffle %d is %d bytes".format(shuffleId, bytes.length)) // Add them into the table only if the epoch hasn't changed while we were working epochLock.synchronized { - if (epoch == epochGotten) { + // Don't put partial outputs in cache + if (epoch == epochGotten && !partial) { cachedSerializedStatuses(shuffleId) = bytes } } From 754768630245974a30a169226ff05f0d6442cd8f Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 17 Jun 2014 11:21:46 +0800 Subject: [PATCH 45/88] RemoveStageBarrier: block reducers waiting for new map outputs --- .../org/apache/spark/MapOutputTracker.scala | 65 +++++++++++++------ .../spark/storage/BlockFetcherIterator.scala | 12 ++-- 2 files changed, 54 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 8e2a087e47d8..7b8ea8de4f67 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -98,6 +98,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging protected val partialForShuffle = Collections.newSetFromMap[Int](new ConcurrentHashMap[Int, java.lang.Boolean]()) + protected val partialEpoch = new mutable.HashMap[Int, Int]() + /** * Incremented every time a fetch fails so that client nodes know to clear * their cache of map output locations if this happens. @@ -237,25 +239,25 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } // Clear outdated map outputs for a shuffle - private def clearOutdatedMapStatuses(shuffleId: Int) { - mapStatuses.synchronized { - if (mapStatuses.get(shuffleId).isDefined) { - val masterCompleteness = askTracker(GetShuffleStatus(shuffleId)).asInstanceOf[Int] - val diff = masterCompleteness - completenessForShuffle(shuffleId) - if (diff > 0) { - logInfo("Master is " + diff + " map statuses ahead of us for shuffleId "+shuffleId+". Clear local cache. ---lirui") - mapStatuses -= shuffleId - } + private def clearOutdatedMapStatuses(shuffleId: Int): Boolean = { + if (mapStatuses.contains(shuffleId)) { + val masterCompleteness = askTracker(GetShuffleStatus(shuffleId)).asInstanceOf[Int] + val diff = masterCompleteness - completenessForShuffle(shuffleId) + if (diff > 0) { + logInfo("Master is " + diff + " map statuses ahead of us for shuffleId " + shuffleId + ". Clear local cache. ---lirui") + mapStatuses -= shuffleId + return true + } else { + return false } } + true } // Compute the completeness of a shuffle def completenessForShuffle(shuffleId: Int): Int = { - mapStatuses.synchronized { - if (mapStatuses.get(shuffleId).isDefined) { - return mapStatuses.get(shuffleId).get.count(_ != null) - } + if (mapStatuses.get(shuffleId).isDefined) { + return mapStatuses.get(shuffleId).get.count(_ != null) } 0 } @@ -263,13 +265,39 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging // A proxy to update partial map statuses periodically class MapStatusUpdater(shuffleId: Int) extends Runnable { override def run() { + partialEpoch.synchronized { + if (!partialEpoch.contains(shuffleId)) { + partialEpoch.put(shuffleId, 0) + } + } logInfo("Updater started for shuffleId "+shuffleId+". ---lirui") while (partialForShuffle.contains(shuffleId)) { Thread.sleep(1000) - clearOutdatedMapStatuses(shuffleId) - getMapStatusesForShuffle(shuffleId, -1) + if (clearOutdatedMapStatuses(shuffleId)) { + getMapStatusesForShuffle(shuffleId, -1) + partialEpoch.synchronized { + partialEpoch.put(shuffleId, partialEpoch.getOrElse(shuffleId, 0) + 1) + partialEpoch.notifyAll() + } + } } logInfo("Map status for shuffleId "+shuffleId+" is now complete. Updater terminated. ---lirui") + partialEpoch.synchronized { + partialEpoch.remove(shuffleId) + partialEpoch.notifyAll() + } + } + } + + def getUpdatedStatus(shuffleId: Int, reduceId: Int, localEpoch: Int): (Array[(BlockManagerId, Long)], Int) = { + partialEpoch.synchronized { + if (!partialEpoch.contains(shuffleId)) { + return (getServerStatuses(shuffleId, reduceId), 0) + } + if (partialEpoch.get(shuffleId).get <= localEpoch) { + partialEpoch.wait() + } + (getServerStatuses(shuffleId, reduceId), partialEpoch.getOrElse(shuffleId, 0)) } } } @@ -313,9 +341,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) /** Register multiple map output information for the given shuffle */ def registerMapOutputs(shuffleId: Int, statuses: Array[MapStatus], changeEpoch: Boolean = false, isPartial: Boolean = false) { - mapStatuses.synchronized{ - mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses) - } + mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses) if (changeEpoch) { incrementEpoch() } @@ -363,7 +389,6 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) def getSerializedMapOutputStatuses(shuffleId: Int): Array[Byte] = { var statuses: Array[MapStatus] = null var epochGotten: Long = -1 - val partial = partialForShuffle.contains(shuffleId) epochLock.synchronized { if (epoch > cacheEpoch) { cachedSerializedStatuses.clear() @@ -379,6 +404,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } // If we got here, we failed to find the serialized locations in the cache, so we pulled // out a snapshot of the locations as "statuses"; let's serialize and return that + val partial = partialForShuffle.contains(shuffleId) val bytes = MapOutputTracker.serializeMapStatuses(statuses,isPartial = partial) logInfo("Size of output statuses for shuffle %d is %d bytes".format(shuffleId, bytes.length)) // Add them into the table only if the epoch hasn't changed while we were working @@ -411,6 +437,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) */ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) { protected val mapStatuses = new HashMap[Int, Array[MapStatus]] + with mutable.SynchronizedMap[Int, Array[MapStatus]] } private[spark] object MapOutputTracker { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index cbeadf546d1b..2cbfbca037ce 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -356,13 +356,17 @@ object BlockFetcherIterator { // Track the map outputs we've delegated private val delegatedStatuses = new HashSet[Int]() + private var localEpoch = 0 + // Check if the map output is partial private def isPartial = statuses.exists(_._1 == null) // Get the updated map output private def updateStatuses() { - logInfo("Updating map statuses for reduceId "+reduceId+" ---lirui") - statuses = mapOutputTracker.getServerStatuses(shuffleId, reduceId) + logInfo("Updating map statuses for reduceId " + reduceId + " ---lirui") + val update = mapOutputTracker.getUpdatedStatus(shuffleId, reduceId, localEpoch) + statuses = update._1 + localEpoch = update._2 } private def readyStatuses = (0 until statuses.size).filter(statuses(_)._1 != null) @@ -380,8 +384,8 @@ object BlockFetcherIterator { //shouldn't get here, just to avoid infinite loop throw new SparkException("All blocks have been delegated for reduceId " + reduceId) } - logInfo("Waiting for new map outputs for reduceId " + reduceId + " ---lirui") - Thread.sleep(1000) +// logInfo("Waiting for new map outputs for reduceId " + reduceId + " ---lirui") +// Thread.sleep(1000) updateStatuses() } val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] From ca83d1955d398003a300a7256ad0ecd49ab8f4ea Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 17 Jun 2014 11:30:28 +0800 Subject: [PATCH 46/88] RemoveStageBarrier: bug fix --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 7b8ea8de4f67..2a4e79615e0f 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -283,7 +283,6 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } logInfo("Map status for shuffleId "+shuffleId+" is now complete. Updater terminated. ---lirui") partialEpoch.synchronized { - partialEpoch.remove(shuffleId) partialEpoch.notifyAll() } } From 539f1a8018827839753f6f532301df3f4cc897bc Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 17 Jun 2014 15:48:27 +0800 Subject: [PATCH 47/88] RemoveStageBarrier: add API to SchedulerBackend to tell if there's free slots --- .../apache/spark/scheduler/DAGScheduler.scala | 48 ++++++++----------- .../spark/scheduler/SchedulerBackend.scala | 2 + .../CoarseGrainedSchedulerBackend.scala | 8 ++++ .../spark/storage/BlockFetcherIterator.scala | 13 ++--- 4 files changed, 33 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 4290c3edcecd..c725eaf0c925 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -925,35 +925,26 @@ class DAGScheduler( } } else { // ShuffleMap stage not finished yet. Maybe we can remove the stage barrier here. - if(removeStageBarrier){ - //TODO: need a better way to get the number of total CPUs - if (taskScheduler.isInstanceOf[TaskSchedulerImpl] && taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.isInstanceOf[CoarseGrainedSchedulerBackend]) { - val backend = taskScheduler.asInstanceOf[TaskSchedulerImpl].backend.asInstanceOf[CoarseGrainedSchedulerBackend] - //check CPU usage - val totalCores = backend.totalCoreCount.get() - val pendingTaskNum = (for (taskSet <- pendingTasks.values) yield taskSet.size).sum - val freeCores = totalCores - pendingTaskNum - val waitingStageNum = waitingStages.size - //TODO: compare free cores with "spark.task.cpus" - if (freeCores > 0 && waitingStageNum > 0 && stage.shuffleDep.isDefined) { - logInfo("We have " + totalCores + " CPUs. " + pendingTaskNum + " tasks are running/pending. " + - waitingStageNum + " stages are waiting to be submitted. ---lirui") - val preStartableStage = getPreStartableStage(stage) - if (preStartableStage.isDefined) { - val preStartedStage = preStartableStage.get - logInfo("Pre-start stage " + preStartedStage.id + " ---lirui") - //register map output finished so far - mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, - stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, - changeEpoch = false, isPartial = true) - waitingStages -= preStartedStage - runningStages += preStartedStage - //inform parent stages that the dependant stage has been pre-started - for (parentStage <- getMissingParentStages(preStartedStage)) { - dependantStagePreStarted.getOrElseUpdate(parentStage, new ArrayBuffer[Stage]()) += preStartedStage - } - submitMissingTasks(preStartedStage, activeJobForStage(preStartedStage).get) + if(removeStageBarrier && taskScheduler.isInstanceOf[TaskSchedulerImpl]){ + // TODO: need a better way to check if there's free slots + val backend = taskScheduler.asInstanceOf[TaskSchedulerImpl].backend + val waitingStageNum = waitingStages.size + if (backend.freeSlotAvail() && waitingStageNum > 0 && stage.shuffleDep.isDefined) { + val preStartableStage = getPreStartableStage(stage) + if (preStartableStage.isDefined) { + val preStartedStage = preStartableStage.get + logInfo("Pre-start stage " + preStartedStage.id + " ---lirui") + // Register map output finished so far + mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, + stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, + changeEpoch = false, isPartial = true) + waitingStages -= preStartedStage + runningStages += preStartedStage + // Inform parent stages that the dependant stage has been pre-started + for (parentStage <- getMissingParentStages(preStartedStage)) { + dependantStagePreStarted.getOrElseUpdate(parentStage, new ArrayBuffer[Stage]()) += preStartedStage } + submitMissingTasks(preStartedStage, activeJobForStage(preStartedStage).get) } } } @@ -1208,7 +1199,6 @@ class DAGScheduler( // Select a waiting stage to pre-start private def getPreStartableStage(stage: Stage): Option[Stage] = { - //select a stage not ready to run for (waitingStage <- waitingStages) { val missingParents = getMissingParentStages(waitingStage) if (missingParents != Nil && missingParents.contains(stage) && diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 6a6d8e609bc3..a30bcc811898 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -30,4 +30,6 @@ private[spark] trait SchedulerBackend { def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = throw new UnsupportedOperationException + + def freeSlotAvail(): Boolean = false } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index e47a060683a2..a31d1ade007c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -46,6 +46,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) + // Also track number of free cores + private val freeCoreCount = new AtomicInteger(0) val conf = scheduler.sc.conf private val timeout = AkkaUtils.askTimeout(conf) private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) @@ -83,6 +85,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A executorAddress(executorId) = sender.path.address addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) + freeCoreCount.addAndGet(cores) makeOffers() } @@ -91,6 +94,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A if (TaskState.isFinished(state)) { if (executorActor.contains(executorId)) { freeCores(executorId) += scheduler.CPUS_PER_TASK + freeCoreCount.addAndGet(scheduler.CPUS_PER_TASK) makeOffers(executorId) } else { // Ignoring the update since we don't know about the executor. @@ -159,6 +163,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } else { freeCores(task.executorId) -= scheduler.CPUS_PER_TASK + freeCoreCount.addAndGet(-scheduler.CPUS_PER_TASK) executorActor(task.executorId) ! LaunchTask(new SerializableBuffer(serializedTask)) } } @@ -176,6 +181,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A totalCores -= executorId freeCores -= executorId totalCoreCount.addAndGet(-numCores) + freeCoreCount.addAndGet(-numCores) scheduler.executorLost(executorId, SlaveLost(reason)) } } @@ -244,6 +250,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A throw new SparkException("Error notifying standalone scheduler's driver actor", e) } } + + override def freeSlotAvail(): Boolean = freeCoreCount.get() > 0 } private[spark] object CoarseGrainedSchedulerBackend { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 2cbfbca037ce..cf4776169a35 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -363,7 +363,7 @@ object BlockFetcherIterator { // Get the updated map output private def updateStatuses() { - logInfo("Updating map statuses for reduceId " + reduceId + " ---lirui") + logInfo("Waiting for new map statuses for reduceId " + reduceId + " ---lirui") val update = mapOutputTracker.getUpdatedStatus(shuffleId, reduceId, localEpoch) statuses = update._1 localEpoch = update._2 @@ -374,18 +374,13 @@ object BlockFetcherIterator { //check if there's new map outputs ready to collect private def newStatusesReady = readyStatuses.exists(!delegatedStatuses.contains(_)) - private def getIterator(initial: Boolean = false) = { - if (isPartial && !initial) { - logInfo("Still missing " + statuses.filter(_._1 == null).size + " map outputs for reduceId " + reduceId + " ---lirui") - updateStatuses() - } + private def getIterator() = { while (!newStatusesReady) { if (!isPartial && delegatedStatuses.size >= statuses.size) { //shouldn't get here, just to avoid infinite loop throw new SparkException("All blocks have been delegated for reduceId " + reduceId) } -// logInfo("Waiting for new map outputs for reduceId " + reduceId + " ---lirui") -// Thread.sleep(1000) + logInfo("Still missing " + statuses.filter(_._1 == null).size + " map outputs for reduceId " + reduceId + " ---lirui") updateStatuses() } val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] @@ -402,7 +397,7 @@ object BlockFetcherIterator { } override def initialize(){ - iterators += getIterator(initial = true) + iterators += getIterator() } override def hasNext: Boolean = { From 6e10488c40d7952ca11fce42c142c888fcfefc8b Mon Sep 17 00:00:00 2001 From: lirui Date: Wed, 18 Jun 2014 11:00:53 +0800 Subject: [PATCH 48/88] RemoveStageBarrier: refine logs --- .../org/apache/spark/storage/BlockFetcherIterator.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index cf4776169a35..75c8a3e47681 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -363,7 +363,7 @@ object BlockFetcherIterator { // Get the updated map output private def updateStatuses() { - logInfo("Waiting for new map statuses for reduceId " + reduceId + " ---lirui") + logInfo("Waiting for new map statuses for reduceId " + reduceId + " of shuffleId " + shuffleId + " ---lirui") val update = mapOutputTracker.getUpdatedStatus(shuffleId, reduceId, localEpoch) statuses = update._1 localEpoch = update._2 @@ -380,7 +380,8 @@ object BlockFetcherIterator { //shouldn't get here, just to avoid infinite loop throw new SparkException("All blocks have been delegated for reduceId " + reduceId) } - logInfo("Still missing " + statuses.filter(_._1 == null).size + " map outputs for reduceId " + reduceId + " ---lirui") + logInfo("Still missing " + statuses.filter(_._1 == null).size + + " map outputs for reduceId " + reduceId + " of shuffleId " + shuffleId + " ---lirui") updateStatuses() } val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] @@ -392,7 +393,8 @@ object BlockFetcherIterator { case (address, splits) => (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) } - logInfo("Delegating " + blocksByAddress.map(_._2.size).sum + " blocks to a new iterator for reduceId "+reduceId+" ---lirui") + logInfo("Delegating " + blocksByAddress.map(_._2.size).sum + + " blocks to a new iterator for reduceId " + reduceId + " of shuffleId " + shuffleId + " ---lirui") blockManager.getMultiple(blocksByAddress, serializer) } From a48d592c83fdbe8f38e30582f546c4c7df1c7a04 Mon Sep 17 00:00:00 2001 From: lirui Date: Wed, 18 Jun 2014 20:52:35 +0800 Subject: [PATCH 49/88] RemoveStageBarrier: fix the way we compute free slots --- .../main/scala/org/apache/spark/MapOutputTracker.scala | 2 ++ .../org/apache/spark/scheduler/DAGScheduler.scala | 5 +++-- .../org/apache/spark/scheduler/SchedulerBackend.scala | 2 +- .../cluster/CoarseGrainedSchedulerBackend.scala | 10 +++------- 4 files changed, 9 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 2a4e79615e0f..721b900dde9e 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -283,6 +283,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } logInfo("Map status for shuffleId "+shuffleId+" is now complete. Updater terminated. ---lirui") partialEpoch.synchronized { + partialEpoch.remove(shuffleId) partialEpoch.notifyAll() } } @@ -362,6 +363,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } } incrementEpoch() + partialForShuffle += shuffleId } else { throw new SparkException("unregisterMapOutput called for nonexistent shuffle ID") } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c725eaf0c925..0859bd1aecb1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -928,8 +928,9 @@ class DAGScheduler( if(removeStageBarrier && taskScheduler.isInstanceOf[TaskSchedulerImpl]){ // TODO: need a better way to check if there's free slots val backend = taskScheduler.asInstanceOf[TaskSchedulerImpl].backend - val waitingStageNum = waitingStages.size - if (backend.freeSlotAvail() && waitingStageNum > 0 && stage.shuffleDep.isDefined) { + val numPendingTask = pendingTasks.values.map(_.size).sum + val numWaitingStage = waitingStages.size + if (backend.freeSlotAvail(numPendingTask) && numWaitingStage > 0 && stage.shuffleDep.isDefined) { val preStartableStage = getPreStartableStage(stage) if (preStartableStage.isDefined) { val preStartedStage = preStartableStage.get diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index a30bcc811898..449f88174578 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -31,5 +31,5 @@ private[spark] trait SchedulerBackend { def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = throw new UnsupportedOperationException - def freeSlotAvail(): Boolean = false + def freeSlotAvail(numPendingTask: Int): Boolean = false } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index a31d1ade007c..d868a557cae3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -46,8 +46,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) - // Also track number of free cores - private val freeCoreCount = new AtomicInteger(0) val conf = scheduler.sc.conf private val timeout = AkkaUtils.askTimeout(conf) private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) @@ -85,7 +83,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A executorAddress(executorId) = sender.path.address addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) - freeCoreCount.addAndGet(cores) makeOffers() } @@ -94,7 +91,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A if (TaskState.isFinished(state)) { if (executorActor.contains(executorId)) { freeCores(executorId) += scheduler.CPUS_PER_TASK - freeCoreCount.addAndGet(scheduler.CPUS_PER_TASK) makeOffers(executorId) } else { // Ignoring the update since we don't know about the executor. @@ -163,7 +159,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } else { freeCores(task.executorId) -= scheduler.CPUS_PER_TASK - freeCoreCount.addAndGet(-scheduler.CPUS_PER_TASK) executorActor(task.executorId) ! LaunchTask(new SerializableBuffer(serializedTask)) } } @@ -181,7 +176,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A totalCores -= executorId freeCores -= executorId totalCoreCount.addAndGet(-numCores) - freeCoreCount.addAndGet(-numCores) scheduler.executorLost(executorId, SlaveLost(reason)) } } @@ -251,7 +245,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } } - override def freeSlotAvail(): Boolean = freeCoreCount.get() > 0 + override def freeSlotAvail(numPendingTask: Int): Boolean = { + numPendingTask * scheduler.CPUS_PER_TASK < totalCoreCount.get() + } } private[spark] object CoarseGrainedSchedulerBackend { From a418f03c7179f339f816fd57237cf63f5e087778 Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 23 Jun 2014 14:06:40 +0800 Subject: [PATCH 50/88] RemoveStageBarrier: when a task finishes, launch new tasks before populating the status update event --- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 6 +++++- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 8b905ad24535..609dd9d1ed19 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -376,7 +376,11 @@ private[spark] class TaskSetManager( } // Finally, if all else has failed, find a speculative task - findSpeculativeTask(execId, host, locality) + val speculativeTask = findSpeculativeTask(execId, host, locality) + if (speculativeTask.isDefined) { + logInfo("Pick a speculative task to run: " + speculativeTask.get._1 + " ---lirui") + } + speculativeTask } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index d868a557cae3..6dd2911876ce 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -87,7 +87,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } case StatusUpdate(executorId, taskId, state, data) => - scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { if (executorActor.contains(executorId)) { freeCores(executorId) += scheduler.CPUS_PER_TASK @@ -98,6 +97,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A logWarning(msg.format(taskId, state, sender, executorId)) } } + scheduler.statusUpdate(taskId, state, data.value) case ReviveOffers => makeOffers() From 3ced2bb6c58cd8ada19fa83ff6d7a9ea376a58ec Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 24 Jun 2014 18:23:52 +0800 Subject: [PATCH 51/88] RemoveStageBarrier: make offer after successful/failed task is properly handled --- .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 8 +++++--- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 5 +++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 17292b4c15b8..97290794f6a0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -314,6 +314,7 @@ private[spark] class TaskSchedulerImpl( tid: Long, taskResult: DirectTaskResult[_]) = synchronized { taskSetManager.handleSuccessfulTask(tid, taskResult) + backend.reviveOffers() } def handleFailedTask( @@ -322,11 +323,12 @@ private[spark] class TaskSchedulerImpl( taskState: TaskState, reason: TaskEndReason) = synchronized { taskSetManager.handleFailedTask(tid, taskState, reason) - if (!taskSetManager.isZombie && taskState != TaskState.KILLED) { + //if (!taskSetManager.isZombie && taskState != TaskState.KILLED) { // Need to revive offers again now that the task set manager state has been updated to // reflect failed tasks that need to be re-run. - backend.reviveOffers() - } + //backend.reviveOffers() + //} + backend.reviveOffers() } def error(message: String) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 6dd2911876ce..1dbc9ff1125c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -90,7 +90,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A if (TaskState.isFinished(state)) { if (executorActor.contains(executorId)) { freeCores(executorId) += scheduler.CPUS_PER_TASK - makeOffers(executorId) + //makeOffers(executorId) } else { // Ignoring the update since we don't know about the executor. val msg = "Ignored task status update (%d state %s) from unknown executor %s with ID %s" @@ -129,7 +129,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A // Make fake resource offers on all executors def makeOffers() { launchTasks(scheduler.resourceOffers( - executorHost.toArray.map {case (id, host) => new WorkerOffer(id, host, freeCores(id))})) + for ((id, host) <- executorHost.toArray if freeCores(id) > 0) yield new WorkerOffer(id, host, freeCores(id)))) + //executorHost.toArray.map {case (id, host) => new WorkerOffer(id, host, freeCores(id))})) } // Make fake resource offers on just one executor From 5b0031afc42fb988b5ab2e29041e908ad116ffcc Mon Sep 17 00:00:00 2001 From: lirui Date: Wed, 25 Jun 2014 13:48:50 +0800 Subject: [PATCH 52/88] RemoveStageBarrier: handle failed task in a synchronized manner --- .../spark/scheduler/TaskResultGetter.scala | 52 ++++++++++++------- .../spark/scheduler/TaskSchedulerImpl.scala | 1 - .../CoarseGrainedSchedulerBackend.scala | 8 ++- 3 files changed, 40 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 99d305b36a95..96c44f659a74 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -77,28 +77,44 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul }) } + // Let's handle failed task in a synchronized manner def enqueueFailedTask(taskSetManager: TaskSetManager, tid: Long, taskState: TaskState, serializedData: ByteBuffer) { var reason : TaskEndReason = UnknownReason - getTaskResultExecutor.execute(new Runnable { - override def run(): Unit = Utils.logUncaughtExceptions { - try { - if (serializedData != null && serializedData.limit() > 0) { - reason = serializer.get().deserialize[TaskEndReason]( - serializedData, Utils.getSparkClassLoader) - } - } catch { - case cnd: ClassNotFoundException => - // Log an error but keep going here -- the task failed, so not catastropic if we can't - // deserialize the reason. - val loader = Utils.getContextOrSparkClassLoader - logError( - "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) - case ex: Exception => {} - } - scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) + try { + if (serializedData != null && serializedData.limit() > 0) { + reason = serializer.get().deserialize[TaskEndReason]( + serializedData, Utils.getSparkClassLoader) } - }) + } catch { + case cnd: ClassNotFoundException => + // Log an error but keep going here -- the task failed, so not catastropic if we can't + // deserialize the reason. + val loader = Utils.getContextOrSparkClassLoader + logError( + "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) + case ex: Exception => {} + } + scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) +// getTaskResultExecutor.execute(new Runnable { +// override def run(): Unit = Utils.logUncaughtExceptions { +// try { +// if (serializedData != null && serializedData.limit() > 0) { +// reason = serializer.get().deserialize[TaskEndReason]( +// serializedData, Utils.getSparkClassLoader) +// } +// } catch { +// case cnd: ClassNotFoundException => +// // Log an error but keep going here -- the task failed, so not catastropic if we can't +// // deserialize the reason. +// val loader = Utils.getContextOrSparkClassLoader +// logError( +// "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) +// case ex: Exception => {} +// } +// scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) +// } +// }) } def stop() { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 97290794f6a0..c42a01e78be5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -314,7 +314,6 @@ private[spark] class TaskSchedulerImpl( tid: Long, taskResult: DirectTaskResult[_]) = synchronized { taskSetManager.handleSuccessfulTask(tid, taskResult) - backend.reviveOffers() } def handleFailedTask( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 1dbc9ff1125c..01dcfa313475 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -87,17 +87,21 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } case StatusUpdate(executorId, taskId, state, data) => + scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { if (executorActor.contains(executorId)) { freeCores(executorId) += scheduler.CPUS_PER_TASK - //makeOffers(executorId) + // If the task finishes successfully, we make offer right away + // Otherwise, we wait for upstream scheduler to revive offer + if (state == TaskState.FINISHED) { + makeOffers(executorId) + } } else { // Ignoring the update since we don't know about the executor. val msg = "Ignored task status update (%d state %s) from unknown executor %s with ID %s" logWarning(msg.format(taskId, state, sender, executorId)) } } - scheduler.statusUpdate(taskId, state, data.value) case ReviveOffers => makeOffers() From 3c52c6941d721ca1d989b9239a5470dbcfd51488 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 27 Jun 2014 11:31:07 +0800 Subject: [PATCH 53/88] RemoveStageBarrier: add temp test code to detect deadlock --- .../org/apache/spark/scheduler/DAGScheduler.scala | 2 +- .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 10 +++++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 0859bd1aecb1..08e684581b08 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1016,7 +1016,7 @@ class DAGScheduler( for ((shuffleId, stage) <- shuffleToMapStage) { stage.removeOutputsOnExecutor(execId) val locs = stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray - mapOutputTracker.registerMapOutputs(shuffleId, locs, changeEpoch = true) + mapOutputTracker.registerMapOutputs(shuffleId, locs, changeEpoch = true, isPartial = true) } if (shuffleToMapStage.isEmpty) { mapOutputTracker.incrementEpoch() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index c42a01e78be5..7e18d6ef806d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -282,7 +282,15 @@ private[spark] class TaskSchedulerImpl( activeTaskSets.get(taskSetId).foreach { taskSet => if (state == TaskState.FINISHED) { taskSet.removeRunningTask(tid) - taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) + // Temporarily added for test only + // This is the last task in the task set + // It reports success, but may fail retrieving the result + if (taskSet.tasksSuccessful + 1 == taskSet.numTasks) { + logInfo("Fail the last successful task for test. ---lirui") + taskResultGetter.enqueueFailedTask(taskSet, tid, state, null) + } else { + taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) + } } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { taskSet.removeRunningTask(tid) taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData) From 0473e3be426d8b711b6cc4b6166a7863ee1e5afe Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 27 Jun 2014 11:49:09 +0800 Subject: [PATCH 54/88] RemoveStageBarrier: maintain support for asynchronous handling failed task --- .../spark/scheduler/TaskResultGetter.scala | 52 ++++++++++--------- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- 2 files changed, 29 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 96c44f659a74..475bb4f7eeb1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -77,8 +77,35 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul }) } - // Let's handle failed task in a synchronized manner def enqueueFailedTask(taskSetManager: TaskSetManager, tid: Long, taskState: TaskState, + serializedData: ByteBuffer) { + var reason : TaskEndReason = UnknownReason + getTaskResultExecutor.execute(new Runnable { + override def run(): Unit = Utils.logUncaughtExceptions { + try { + if (serializedData != null && serializedData.limit() > 0) { + reason = serializer.get().deserialize[TaskEndReason]( + serializedData, Utils.getSparkClassLoader) + } + } catch { + case cnd: ClassNotFoundException => + // Log an error but keep going here -- the task failed, so not catastropic if we can't + // deserialize the reason. + val loader = Utils.getContextOrSparkClassLoader + logError( + "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) + case ex: Exception => {} + } + scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) + } + }) + } + + def stop() { + getTaskResultExecutor.shutdownNow() + } + + def enqueueFailedTaskSync(taskSetManager: TaskSetManager, tid: Long, taskState: TaskState, serializedData: ByteBuffer) { var reason : TaskEndReason = UnknownReason try { @@ -96,28 +123,5 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul case ex: Exception => {} } scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) -// getTaskResultExecutor.execute(new Runnable { -// override def run(): Unit = Utils.logUncaughtExceptions { -// try { -// if (serializedData != null && serializedData.limit() > 0) { -// reason = serializer.get().deserialize[TaskEndReason]( -// serializedData, Utils.getSparkClassLoader) -// } -// } catch { -// case cnd: ClassNotFoundException => -// // Log an error but keep going here -- the task failed, so not catastropic if we can't -// // deserialize the reason. -// val loader = Utils.getContextOrSparkClassLoader -// logError( -// "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) -// case ex: Exception => {} -// } -// scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) -// } -// }) - } - - def stop() { - getTaskResultExecutor.shutdownNow() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 7e18d6ef806d..b6a04e25bb19 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -293,7 +293,7 @@ private[spark] class TaskSchedulerImpl( } } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { taskSet.removeRunningTask(tid) - taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData) + taskResultGetter.enqueueFailedTaskSync(taskSet, tid, state, serializedData) } } case None => From d267c9b380ecf9ddf76746f4009f84fb64b9f737 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 27 Jun 2014 14:31:30 +0800 Subject: [PATCH 55/88] RemoveStageBarrier: fix previously found problem --- .../org/apache/spark/scheduler/TaskResultGetter.scala | 2 ++ .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 8 ++------ 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 475bb4f7eeb1..65d09a1fce49 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -59,6 +59,8 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul taskSetManager, tid, TaskState.FINISHED, TaskResultLost) return } + // Now we know for sure the task is successful + scheduler.backend.reviveOffers() val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( serializedTaskResult.get) sparkEnv.blockManager.master.removeBlock(blockId) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index b6a04e25bb19..e2816ccff094 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -287,7 +287,7 @@ private[spark] class TaskSchedulerImpl( // It reports success, but may fail retrieving the result if (taskSet.tasksSuccessful + 1 == taskSet.numTasks) { logInfo("Fail the last successful task for test. ---lirui") - taskResultGetter.enqueueFailedTask(taskSet, tid, state, null) + taskResultGetter.enqueueFailedTask(taskSet, tid, TaskState.FAILED, null) } else { taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 01dcfa313475..1dbc9ff1125c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -87,21 +87,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } case StatusUpdate(executorId, taskId, state, data) => - scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { if (executorActor.contains(executorId)) { freeCores(executorId) += scheduler.CPUS_PER_TASK - // If the task finishes successfully, we make offer right away - // Otherwise, we wait for upstream scheduler to revive offer - if (state == TaskState.FINISHED) { - makeOffers(executorId) - } + //makeOffers(executorId) } else { // Ignoring the update since we don't know about the executor. val msg = "Ignored task status update (%d state %s) from unknown executor %s with ID %s" logWarning(msg.format(taskId, state, sender, executorId)) } } + scheduler.statusUpdate(taskId, state, data.value) case ReviveOffers => makeOffers() From 118914ba53b62f3977307165291a679e596dacd0 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 27 Jun 2014 14:59:18 +0800 Subject: [PATCH 56/88] RemoveStageBarrier: fix test code --- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index e2816ccff094..fdff172b7f10 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -111,6 +111,8 @@ private[spark] class TaskSchedulerImpl( // This is a var so that we can reset it for testing purposes. private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) + private var failureGenerated = false + override def setDAGScheduler(dagScheduler: DAGScheduler) { this.dagScheduler = dagScheduler } @@ -285,7 +287,8 @@ private[spark] class TaskSchedulerImpl( // Temporarily added for test only // This is the last task in the task set // It reports success, but may fail retrieving the result - if (taskSet.tasksSuccessful + 1 == taskSet.numTasks) { + if (taskSet.tasksSuccessful + 1 == taskSet.numTasks && !failureGenerated) { + failureGenerated = true logInfo("Fail the last successful task for test. ---lirui") taskResultGetter.enqueueFailedTask(taskSet, tid, TaskState.FAILED, null) } else { From fe630245be717bca9c672524e5cf195ac08f93d3 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 27 Jun 2014 16:07:25 +0800 Subject: [PATCH 57/88] RemoveStageBarrier: remove temp code --- .../org/apache/spark/scheduler/TaskResultGetter.scala | 2 -- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 11 +---------- .../cluster/CoarseGrainedSchedulerBackend.scala | 4 +++- 3 files changed, 4 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 65d09a1fce49..475bb4f7eeb1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -59,8 +59,6 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul taskSetManager, tid, TaskState.FINISHED, TaskResultLost) return } - // Now we know for sure the task is successful - scheduler.backend.reviveOffers() val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( serializedTaskResult.get) sparkEnv.blockManager.master.removeBlock(blockId) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index fdff172b7f10..a0853d051dc8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -284,16 +284,7 @@ private[spark] class TaskSchedulerImpl( activeTaskSets.get(taskSetId).foreach { taskSet => if (state == TaskState.FINISHED) { taskSet.removeRunningTask(tid) - // Temporarily added for test only - // This is the last task in the task set - // It reports success, but may fail retrieving the result - if (taskSet.tasksSuccessful + 1 == taskSet.numTasks && !failureGenerated) { - failureGenerated = true - logInfo("Fail the last successful task for test. ---lirui") - taskResultGetter.enqueueFailedTask(taskSet, tid, TaskState.FAILED, null) - } else { - taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) - } + taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { taskSet.removeRunningTask(tid) taskResultGetter.enqueueFailedTaskSync(taskSet, tid, state, serializedData) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 1dbc9ff1125c..1e4c4a7a6460 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -90,7 +90,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A if (TaskState.isFinished(state)) { if (executorActor.contains(executorId)) { freeCores(executorId) += scheduler.CPUS_PER_TASK - //makeOffers(executorId) + if (state == TaskState.FINISHED) { + makeOffers(executorId) + } } else { // Ignoring the update since we don't know about the executor. val msg = "Ignored task status update (%d state %s) from unknown executor %s with ID %s" From a996c77e9b383b6e45c1d4403bfacb882a13bfbd Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 27 Jun 2014 16:40:11 +0800 Subject: [PATCH 58/88] RemoveStageBarrier: add temp test code --- .../org/apache/spark/scheduler/DAGScheduler.scala | 11 ++++++++--- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 1 + .../org/apache/spark/scheduler/TaskSetManager.scala | 1 + 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 08e684581b08..0a087283936a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -124,8 +124,11 @@ class DAGScheduler( // Whether to enable remove stage barrier val removeStageBarrier = env.conf.getBoolean("spark.scheduler.removeStageBarrier", false) - //track the pre-started stages depending on a stage (the key) - val dependantStagePreStarted = new mutable.HashMap[Stage, ArrayBuffer[Stage]]() + // Track the pre-started stages depending on a stage (the key) + private val dependantStagePreStarted = new mutable.HashMap[Stage, ArrayBuffer[Stage]]() + + // Temporarily added for test + private var failureGenerated = false private def initializeEventProcessActor() { // blocking the thread until supervisor is started, which ensures eventProcessActor is @@ -883,7 +886,9 @@ class DAGScheduler( changeEpoch = true) } clearCacheLocs() - if (stage.outputLocs.exists(_ == Nil)) { + if (stage.outputLocs.exists(_ == Nil) || !failureGenerated) { + stage.outputLocs(0) = Nil + failureGenerated = true // Some tasks had failed; let's resubmit this stage // TODO: Lower-level scheduler should also deal with this logInfo("Resubmitting " + stage + " (" + stage.name + diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index a0853d051dc8..e79d834fe9ce 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -111,6 +111,7 @@ private[spark] class TaskSchedulerImpl( // This is a var so that we can reset it for testing purposes. private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) + // Temporarily added for test private var failureGenerated = false override def setDAGScheduler(dagScheduler: DAGScheduler) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 609dd9d1ed19..3d21f14523e4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -575,6 +575,7 @@ private[spark] class TaskSetManager( case TaskResultLost => failureReason = "Lost result for TID %s on host %s".format(tid, info.host) logWarning(failureReason) + // TODO: may cause some sort of "deadlock" if we lost the reuslt of a shuffle map task case _ => failureReason = "TID %s on host %s failed for unknown reason".format(tid, info.host) From cef517b746f0c65d485bc7210f093023eb3f6cd7 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 27 Jun 2014 16:55:23 +0800 Subject: [PATCH 59/88] RemoveStageBarrier: fix shuffle map stage fail over --- .../org/apache/spark/scheduler/DAGScheduler.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 0a087283936a..0110343c9fd9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -867,7 +867,6 @@ class DAGScheduler( } } if (runningStages.contains(stage) && pendingTasks(stage).isEmpty) { - dependantStagePreStarted -= stage markStageAsFinished(stage) logInfo("looking for newly runnable stages") logInfo("running: " + runningStages) @@ -896,16 +895,16 @@ class DAGScheduler( stage.outputLocs.zipWithIndex.filter(_._1 == Nil).map(_._2).mkString(", ")) // Pre-started dependant stages should fail if (dependantStagePreStarted.contains(stage)) { + if (failedStages.isEmpty && eventProcessActor != null) { + import env.actorSystem.dispatcher + env.actorSystem.scheduler.scheduleOnce( + RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) + } for (preStartedStage <- dependantStagePreStarted.get(stage).get) { runningStages -= preStartedStage // TODO: Cancel running tasks in the stage logInfo("Marking " + preStartedStage + " (" + preStartedStage.name + ") for resubmision due to parent stage resubmission") - if (failedStages.isEmpty && eventProcessActor != null) { - import env.actorSystem.dispatcher - env.actorSystem.scheduler.scheduleOnce( - RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) - } failedStages += preStartedStage } } @@ -928,6 +927,7 @@ class DAGScheduler( submitMissingTasks(stage, jobId) } } + dependantStagePreStarted -= stage } else { // ShuffleMap stage not finished yet. Maybe we can remove the stage barrier here. if(removeStageBarrier && taskScheduler.isInstanceOf[TaskSchedulerImpl]){ From 7d9a4a4c9802783cee8bfde1c9ac68f8f71ff1b1 Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 30 Jun 2014 13:16:09 +0800 Subject: [PATCH 60/88] RemoveStageBarrier: kill running tasks when resubmit failed stages --- .../apache/spark/scheduler/DAGScheduler.scala | 28 +++++++++++++------ .../spark/scheduler/TaskScheduler.scala | 4 +++ .../spark/scheduler/TaskSchedulerImpl.scala | 11 ++++++++ .../spark/scheduler/TaskSetManager.scala | 7 +++++ 4 files changed, 41 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 0110343c9fd9..50b7032c0b3a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -893,21 +893,17 @@ class DAGScheduler( logInfo("Resubmitting " + stage + " (" + stage.name + ") because some of its tasks had failed: " + stage.outputLocs.zipWithIndex.filter(_._1 == Nil).map(_._2).mkString(", ")) - // Pre-started dependant stages should fail + val stages = new ArrayBuffer[Stage]() + // Pre-started dependant stages should fail as well if (dependantStagePreStarted.contains(stage)) { - if (failedStages.isEmpty && eventProcessActor != null) { - import env.actorSystem.dispatcher - env.actorSystem.scheduler.scheduleOnce( - RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) - } for (preStartedStage <- dependantStagePreStarted.get(stage).get) { - runningStages -= preStartedStage - // TODO: Cancel running tasks in the stage logInfo("Marking " + preStartedStage + " (" + preStartedStage.name + ") for resubmision due to parent stage resubmission") - failedStages += preStartedStage + runningStages -= preStartedStage + stages += preStartedStage } } + failStages(stages.toArray) submitStage(stage) } else { val newlyRunnable = new ArrayBuffer[Stage] @@ -1227,6 +1223,20 @@ class DAGScheduler( } false } + + // Mark some stages as failed and resubmit them + private def failStages(stages: Array[Stage]) { + // Let's first kill all the running tasks in the failed stage + for (failedStage <- stages) { + taskScheduler.killTasks(failedStage.id, true) + } + if (failedStages.isEmpty && eventProcessActor != null) { + import env.actorSystem.dispatcher + env.actorSystem.scheduler.scheduleOnce( + RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) + } + failedStages ++= stages + } } private[scheduler] class DAGSchedulerActorSupervisor(dagScheduler: DAGScheduler) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 819c35257b5a..b8a2da4ef0e4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -54,4 +54,8 @@ private[spark] trait TaskScheduler { // Get the default level of parallelism to use in the cluster, as a hint for sizing jobs. def defaultParallelism(): Int + + // Kill the running tasks in a stage, without aborting the job + // This is used for resubmitting a failed stage + def killTasks(stageId: Int, interruptThread: Boolean): Unit = {} } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index e79d834fe9ce..895195dd7e8a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -435,6 +435,17 @@ private[spark] class TaskSchedulerImpl( // By default, rack is unknown def getRackForHost(value: String): Option[String] = None + + override def killTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { + logInfo("Killing tasks in stage " + stageId) + activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) => + tsm.runningTasksSet.foreach { tid => + val execId = taskIdToExecutorId(tid) + backend.killTask(tid, execId, interruptThread) + } + tsm.kill() + } + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 3d21f14523e4..abf795e0f08b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -750,4 +750,11 @@ private[spark] class TaskSetManager( // Test if this stage is in pre-start state def isPreStart() = sched.dagScheduler.isPreStartStage(stageId) + + // Kill this task set manager + def kill() { + isZombie = true + runningTasksSet.clear() + maybeFinishTaskSet() + } } From 5697b98e6027b41e6648c0b06e080885c5eb2446 Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 30 Jun 2014 14:07:36 +0800 Subject: [PATCH 61/88] RemoveStageBarrier: refine temp test code --- .../scala/org/apache/spark/MapOutputTracker.scala | 2 +- .../org/apache/spark/scheduler/DAGScheduler.scala | 14 +++++++++----- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 721b900dde9e..6634d952a29c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -254,7 +254,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging true } - // Compute the completeness of a shuffle + // Compute the completeness of map statuses for a shuffle def completenessForShuffle(shuffleId: Int): Int = { if (mapStatuses.get(shuffleId).isDefined) { return mapStatuses.get(shuffleId).get.count(_ != null) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 50b7032c0b3a..f56de4ed07af 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -859,6 +859,12 @@ class DAGScheduler( logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId) } else { stage.addOutputLoc(smt.partitionId, status) + // Temporarily added for test + if (pendingTasks(stage).isEmpty && !failureGenerated) { + stage.removeOutputLoc(smt.partitionId, null) + failureGenerated = true + logInfo("Remove some map status to created a failuer. ---lirui") + } // Need to register map outputs progressively if remove stage barrier is enabled if (removeStageBarrier && dependantStagePreStarted.contains(stage) && stage.shuffleDep.isDefined) { mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, @@ -882,19 +888,17 @@ class DAGScheduler( mapOutputTracker.registerMapOutputs( stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, - changeEpoch = true) + changeEpoch = true,isPartial = true) } clearCacheLocs() - if (stage.outputLocs.exists(_ == Nil) || !failureGenerated) { - stage.outputLocs(0) = Nil - failureGenerated = true + if (stage.outputLocs.exists(_ == Nil)) { // Some tasks had failed; let's resubmit this stage // TODO: Lower-level scheduler should also deal with this logInfo("Resubmitting " + stage + " (" + stage.name + ") because some of its tasks had failed: " + stage.outputLocs.zipWithIndex.filter(_._1 == Nil).map(_._2).mkString(", ")) + // Pre-started dependant stages should fail val stages = new ArrayBuffer[Stage]() - // Pre-started dependant stages should fail as well if (dependantStagePreStarted.contains(stage)) { for (preStartedStage <- dependantStagePreStarted.get(stage).get) { logInfo("Marking " + preStartedStage + " (" + preStartedStage.name + From 4f80b1d692ef4999a1e6fd380d5522807d84eb98 Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 30 Jun 2014 14:42:01 +0800 Subject: [PATCH 62/88] RemoveStageBarrier: fix test code --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index f56de4ed07af..87731649b885 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -861,7 +861,7 @@ class DAGScheduler( stage.addOutputLoc(smt.partitionId, status) // Temporarily added for test if (pendingTasks(stage).isEmpty && !failureGenerated) { - stage.removeOutputLoc(smt.partitionId, null) + stage.removeOutputLoc(smt.partitionId, status.location) failureGenerated = true logInfo("Remove some map status to created a failuer. ---lirui") } From 39ddb9d26fc9804e612c146f89da28a7dd4a6d20 Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 30 Jun 2014 16:33:43 +0800 Subject: [PATCH 63/88] RemoveStageBarrier: remove temp code --- .../org/apache/spark/scheduler/DAGScheduler.scala | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 87731649b885..c6cbcdef0062 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -123,7 +123,7 @@ class DAGScheduler( private[scheduler] var eventProcessActor: ActorRef = _ // Whether to enable remove stage barrier - val removeStageBarrier = env.conf.getBoolean("spark.scheduler.removeStageBarrier", false) + private val removeStageBarrier = env.conf.getBoolean("spark.scheduler.removeStageBarrier", false) // Track the pre-started stages depending on a stage (the key) private val dependantStagePreStarted = new mutable.HashMap[Stage, ArrayBuffer[Stage]]() @@ -859,12 +859,6 @@ class DAGScheduler( logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId) } else { stage.addOutputLoc(smt.partitionId, status) - // Temporarily added for test - if (pendingTasks(stage).isEmpty && !failureGenerated) { - stage.removeOutputLoc(smt.partitionId, status.location) - failureGenerated = true - logInfo("Remove some map status to created a failuer. ---lirui") - } // Need to register map outputs progressively if remove stage barrier is enabled if (removeStageBarrier && dependantStagePreStarted.contains(stage) && stage.shuffleDep.isDefined) { mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, @@ -888,7 +882,7 @@ class DAGScheduler( mapOutputTracker.registerMapOutputs( stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, - changeEpoch = true,isPartial = true) + changeEpoch = true) } clearCacheLocs() if (stage.outputLocs.exists(_ == Nil)) { @@ -974,6 +968,7 @@ class DAGScheduler( mapStage.removeOutputLoc(mapId, bmAddress) mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) } + runningStages -= mapStage logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name + "); marking it for resubmission") if (failedStages.isEmpty && eventProcessActor != null) { From 8cb8e4c2345a504e174c94f64b39dea98145dd8e Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 30 Jun 2014 16:57:43 +0800 Subject: [PATCH 64/88] RemoveStageBarrier: kill running tasks before resubmit failed stages --- .../apache/spark/scheduler/DAGScheduler.scala | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c6cbcdef0062..cf65c6a310ff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -956,10 +956,11 @@ class DAGScheduler( pendingTasks(stage) += task case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => + val stages = new ArrayBuffer[Stage]() // Mark the stage that the reducer was in as unrunnable val failedStage = stageIdToStage(task.stageId) runningStages -= failedStage - // TODO: Cancel running tasks in the stage + stages += failedStage logInfo("Marking " + failedStage + " (" + failedStage.name + ") for resubmision due to a fetch failure") // Mark the map whose fetch failed as broken in the map stage @@ -969,18 +970,10 @@ class DAGScheduler( mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) } runningStages -= mapStage + stages += mapStage logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name + "); marking it for resubmission") - if (failedStages.isEmpty && eventProcessActor != null) { - // Don't schedule an event to resubmit failed stages if failed isn't empty, because - // in that case the event will already have been scheduled. eventProcessActor may be - // null during unit tests. - import env.actorSystem.dispatcher - env.actorSystem.scheduler.scheduleOnce( - RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) - } - failedStages += failedStage - failedStages += mapStage + failStages(stages.toArray) // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { handleExecutorLost(bmAddress.executorId, Some(task.epoch)) @@ -1230,6 +1223,9 @@ class DAGScheduler( taskScheduler.killTasks(failedStage.id, true) } if (failedStages.isEmpty && eventProcessActor != null) { + // Don't schedule an event to resubmit failed stages if failed isn't empty, because + // in that case the event will already have been scheduled. eventProcessActor may be + // null during unit tests. import env.actorSystem.dispatcher env.actorSystem.scheduler.scheduleOnce( RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) From bc69fed15e653745426a7ec17c60aa589586d9d3 Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 1 Jul 2014 09:33:15 +0800 Subject: [PATCH 65/88] RemoveStageBarrier: add temp test code --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 5 ++++- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 3 --- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index cf65c6a310ff..45db3643ad12 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -937,7 +937,10 @@ class DAGScheduler( // Register map output finished so far mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, - changeEpoch = false, isPartial = true) + changeEpoch = false, isPartial = failureGenerated) + if (!failureGenerated) { + failureGenerated = true + } waitingStages -= preStartedStage runningStages += preStartedStage // Inform parent stages that the dependant stage has been pre-started diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 895195dd7e8a..1e9fb37c3bd0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -111,9 +111,6 @@ private[spark] class TaskSchedulerImpl( // This is a var so that we can reset it for testing purposes. private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) - // Temporarily added for test - private var failureGenerated = false - override def setDAGScheduler(dagScheduler: DAGScheduler) { this.dagScheduler = dagScheduler } From 1e1907d018c77db401bd79e9bcdaa1bc7f9b358e Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 1 Jul 2014 10:02:41 +0800 Subject: [PATCH 66/88] RemoveStageBarrier: fix test code --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 1 - .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 ++- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 6634d952a29c..e5599621ec8c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -91,7 +91,6 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * On the workers, it simply serves as a cache, in which a miss triggers a fetch from the * master's corresponding HashMap. */ - //TODO: we should also record if the output for a shuffle is partial protected val mapStatuses: Map[Int, Array[MapStatus]] // Track if we have partial map outputs for a shuffle diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 45db3643ad12..395db01ee6c9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -123,7 +123,7 @@ class DAGScheduler( private[scheduler] var eventProcessActor: ActorRef = _ // Whether to enable remove stage barrier - private val removeStageBarrier = env.conf.getBoolean("spark.scheduler.removeStageBarrier", false) + private var removeStageBarrier = env.conf.getBoolean("spark.scheduler.removeStageBarrier", false) // Track the pre-started stages depending on a stage (the key) private val dependantStagePreStarted = new mutable.HashMap[Stage, ArrayBuffer[Stage]]() @@ -940,6 +940,7 @@ class DAGScheduler( changeEpoch = false, isPartial = failureGenerated) if (!failureGenerated) { failureGenerated = true + removeStageBarrier = false } waitingStages -= preStartedStage runningStages += preStartedStage From 930136dd833d2c6a5891ce2fc8712e49f112b3db Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 1 Jul 2014 11:28:25 +0800 Subject: [PATCH 67/88] RemoveStageBarrier: handle fetch failed task only if it comes from a running stage --- .../apache/spark/scheduler/DAGScheduler.scala | 39 ++++++++++--------- 1 file changed, 20 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 395db01ee6c9..81d8e6f50356 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -960,27 +960,28 @@ class DAGScheduler( pendingTasks(stage) += task case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => - val stages = new ArrayBuffer[Stage]() // Mark the stage that the reducer was in as unrunnable val failedStage = stageIdToStage(task.stageId) - runningStages -= failedStage - stages += failedStage - logInfo("Marking " + failedStage + " (" + failedStage.name + - ") for resubmision due to a fetch failure") - // Mark the map whose fetch failed as broken in the map stage - val mapStage = shuffleToMapStage(shuffleId) - if (mapId != -1) { - mapStage.removeOutputLoc(mapId, bmAddress) - mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) - } - runningStages -= mapStage - stages += mapStage - logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name + - "); marking it for resubmission") - failStages(stages.toArray) - // TODO: mark the executor as failed only if there were lots of fetch failures on it - if (bmAddress != null) { - handleExecutorLost(bmAddress.executorId, Some(task.epoch)) + if(runningStages.remove(failedStage)){ + val stages = new ArrayBuffer[Stage]() + stages += failedStage + logInfo("Marking " + failedStage + " (" + failedStage.name + + ") for resubmision due to a fetch failure") + // Mark the map whose fetch failed as broken in the map stage + val mapStage = shuffleToMapStage(shuffleId) + if (mapId != -1) { + mapStage.removeOutputLoc(mapId, bmAddress) + mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) + } + runningStages -= mapStage + stages += mapStage + logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name + + "); marking it for resubmission") + failStages(stages.toArray) + // TODO: mark the executor as failed only if there were lots of fetch failures on it + if (bmAddress != null) { + handleExecutorLost(bmAddress.executorId, Some(task.epoch)) + } } case ExceptionFailure(className, description, stackTrace, metrics) => From b49cbdbe6d7144bbd2ac9da8cc498a473152dc7c Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 1 Jul 2014 16:35:24 +0800 Subject: [PATCH 68/88] RemoveStageBarrier: kill tasks without interrupting the thread --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- .../scala/org/apache/spark/storage/ShuffleBlockManager.scala | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 81d8e6f50356..a6a79ab10c34 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1225,7 +1225,7 @@ class DAGScheduler( private def failStages(stages: Array[Stage]) { // Let's first kill all the running tasks in the failed stage for (failedStage <- stages) { - taskScheduler.killTasks(failedStage.id, true) + taskScheduler.killTasks(failedStage.id, false) } if (failedStages.isEmpty && eventProcessActor != null) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 85bc6f139d99..479b8dfb457c 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -253,7 +253,6 @@ object ShuffleBlockManager { /** Returns the FileSegment associated with the given map task, or None if no entry exists. */ def getFileSegmentFor(mapId: Int, reducerId: Int): Option[FileSegment] = { - //TODO: potential issue here when map and reduce stages overlap val file = files(reducerId) val blockOffsets = blockOffsetsByReducer(reducerId) val index = mapIdToIndex.getOrElse(mapId, -1) From 6bcca9b088f415c9a7a77682688c8519cfa6f538 Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 1 Jul 2014 17:04:10 +0800 Subject: [PATCH 69/88] RemoveStageBarrier: remove test code --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index a6a79ab10c34..bb63b53d0a6b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -127,9 +127,6 @@ class DAGScheduler( // Track the pre-started stages depending on a stage (the key) private val dependantStagePreStarted = new mutable.HashMap[Stage, ArrayBuffer[Stage]]() - // Temporarily added for test - private var failureGenerated = false - private def initializeEventProcessActor() { // blocking the thread until supervisor is started, which ensures eventProcessActor is // not null before any job is submitted @@ -937,11 +934,7 @@ class DAGScheduler( // Register map output finished so far mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, - changeEpoch = false, isPartial = failureGenerated) - if (!failureGenerated) { - failureGenerated = true - removeStageBarrier = false - } + changeEpoch = false, isPartial = true) waitingStages -= preStartedStage runningStages += preStartedStage // Inform parent stages that the dependant stage has been pre-started From 8fded0e1ef57fbdfaba602de6f29e8280b95b335 Mon Sep 17 00:00:00 2001 From: lirui Date: Wed, 2 Jul 2014 14:26:48 +0800 Subject: [PATCH 70/88] RemoveStageBarrier: use AKKA actor to access DAGScheduler's data structures --- .../org/apache/spark/MapOutputTracker.scala | 3 ++- .../apache/spark/scheduler/DAGScheduler.scala | 19 ++++++++++++++++--- .../spark/scheduler/DAGSchedulerEvent.scala | 2 ++ 3 files changed, 20 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index e5599621ec8c..2fcef8305fcd 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -316,7 +316,8 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) * so that statuses are dropped only by explicit de-registering or by TTL-based cleaning (if set). * Other than these two scenarios, nothing should be dropped from this HashMap. */ - protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]() + protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]() with + mutable.SynchronizedMap[Int, Array[MapStatus]] private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]() // For cleaning up TimeStampedHashMaps diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index bb63b53d0a6b..3e3faf16afbb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -38,8 +38,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId} -import org.apache.spark.util.Utils -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.{AkkaUtils, Utils} import scala.collection.mutable /** @@ -1202,7 +1201,7 @@ class DAGScheduler( } // Check if the given stageId is a pre-started stage - def isPreStartStage(stageId: Int): Boolean = { + private[scheduler] def handleCheckIfPreStarted(stageId: Int): Boolean = { if (stageIdToStage.contains(stageId)) { val stage = stageIdToStage(stageId) for (preStartedStages <- dependantStagePreStarted.values) { @@ -1214,6 +1213,17 @@ class DAGScheduler( false } + def isPreStartStage(stageId: Int): Boolean = { + try { + val timeout = AkkaUtils.askTimeout(sc.conf) + val future = eventProcessActor.ask(CheckIfPreStarted(stageId))(timeout) + Await.result(future, timeout).asInstanceOf[Boolean] + } catch { + case e: Exception => + throw new SparkException("Time out asking event processor.", e) + } + } + // Mark some stages as failed and resubmit them private def failStages(stages: Array[Stage]) { // Let's first kill all the running tasks in the failed stage @@ -1300,6 +1310,9 @@ private[scheduler] class DAGSchedulerEventProcessActor(dagScheduler: DAGSchedule case ResubmitFailedStages => dagScheduler.resubmitFailedStages() + + case CheckIfPreStarted(stageId) => + dagScheduler.handleCheckIfPreStarted(stageId) } override def postStop() { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 23f57441b4b1..0392839af872 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -76,3 +76,5 @@ private[scheduler] case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent private[scheduler] case object ResubmitFailedStages extends DAGSchedulerEvent + +private[scheduler] case class CheckIfPreStarted(stageId: Int) extends DAGSchedulerEvent From aa2e0f22600874bfad2aebf751591771d67f3fe6 Mon Sep 17 00:00:00 2001 From: lirui Date: Wed, 2 Jul 2014 14:32:08 +0800 Subject: [PATCH 71/88] RemoveStageBarrier: fix bug --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 3e3faf16afbb..0d657db914d5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1312,7 +1312,7 @@ private[scheduler] class DAGSchedulerEventProcessActor(dagScheduler: DAGSchedule dagScheduler.resubmitFailedStages() case CheckIfPreStarted(stageId) => - dagScheduler.handleCheckIfPreStarted(stageId) + sender ! dagScheduler.handleCheckIfPreStarted(stageId) } override def postStop() { From 0bbdb5dc579e9687c53e5d6e844ed8cd00e5e5ee Mon Sep 17 00:00:00 2001 From: lirui Date: Thu, 3 Jul 2014 11:13:13 +0800 Subject: [PATCH 72/88] RemoveStageBarrier: compute sorted task sets without holding a lock on TaskSchedulerImpl --- .../spark/scheduler/TaskSchedulerImpl.scala | 95 ++++++++++--------- .../spark/scheduler/TaskSetManager.scala | 4 +- 2 files changed, 50 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 1e9fb37c3bd0..0382174e0afb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -206,59 +206,62 @@ private[spark] class TaskSchedulerImpl( * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so * that tasks are balanced across the cluster. */ - def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized { - SparkEnv.set(sc.env) - - // Mark each slave as alive and remember its hostname - for (o <- offers) { - executorIdToHost(o.executorId) = o.host - if (!executorsByHost.contains(o.host)) { - executorsByHost(o.host) = new HashSet[String]() - executorAdded(o.executorId, o.host) + def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = { + val sortedTaskSets = rootPool.getSortedTaskSetQueue + this.synchronized { + SparkEnv.set(sc.env) + + // Mark each slave as alive and remember its hostname + for (o <- offers) { + executorIdToHost(o.executorId) = o.host + if (!executorsByHost.contains(o.host)) { + executorsByHost(o.host) = new HashSet[String]() + executorAdded(o.executorId, o.host) + } } - } - // Randomly shuffle offers to avoid always placing tasks on the same set of workers. - val shuffledOffers = Random.shuffle(offers) - // Build a list of tasks to assign to each worker. - val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores)) - val availableCpus = shuffledOffers.map(o => o.cores).toArray - val sortedTaskSets = rootPool.getSortedTaskSetQueue - for (taskSet <- sortedTaskSets) { - logDebug("parentName: %s, name: %s, runningTasks: %s".format( - taskSet.parent.name, taskSet.name, taskSet.runningTasks)) - } + // Randomly shuffle offers to avoid always placing tasks on the same set of workers. + val shuffledOffers = Random.shuffle(offers) + // Build a list of tasks to assign to each worker. + val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores)) + val availableCpus = shuffledOffers.map(o => o.cores).toArray - // Take each TaskSet in our scheduling order, and then offer it each node in increasing order - // of locality levels so that it gets a chance to launch local tasks on all of them. - var launchedTask = false - for (taskSet <- sortedTaskSets; maxLocality <- TaskLocality.values) { - do { - launchedTask = false - for (i <- 0 until shuffledOffers.size) { - val execId = shuffledOffers(i).executorId - val host = shuffledOffers(i).host - if (availableCpus(i) >= CPUS_PER_TASK) { - for (task <- taskSet.resourceOffer(execId, host, maxLocality)) { - tasks(i) += task - val tid = task.taskId - taskIdToTaskSetId(tid) = taskSet.taskSet.id - taskIdToExecutorId(tid) = execId - activeExecutorIds += execId - executorsByHost(host) += execId - availableCpus(i) -= CPUS_PER_TASK - assert (availableCpus(i) >= 0) - launchedTask = true + for (taskSet <- sortedTaskSets) { + logDebug("parentName: %s, name: %s, runningTasks: %s".format( + taskSet.parent.name, taskSet.name, taskSet.runningTasks)) + } + + // Take each TaskSet in our scheduling order, and then offer it each node in increasing order + // of locality levels so that it gets a chance to launch local tasks on all of them. + var launchedTask = false + for (taskSet <- sortedTaskSets; maxLocality <- TaskLocality.values) { + do { + launchedTask = false + for (i <- 0 until shuffledOffers.size) { + val execId = shuffledOffers(i).executorId + val host = shuffledOffers(i).host + if (availableCpus(i) >= CPUS_PER_TASK) { + for (task <- taskSet.resourceOffer(execId, host, maxLocality)) { + tasks(i) += task + val tid = task.taskId + taskIdToTaskSetId(tid) = taskSet.taskSet.id + taskIdToExecutorId(tid) = execId + activeExecutorIds += execId + executorsByHost(host) += execId + availableCpus(i) -= CPUS_PER_TASK + assert(availableCpus(i) >= 0) + launchedTask = true + } } } - } - } while (launchedTask) - } + } while (launchedTask) + } - if (tasks.size > 0) { - hasLaunchedTask = true + if (tasks.size > 0) { + hasLaunchedTask = true + } + return tasks } - return tasks } def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index abf795e0f08b..156dba9a95c9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -632,9 +632,7 @@ private[spark] class TaskSetManager( override def removeSchedulable(schedulable: Schedulable) {} override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { - var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager]() - sortedTaskSetQueue += this - sortedTaskSetQueue + ArrayBuffer[TaskSetManager](this) } /** Called by TaskScheduler when an executor is lost so we can re-enqueue our tasks */ From d941899cdb017a593a4163bc29da35eaff968522 Mon Sep 17 00:00:00 2001 From: lirui Date: Thu, 3 Jul 2014 14:59:27 +0800 Subject: [PATCH 73/88] RemoveStageBarrier: make the updater sleep a little longer if maps are finishing slowly --- .../scala/org/apache/spark/MapOutputTracker.scala | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 2fcef8305fcd..8e299bd049df 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -255,10 +255,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging // Compute the completeness of map statuses for a shuffle def completenessForShuffle(shuffleId: Int): Int = { - if (mapStatuses.get(shuffleId).isDefined) { - return mapStatuses.get(shuffleId).get.count(_ != null) - } - 0 + mapStatuses.getOrElse(shuffleId, new Array[MapStatus](0)).count(_ != null) } // A proxy to update partial map statuses periodically @@ -270,14 +267,20 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } } logInfo("Updater started for shuffleId "+shuffleId+". ---lirui") + val minInterval = 1000 + val maxInterval = 3000 + var sleepInterval = minInterval while (partialForShuffle.contains(shuffleId)) { - Thread.sleep(1000) + Thread.sleep(sleepInterval) if (clearOutdatedMapStatuses(shuffleId)) { getMapStatusesForShuffle(shuffleId, -1) partialEpoch.synchronized { partialEpoch.put(shuffleId, partialEpoch.getOrElse(shuffleId, 0) + 1) partialEpoch.notifyAll() } + sleepInterval = math.max(minInterval, sleepInterval - 500) + } else { + sleepInterval = math.min(maxInterval, sleepInterval + 200) } } logInfo("Map status for shuffleId "+shuffleId+" is now complete. Updater terminated. ---lirui") From 12b809356bafec930e1785227c79ebc13dfe92ca Mon Sep 17 00:00:00 2001 From: lirui Date: Thu, 3 Jul 2014 16:50:25 +0800 Subject: [PATCH 74/88] RemoveStageBarrier: fix bug --- .../org/apache/spark/MapOutputTracker.scala | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 8e299bd049df..54fbdc4597b6 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -135,9 +135,11 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * a given shuffle. */ def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = { - val statuses = getMapStatusesForShuffle(shuffleId, reduceId) + val mapStatuses = getMapStatusesForShuffle(shuffleId, reduceId) + val statuses = mapStatuses._1 + val partial = mapStatuses._2 statuses.synchronized { - MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses, isPartial = partialForShuffle.contains(shuffleId)) + MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses, isPartial = partial) } } @@ -172,11 +174,12 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging def stop() { } // Get map statuses for a shuffle - private def getMapStatusesForShuffle(shuffleId: Int, reduceId: Int): Array[MapStatus]={ + private def getMapStatusesForShuffle(shuffleId: Int, reduceId: Int): (Array[MapStatus], Boolean) ={ val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") var fetchedStatuses: Array[MapStatus] = null + var fetchedPartial = partialForShuffle.contains(shuffleId) fetching.synchronized { if (fetching.contains(shuffleId)) { // Someone else is fetching it; wait for them to be done @@ -207,11 +210,11 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging askTracker(GetMapOutputStatuses(shuffleId)).asInstanceOf[Array[Byte]] val fetchedResults = MapOutputTracker.deserializeMapStatuses(fetchedBytes) fetchedStatuses = fetchedResults._1 + fetchedPartial = fetchedResults._2 if (fetchedResults._2) { logInfo("Got partial map outputs from master for shuffleId " + shuffleId + ". ---lirui") - if(!partialForShuffle.contains(shuffleId)){ - partialForShuffle += shuffleId - new Thread(new MapStatusUpdater(shuffleId)).start() + if (partialForShuffle.add(shuffleId)) { + new Thread(new MapStatusUpdater(shuffleId), "MapStatusUpdater-for-shuffle" + shuffleId).start() } } else { logInfo("Got complete map outputs from master for shuffleId " + shuffleId + ". ---lirui") @@ -227,13 +230,13 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } } if (fetchedStatuses != null) { - fetchedStatuses + (fetchedStatuses, fetchedPartial) } else { throw new FetchFailedException(null, shuffleId, -1, reduceId, new Exception("Missing all output locations for shuffle " + shuffleId)) } } else { - statuses + (statuses, partialForShuffle.contains(shuffleId)) } } From c74a876e9d9156ddd6ab1cbac3cbe2c1b20c1c3f Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 4 Jul 2014 13:29:12 +0800 Subject: [PATCH 75/88] Revert "RemoveStageBarrier: fix bug" This reverts commit 12b809356bafec930e1785227c79ebc13dfe92ca. --- .../org/apache/spark/MapOutputTracker.scala | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 54fbdc4597b6..8e299bd049df 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -135,11 +135,9 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * a given shuffle. */ def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = { - val mapStatuses = getMapStatusesForShuffle(shuffleId, reduceId) - val statuses = mapStatuses._1 - val partial = mapStatuses._2 + val statuses = getMapStatusesForShuffle(shuffleId, reduceId) statuses.synchronized { - MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses, isPartial = partial) + MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses, isPartial = partialForShuffle.contains(shuffleId)) } } @@ -174,12 +172,11 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging def stop() { } // Get map statuses for a shuffle - private def getMapStatusesForShuffle(shuffleId: Int, reduceId: Int): (Array[MapStatus], Boolean) ={ + private def getMapStatusesForShuffle(shuffleId: Int, reduceId: Int): Array[MapStatus]={ val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") var fetchedStatuses: Array[MapStatus] = null - var fetchedPartial = partialForShuffle.contains(shuffleId) fetching.synchronized { if (fetching.contains(shuffleId)) { // Someone else is fetching it; wait for them to be done @@ -210,11 +207,11 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging askTracker(GetMapOutputStatuses(shuffleId)).asInstanceOf[Array[Byte]] val fetchedResults = MapOutputTracker.deserializeMapStatuses(fetchedBytes) fetchedStatuses = fetchedResults._1 - fetchedPartial = fetchedResults._2 if (fetchedResults._2) { logInfo("Got partial map outputs from master for shuffleId " + shuffleId + ". ---lirui") - if (partialForShuffle.add(shuffleId)) { - new Thread(new MapStatusUpdater(shuffleId), "MapStatusUpdater-for-shuffle" + shuffleId).start() + if(!partialForShuffle.contains(shuffleId)){ + partialForShuffle += shuffleId + new Thread(new MapStatusUpdater(shuffleId)).start() } } else { logInfo("Got complete map outputs from master for shuffleId " + shuffleId + ". ---lirui") @@ -230,13 +227,13 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } } if (fetchedStatuses != null) { - (fetchedStatuses, fetchedPartial) + fetchedStatuses } else { throw new FetchFailedException(null, shuffleId, -1, reduceId, new Exception("Missing all output locations for shuffle " + shuffleId)) } } else { - (statuses, partialForShuffle.contains(shuffleId)) + statuses } } From c313fe0aac29add6ca2442aa6e3d5497bdcd30c4 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 4 Jul 2014 13:34:11 +0800 Subject: [PATCH 76/88] RemoveStageBarrier: pre-start a stage if all of its parents' tasks have been launched --- .../org/apache/spark/scheduler/DAGScheduler.scala | 15 +++++++++++++-- .../cluster/CoarseGrainedSchedulerBackend.scala | 4 ++++ 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 0d657db914d5..51c60948e83e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -103,6 +103,9 @@ class DAGScheduler( // Missing tasks from each stage private[scheduler] val pendingTasks = new HashMap[Stage, HashSet[Task[_]]] + // Track the number of tasks not launched yet of a stage + private val waitingTaskNum = new HashMap[Stage, Int]() + private[scheduler] val activeJobs = new HashSet[ActiveJob] // Contains the locations that each RDD's partitions are cached on @@ -365,6 +368,7 @@ class DAGScheduler( logDebug("Removing pending status for stage %d".format(stageId)) } pendingTasks -= stage + waitingTaskNum -= stage if (waitingStages.contains(stage)) { logDebug("Removing stage %d from waiting set.".format(stageId)) waitingStages -= stage @@ -634,6 +638,7 @@ class DAGScheduler( task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, DAGScheduler.TASK_SIZE_TO_WARN)) } + waitingTaskNum.put(stage, waitingTaskNum.getOrElse(stage, 1) - 1) } listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) submitWaitingStages() @@ -785,6 +790,8 @@ class DAGScheduler( logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) + // Initially, all tasks are waiting + waitingTaskNum.put(stage, myPending.size) taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) stageToInfos(stage).submissionTime = Some(System.currentTimeMillis()) @@ -950,6 +957,7 @@ class DAGScheduler( case Resubmitted => logInfo("Resubmitted " + task + ", so marking it as still running") pendingTasks(stage) += task + waitingTaskNum.put(stage, waitingTaskNum.getOrElse(stage, 0) + 1) case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => // Mark the stage that the reducer was in as unrunnable @@ -977,9 +985,11 @@ class DAGScheduler( } case ExceptionFailure(className, description, stackTrace, metrics) => + waitingTaskNum.put(stage, waitingTaskNum.getOrElse(stage, 0) + 1) // Do nothing here, left up to the TaskScheduler to decide how to handle user failures case TaskResultLost => + waitingTaskNum.put(stage, waitingTaskNum.getOrElse(stage, 0) + 1) // Do nothing here; the TaskScheduler handles these failures and resubmits the task. case other => @@ -1192,8 +1202,9 @@ class DAGScheduler( private def getPreStartableStage(stage: Stage): Option[Stage] = { for (waitingStage <- waitingStages) { val missingParents = getMissingParentStages(waitingStage) - if (missingParents != Nil && missingParents.contains(stage) && - missingParents.forall(parent => !(waitingStages.contains(parent) || failedStages.contains(parent)))) { + if (missingParents.contains(stage) && + missingParents.forall(parent => !(waitingStages.contains(parent) || failedStages.contains(parent)) && + waitingTaskNum.getOrElse(parent, 0) <= 0)) { return Some(waitingStage) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 1e4c4a7a6460..8ec0ef1f10bb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -90,6 +90,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A if (TaskState.isFinished(state)) { if (executorActor.contains(executorId)) { freeCores(executorId) += scheduler.CPUS_PER_TASK + // Make offer right away if the task succeeds, otherwise wait for the scheduler to + // revive offer + // TODO: there may still be deadlock if a task reports success + // but we failed retrieving its result if (state == TaskState.FINISHED) { makeOffers(executorId) } From 033ffc0b6194588470a3403ffd04a189f3df72ec Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 4 Jul 2014 14:56:48 +0800 Subject: [PATCH 77/88] RemoveStageBarrier: code refactor --- .../apache/spark/scheduler/DAGScheduler.scala | 51 ++++++++++--------- 1 file changed, 26 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 51c60948e83e..56bc9e3449e6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -926,31 +926,7 @@ class DAGScheduler( } dependantStagePreStarted -= stage } else { - // ShuffleMap stage not finished yet. Maybe we can remove the stage barrier here. - if(removeStageBarrier && taskScheduler.isInstanceOf[TaskSchedulerImpl]){ - // TODO: need a better way to check if there's free slots - val backend = taskScheduler.asInstanceOf[TaskSchedulerImpl].backend - val numPendingTask = pendingTasks.values.map(_.size).sum - val numWaitingStage = waitingStages.size - if (backend.freeSlotAvail(numPendingTask) && numWaitingStage > 0 && stage.shuffleDep.isDefined) { - val preStartableStage = getPreStartableStage(stage) - if (preStartableStage.isDefined) { - val preStartedStage = preStartableStage.get - logInfo("Pre-start stage " + preStartedStage.id + " ---lirui") - // Register map output finished so far - mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, - stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, - changeEpoch = false, isPartial = true) - waitingStages -= preStartedStage - runningStages += preStartedStage - // Inform parent stages that the dependant stage has been pre-started - for (parentStage <- getMissingParentStages(preStartedStage)) { - dependantStagePreStarted.getOrElseUpdate(parentStage, new ArrayBuffer[Stage]()) += preStartedStage - } - submitMissingTasks(preStartedStage, activeJobForStage(preStartedStage).get) - } - } - } + maybePreStartWaitingStage(stage) } } @@ -1251,6 +1227,31 @@ class DAGScheduler( } failedStages ++= stages } + + private def maybePreStartWaitingStage(stage: Stage) { + if (removeStageBarrier && taskScheduler.isInstanceOf[TaskSchedulerImpl]) { + // TODO: need a better way to check if there's free slots + val backend = taskScheduler.asInstanceOf[TaskSchedulerImpl].backend + val numPendingTask = pendingTasks.values.map(_.size).sum + val numWaitingStage = waitingStages.size + if (backend.freeSlotAvail(numPendingTask) && numWaitingStage > 0 && stage.shuffleDep.isDefined) { + for (preStartStage <- getPreStartableStage(stage)) { + logInfo("Pre-start stage " + preStartStage.id + " ---lirui") + // Register map output finished so far + mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, + stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, + changeEpoch = false, isPartial = true) + waitingStages -= preStartStage + runningStages += preStartStage + // Inform parent stages that the dependant stage has been pre-started + for (parentStage <- getMissingParentStages(preStartStage) if runningStages.contains(parentStage)) { + dependantStagePreStarted.getOrElseUpdate(parentStage, new ArrayBuffer[Stage]()) += preStartStage + } + submitMissingTasks(preStartStage, activeJobForStage(preStartStage).get) + } + } + } + } } private[scheduler] class DAGSchedulerActorSupervisor(dagScheduler: DAGScheduler) From 8a08a6cd13de71fd33bd601b6a0c91e7e477e116 Mon Sep 17 00:00:00 2001 From: lirui Date: Fri, 4 Jul 2014 17:11:16 +0800 Subject: [PATCH 78/88] RemoveStageBarrier: add some log --- .../src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 56bc9e3449e6..e27b95e1ee71 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -639,6 +639,7 @@ class DAGScheduler( DAGScheduler.TASK_SIZE_TO_WARN)) } waitingTaskNum.put(stage, waitingTaskNum.getOrElse(stage, 1) - 1) + logInfo(""+waitingTaskNum(stage)+" tasks waiting to be launched. ---lirui") } listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) submitWaitingStages() From a8b5d7577bb2d529b6d28a7061121c1f7f29fa81 Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 7 Jul 2014 10:21:54 +0800 Subject: [PATCH 79/88] RemoveStageBarrier: revert change about tracking waiting tasks --- .../scala/org/apache/spark/MapOutputTracker.scala | 2 +- .../org/apache/spark/scheduler/DAGScheduler.scala | 14 +------------- 2 files changed, 2 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 8e299bd049df..ec7cb0f5f091 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -268,7 +268,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } logInfo("Updater started for shuffleId "+shuffleId+". ---lirui") val minInterval = 1000 - val maxInterval = 3000 + val maxInterval = 2000 var sleepInterval = minInterval while (partialForShuffle.contains(shuffleId)) { Thread.sleep(sleepInterval) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index e27b95e1ee71..5aed5b57f910 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -103,9 +103,6 @@ class DAGScheduler( // Missing tasks from each stage private[scheduler] val pendingTasks = new HashMap[Stage, HashSet[Task[_]]] - // Track the number of tasks not launched yet of a stage - private val waitingTaskNum = new HashMap[Stage, Int]() - private[scheduler] val activeJobs = new HashSet[ActiveJob] // Contains the locations that each RDD's partitions are cached on @@ -368,7 +365,6 @@ class DAGScheduler( logDebug("Removing pending status for stage %d".format(stageId)) } pendingTasks -= stage - waitingTaskNum -= stage if (waitingStages.contains(stage)) { logDebug("Removing stage %d from waiting set.".format(stageId)) waitingStages -= stage @@ -638,8 +634,6 @@ class DAGScheduler( task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, DAGScheduler.TASK_SIZE_TO_WARN)) } - waitingTaskNum.put(stage, waitingTaskNum.getOrElse(stage, 1) - 1) - logInfo(""+waitingTaskNum(stage)+" tasks waiting to be launched. ---lirui") } listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) submitWaitingStages() @@ -791,8 +785,6 @@ class DAGScheduler( logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) - // Initially, all tasks are waiting - waitingTaskNum.put(stage, myPending.size) taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) stageToInfos(stage).submissionTime = Some(System.currentTimeMillis()) @@ -934,7 +926,6 @@ class DAGScheduler( case Resubmitted => logInfo("Resubmitted " + task + ", so marking it as still running") pendingTasks(stage) += task - waitingTaskNum.put(stage, waitingTaskNum.getOrElse(stage, 0) + 1) case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => // Mark the stage that the reducer was in as unrunnable @@ -962,11 +953,9 @@ class DAGScheduler( } case ExceptionFailure(className, description, stackTrace, metrics) => - waitingTaskNum.put(stage, waitingTaskNum.getOrElse(stage, 0) + 1) // Do nothing here, left up to the TaskScheduler to decide how to handle user failures case TaskResultLost => - waitingTaskNum.put(stage, waitingTaskNum.getOrElse(stage, 0) + 1) // Do nothing here; the TaskScheduler handles these failures and resubmits the task. case other => @@ -1180,8 +1169,7 @@ class DAGScheduler( for (waitingStage <- waitingStages) { val missingParents = getMissingParentStages(waitingStage) if (missingParents.contains(stage) && - missingParents.forall(parent => !(waitingStages.contains(parent) || failedStages.contains(parent)) && - waitingTaskNum.getOrElse(parent, 0) <= 0)) { + missingParents.forall(parent => !(waitingStages.contains(parent) || failedStages.contains(parent)))) { return Some(waitingStage) } } From f66a8ebafe63b6e2cd761f43dd5923d1423d086b Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 7 Jul 2014 13:44:38 +0800 Subject: [PATCH 80/88] RemoveStageBarrier: code cleanup --- .../org/apache/spark/MapOutputTracker.scala | 11 +++++------ .../apache/spark/scheduler/DAGScheduler.scala | 4 ++-- .../spark/scheduler/TaskSetManager.scala | 18 +++++------------- .../CoarseGrainedSchedulerBackend.scala | 5 ++--- .../spark/storage/BlockFetcherIterator.scala | 7 +++---- 5 files changed, 17 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index ec7cb0f5f091..01eac6df95d9 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -208,13 +208,11 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging val fetchedResults = MapOutputTracker.deserializeMapStatuses(fetchedBytes) fetchedStatuses = fetchedResults._1 if (fetchedResults._2) { - logInfo("Got partial map outputs from master for shuffleId " + shuffleId + ". ---lirui") if(!partialForShuffle.contains(shuffleId)){ partialForShuffle += shuffleId new Thread(new MapStatusUpdater(shuffleId)).start() } } else { - logInfo("Got complete map outputs from master for shuffleId " + shuffleId + ". ---lirui") partialForShuffle -= shuffleId } logInfo("Got the output locations") @@ -243,7 +241,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging val masterCompleteness = askTracker(GetShuffleStatus(shuffleId)).asInstanceOf[Int] val diff = masterCompleteness - completenessForShuffle(shuffleId) if (diff > 0) { - logInfo("Master is " + diff + " map statuses ahead of us for shuffleId " + shuffleId + ". Clear local cache. ---lirui") + logInfo("Master is " + diff + " map statuses ahead of us for shuffle " + + shuffleId + ". Clear local cache.") mapStatuses -= shuffleId return true } else { @@ -266,9 +265,9 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging partialEpoch.put(shuffleId, 0) } } - logInfo("Updater started for shuffleId "+shuffleId+". ---lirui") + logInfo("Updater started for shuffle "+shuffleId+".") val minInterval = 1000 - val maxInterval = 2000 + val maxInterval = 3000 var sleepInterval = minInterval while (partialForShuffle.contains(shuffleId)) { Thread.sleep(sleepInterval) @@ -283,7 +282,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging sleepInterval = math.min(maxInterval, sleepInterval + 200) } } - logInfo("Map status for shuffleId "+shuffleId+" is now complete. Updater terminated. ---lirui") + logInfo("Map status for shuffle "+shuffleId+" is now complete. Updater terminated.") partialEpoch.synchronized { partialEpoch.remove(shuffleId) partialEpoch.notifyAll() diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5aed5b57f910..1270a098dfbe 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -124,7 +124,7 @@ class DAGScheduler( // Whether to enable remove stage barrier private var removeStageBarrier = env.conf.getBoolean("spark.scheduler.removeStageBarrier", false) // Track the pre-started stages depending on a stage (the key) - private val dependantStagePreStarted = new mutable.HashMap[Stage, ArrayBuffer[Stage]]() + private val dependantStagePreStarted = new HashMap[Stage, ArrayBuffer[Stage]]() private def initializeEventProcessActor() { // blocking the thread until supervisor is started, which ensures eventProcessActor is @@ -1225,7 +1225,7 @@ class DAGScheduler( val numWaitingStage = waitingStages.size if (backend.freeSlotAvail(numPendingTask) && numWaitingStage > 0 && stage.shuffleDep.isDefined) { for (preStartStage <- getPreStartableStage(stage)) { - logInfo("Pre-start stage " + preStartStage.id + " ---lirui") + logInfo("Pre-start stage " + preStartStage.id) // Register map output finished so far mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 156dba9a95c9..1d0e9223cae5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -148,10 +148,8 @@ private[spark] class TaskSetManager( // Add all our tasks to the pending lists. We do this in reverse order // of task index so that tasks with low indices get launched first. - val delaySchedule = conf.getBoolean("spark.schedule.delaySchedule", true) for (i <- (0 until numTasks).reverse) { - //if delay schedule is set, we shouldn't enforce check since executors may haven't registered yet - addPendingTask(i, enforceCheck = !delaySchedule) + addPendingTask(i) } // Figure out which locality levels we have in our TaskSet, so we can do delay scheduling @@ -171,10 +169,8 @@ private[spark] class TaskSetManager( /** * Add a task to all the pending-task lists that it should be on. If readding is set, we are * re-adding the task so only include it in each list if it's not already there. - * If enforceCheck is set, we'll check the availability of executors/hosts before adding a task - * to the pending list, otherwise, we simply add the task according to its preference. */ - private def addPendingTask(index: Int, readding: Boolean = false, enforceCheck: Boolean = true) { + private def addPendingTask(index: Int, readding: Boolean = false) { // Utility method that adds `index` to a list only if readding=false or it's not already there def addTo(list: ArrayBuffer[Int]) { if (!readding || !list.contains(index)) { @@ -185,12 +181,12 @@ private[spark] class TaskSetManager( var hadAliveLocations = false for (loc <- tasks(index).preferredLocations) { for (execId <- loc.executorId) { - if (!enforceCheck || sched.isExecutorAlive(execId)) { + if (sched.isExecutorAlive(execId)) { addTo(pendingTasksForExecutor.getOrElseUpdate(execId, new ArrayBuffer)) hadAliveLocations = true } } - if (!enforceCheck || sched.hasExecutorsAliveOnHost(loc.host)) { + if (sched.hasExecutorsAliveOnHost(loc.host)) { addTo(pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer)) for (rack <- sched.getRackForHost(loc.host)) { addTo(pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer)) @@ -376,11 +372,7 @@ private[spark] class TaskSetManager( } // Finally, if all else has failed, find a speculative task - val speculativeTask = findSpeculativeTask(execId, host, locality) - if (speculativeTask.isDefined) { - logInfo("Pick a speculative task to run: " + speculativeTask.get._1 + " ---lirui") - } - speculativeTask + findSpeculativeTask(execId, host, locality) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 8ec0ef1f10bb..a4aff4b94ba2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -87,6 +87,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } case StatusUpdate(executorId, taskId, state, data) => + scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { if (executorActor.contains(executorId)) { freeCores(executorId) += scheduler.CPUS_PER_TASK @@ -103,7 +104,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A logWarning(msg.format(taskId, state, sender, executorId)) } } - scheduler.statusUpdate(taskId, state, data.value) case ReviveOffers => makeOffers() @@ -135,8 +135,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A // Make fake resource offers on all executors def makeOffers() { launchTasks(scheduler.resourceOffers( - for ((id, host) <- executorHost.toArray if freeCores(id) > 0) yield new WorkerOffer(id, host, freeCores(id)))) - //executorHost.toArray.map {case (id, host) => new WorkerOffer(id, host, freeCores(id))})) + executorHost.toArray.map {case (id, host) => new WorkerOffer(id, host, freeCores(id))})) } // Make fake resource offers on just one executor diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 75c8a3e47681..11ac8458e83b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -189,7 +189,7 @@ object BlockFetcherIterator { } } } - logDebug("Getting " + _numBlocksToFetch + " non-empty blocks out of " + + logInfo("Getting " + _numBlocksToFetch + " non-empty blocks out of " + totalBlocks + " blocks") remoteRequests } @@ -363,7 +363,6 @@ object BlockFetcherIterator { // Get the updated map output private def updateStatuses() { - logInfo("Waiting for new map statuses for reduceId " + reduceId + " of shuffleId " + shuffleId + " ---lirui") val update = mapOutputTracker.getUpdatedStatus(shuffleId, reduceId, localEpoch) statuses = update._1 localEpoch = update._2 @@ -381,7 +380,7 @@ object BlockFetcherIterator { throw new SparkException("All blocks have been delegated for reduceId " + reduceId) } logInfo("Still missing " + statuses.filter(_._1 == null).size + - " map outputs for reduceId " + reduceId + " of shuffleId " + shuffleId + " ---lirui") + " map outputs for reduce " + reduceId + " of shuffle " + shuffleId) updateStatuses() } val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] @@ -394,7 +393,7 @@ object BlockFetcherIterator { (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) } logInfo("Delegating " + blocksByAddress.map(_._2.size).sum + - " blocks to a new iterator for reduceId " + reduceId + " of shuffleId " + shuffleId + " ---lirui") + " blocks to a new iterator for reduce " + reduceId + " of shuffle " + shuffleId) blockManager.getMultiple(blocksByAddress, serializer) } From 9747d6bbe54573432e59988cb8e1f3fbe59cae4a Mon Sep 17 00:00:00 2001 From: lirui Date: Mon, 7 Jul 2014 19:28:42 +0800 Subject: [PATCH 81/88] RemoveStageBarrier: fix code style --- .../org/apache/spark/MapOutputTracker.scala | 10 +++++++--- .../apache/spark/scheduler/DAGScheduler.scala | 17 +++++++++++------ .../cluster/CoarseGrainedSchedulerBackend.scala | 2 +- .../org/apache/spark/storage/BlockManager.scala | 3 ++- .../spark/storage/ShuffleBlockManager.scala | 3 ++- 5 files changed, 23 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 57caa95432e9..505b88449caf 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -142,7 +142,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = { val statuses = getMapStatusesForShuffle(shuffleId, reduceId) statuses.synchronized { - MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses, isPartial = partialForShuffle.contains(shuffleId)) + MapOutputTracker.convertMapStatuses( + shuffleId, reduceId, statuses, isPartial = partialForShuffle.contains(shuffleId)) } } @@ -295,7 +296,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } } - def getUpdatedStatus(shuffleId: Int, reduceId: Int, localEpoch: Int): (Array[(BlockManagerId, Long)], Int) = { + def getUpdatedStatus( + shuffleId: Int, reduceId: Int, localEpoch: Int): (Array[(BlockManagerId, Long)], Int) = { partialEpoch.synchronized { if (!partialEpoch.contains(shuffleId)) { return (getServerStatuses(shuffleId, reduceId), 0) @@ -347,7 +349,9 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } /** Register multiple map output information for the given shuffle */ - def registerMapOutputs(shuffleId: Int, statuses: Array[MapStatus], changeEpoch: Boolean = false, isPartial: Boolean = false) { + def registerMapOutputs( + shuffleId: Int, statuses: Array[MapStatus], + changeEpoch: Boolean = false, isPartial: Boolean = false) { mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses) if (changeEpoch) { incrementEpoch() diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 21885a98bde4..a2d1fae135fb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -122,7 +122,7 @@ class DAGScheduler( private[scheduler] var eventProcessActor: ActorRef = _ // Whether to enable remove stage barrier - private var removeStageBarrier = env.conf.getBoolean("spark.scheduler.removeStageBarrier", false) + private val removeStageBarrier = env.conf.getBoolean("spark.scheduler.removeStageBarrier", false) // Track the pre-started stages depending on a stage (the key) private val dependantStagePreStarted = new HashMap[Stage, ArrayBuffer[Stage]]() @@ -859,7 +859,8 @@ class DAGScheduler( } else { stage.addOutputLoc(smt.partitionId, status) // Need to register map outputs progressively if remove stage barrier is enabled - if (removeStageBarrier && dependantStagePreStarted.contains(stage) && stage.shuffleDep.isDefined) { + if (removeStageBarrier && dependantStagePreStarted.contains(stage) && + stage.shuffleDep.isDefined) { mapOutputTracker.registerMapOutputs(stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, changeEpoch = false, isPartial = true) @@ -1180,7 +1181,8 @@ class DAGScheduler( for (waitingStage <- waitingStages) { val missingParents = getMissingParentStages(waitingStage) if (missingParents.contains(stage) && - missingParents.forall(parent => !(waitingStages.contains(parent) || failedStages.contains(parent)))) { + missingParents.forall( + parent => !(waitingStages.contains(parent) || failedStages.contains(parent)))) { return Some(waitingStage) } } @@ -1234,7 +1236,8 @@ class DAGScheduler( val backend = taskScheduler.asInstanceOf[TaskSchedulerImpl].backend val numPendingTask = pendingTasks.values.map(_.size).sum val numWaitingStage = waitingStages.size - if (backend.freeSlotAvail(numPendingTask) && numWaitingStage > 0 && stage.shuffleDep.isDefined) { + if (backend.freeSlotAvail(numPendingTask) && numWaitingStage > 0 && + stage.shuffleDep.isDefined) { for (preStartStage <- getPreStartableStage(stage)) { logInfo("Pre-start stage " + preStartStage.id) // Register map output finished so far @@ -1244,8 +1247,10 @@ class DAGScheduler( waitingStages -= preStartStage runningStages += preStartStage // Inform parent stages that the dependant stage has been pre-started - for (parentStage <- getMissingParentStages(preStartStage) if runningStages.contains(parentStage)) { - dependantStagePreStarted.getOrElseUpdate(parentStage, new ArrayBuffer[Stage]()) += preStartStage + for (parentStage <- getMissingParentStages(preStartStage) + if runningStages.contains(parentStage)) { + dependantStagePreStarted.getOrElseUpdate( + parentStage, new ArrayBuffer[Stage]()) += preStartStage } submitMissingTasks(preStartStage, activeJobForStage(preStartStage).get) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 17faf542a726..d820de763741 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -87,7 +87,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } case StatusUpdate(executorId, taskId, state, data) => - scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { if (executorActor.contains(executorId)) { freeCores(executorId) += scheduler.CPUS_PER_TASK @@ -103,6 +102,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A val msg = "Ignored task status update (%d state %s) from unknown executor %s with ID %s" logWarning(msg.format(taskId, state, sender, executorId)) } + scheduler.statusUpdate(taskId, state, data.value) } case ReviveOffers => diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 9ea80d809ef9..2be3bdd945b3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -567,7 +567,8 @@ private[spark] class BlockManager( serializer: Serializer, shuffleId: Int, reduceId: Int) = { - val iter = new BlockFetcherIterator.PartialBlockFetcherIterator(this, statuses, mapOutputTracker, serializer, shuffleId, reduceId) + val iter = new BlockFetcherIterator.PartialBlockFetcherIterator( + this, statuses, mapOutputTracker, serializer, shuffleId, reduceId) iter.initialize() iter } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 479b8dfb457c..848c4f896c0e 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -257,7 +257,8 @@ object ShuffleBlockManager { val blockOffsets = blockOffsetsByReducer(reducerId) val index = mapIdToIndex.getOrElse(mapId, -1) if (index >= 0) { - assert(index + 1 < blockOffsets.size, "Index is " + index + ", total size is " + blockOffsets.size) + assert(index + 1 < blockOffsets.size, + "Index is " + index + ", total size is " + blockOffsets.size) val offset = blockOffsets(index) val length = blockOffsets(index + 1) - offset // val length = From 8f798d8985526bbabf6a52d8fac0d3a1fe716e84 Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 8 Jul 2014 10:09:25 +0800 Subject: [PATCH 82/88] RemoveStageBarrier: minor fix --- .../scala/org/apache/spark/MapOutputTracker.scala | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 505b88449caf..d5b9c96758c1 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -214,8 +214,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging val fetchedResults = MapOutputTracker.deserializeMapStatuses(fetchedBytes) fetchedStatuses = fetchedResults._1 if (fetchedResults._2) { - if(!partialForShuffle.contains(shuffleId)){ - partialForShuffle += shuffleId + if(partialForShuffle.add(shuffleId)){ new Thread(new MapStatusUpdater(shuffleId)).start() } } else { @@ -482,10 +481,10 @@ private[spark] object MapOutputTracker { // any of the statuses is null (indicating a missing location due to a failed mapper), // throw a FetchFailedException. private def convertMapStatuses( - shuffleId: Int, - reduceId: Int, - statuses: Array[MapStatus], - isPartial: Boolean = false): Array[(BlockManagerId, Long)] = { + shuffleId: Int, + reduceId: Int, + statuses: Array[MapStatus], + isPartial: Boolean = false): Array[(BlockManagerId, Long)] = { assert (statuses != null) statuses.map { status => From 1ab7a1516478b3ee57e968e97055e3cea64d8615 Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 8 Jul 2014 11:09:00 +0800 Subject: [PATCH 83/88] RemoveStageBarrier: minor fix --- .../main/scala/org/apache/spark/MapOutputTracker.scala | 1 + .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 8 +++++++- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index d5b9c96758c1..f72970a5fad1 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -302,6 +302,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging return (getServerStatuses(shuffleId, reduceId), 0) } if (partialEpoch.get(shuffleId).get <= localEpoch) { + logInfo("Reduce "+reduceId+" waiting for map outputs of shuffle "+shuffleId+".") partialEpoch.wait() } (getServerStatuses(shuffleId, reduceId), partialEpoch.getOrElse(shuffleId, 0)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index a2d1fae135fb..3eb4c7cd359e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -122,7 +122,7 @@ class DAGScheduler( private[scheduler] var eventProcessActor: ActorRef = _ // Whether to enable remove stage barrier - private val removeStageBarrier = env.conf.getBoolean("spark.scheduler.removeStageBarrier", false) + val removeStageBarrier = env.conf.getBoolean("spark.scheduler.removeStageBarrier", false) // Track the pre-started stages depending on a stage (the key) private val dependantStagePreStarted = new HashMap[Stage, ArrayBuffer[Stage]]() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index a95ca26ac811..45820dbce3d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -756,7 +756,13 @@ private[spark] class TaskSetManager( } // Test if this stage is in pre-start state - def isPreStart() = sched.dagScheduler.isPreStartStage(stageId) + def isPreStart() = { + if (sched.dagScheduler.removeStageBarrier) { + sched.dagScheduler.isPreStartStage(stageId) + } else { + false + } + } // Kill this task set manager def kill() { From 8417ffed8e5e7a99852b558969366b2438131a13 Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 8 Jul 2014 11:24:46 +0800 Subject: [PATCH 84/88] RemoveStageBarrier: let the reducer wake the updater --- .../org/apache/spark/MapOutputTracker.scala | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index f72970a5fad1..1128557d68a5 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -101,6 +101,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging protected val partialEpoch = new mutable.HashMap[Int, Int]() + protected val updaterLock = new ConcurrentHashMap[Int, AnyRef]() + /** * Incremented every time a fetch fails so that client nodes know to clear * their cache of map output locations if this happens. @@ -265,26 +267,23 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging // A proxy to update partial map statuses periodically class MapStatusUpdater(shuffleId: Int) extends Runnable { override def run() { + updaterLock.put(shuffleId, new AnyRef) partialEpoch.synchronized { if (!partialEpoch.contains(shuffleId)) { partialEpoch.put(shuffleId, 0) } } logInfo("Updater started for shuffle "+shuffleId+".") - val minInterval = 1000 - val maxInterval = 3000 - var sleepInterval = minInterval while (partialForShuffle.contains(shuffleId)) { - Thread.sleep(sleepInterval) + updaterLock.getOrElseUpdate(shuffleId, new AnyRef).synchronized { + updaterLock(shuffleId).wait(3000) + } if (clearOutdatedMapStatuses(shuffleId)) { getMapStatusesForShuffle(shuffleId, -1) partialEpoch.synchronized { partialEpoch.put(shuffleId, partialEpoch.getOrElse(shuffleId, 0) + 1) partialEpoch.notifyAll() } - sleepInterval = math.max(minInterval, sleepInterval - 500) - } else { - sleepInterval = math.min(maxInterval, sleepInterval + 200) } } logInfo("Map status for shuffle "+shuffleId+" is now complete. Updater terminated.") @@ -302,6 +301,9 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging return (getServerStatuses(shuffleId, reduceId), 0) } if (partialEpoch.get(shuffleId).get <= localEpoch) { + updaterLock.getOrElseUpdate(shuffleId, new AnyRef).synchronized { + updaterLock(shuffleId).notifyAll() + } logInfo("Reduce "+reduceId+" waiting for map outputs of shuffle "+shuffleId+".") partialEpoch.wait() } From 31c4634289db3ba25adbc26cd76fd9b491e92897 Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 8 Jul 2014 13:50:42 +0800 Subject: [PATCH 85/88] RemoveStageBarrier: introduce a min interval to update map status --- .../org/apache/spark/MapOutputTracker.scala | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 1128557d68a5..413b4fffb697 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -273,20 +273,26 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging partialEpoch.put(shuffleId, 0) } } - logInfo("Updater started for shuffle "+shuffleId+".") + logInfo("Updater started for shuffle " + shuffleId + ".") + val minInterval = 1000 + val maxInterval = 5000 + var lastUpdate = System.currentTimeMillis() while (partialForShuffle.contains(shuffleId)) { updaterLock.getOrElseUpdate(shuffleId, new AnyRef).synchronized { - updaterLock(shuffleId).wait(3000) + updaterLock(shuffleId).wait(maxInterval) } - if (clearOutdatedMapStatuses(shuffleId)) { - getMapStatusesForShuffle(shuffleId, -1) - partialEpoch.synchronized { - partialEpoch.put(shuffleId, partialEpoch.getOrElse(shuffleId, 0) + 1) - partialEpoch.notifyAll() + if (System.currentTimeMillis() - lastUpdate >= minInterval) { + lastUpdate = System.currentTimeMillis() + if (clearOutdatedMapStatuses(shuffleId)) { + getMapStatusesForShuffle(shuffleId, -1) + partialEpoch.synchronized { + partialEpoch.put(shuffleId, partialEpoch.getOrElse(shuffleId, 0) + 1) + partialEpoch.notifyAll() + } } } } - logInfo("Map status for shuffle "+shuffleId+" is now complete. Updater terminated.") + logInfo("Map status for shuffle " + shuffleId + " is now complete. Updater terminated.") partialEpoch.synchronized { partialEpoch.remove(shuffleId) partialEpoch.notifyAll() From e1c374c3f61c73f48bc09896774f2b0aec30252d Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 8 Jul 2014 14:50:46 +0800 Subject: [PATCH 86/88] RemoveStageBarrier: fix bug --- .../org/apache/spark/MapOutputTracker.scala | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 413b4fffb697..d7f0a32bbc1d 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -281,14 +281,16 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging updaterLock.getOrElseUpdate(shuffleId, new AnyRef).synchronized { updaterLock(shuffleId).wait(maxInterval) } - if (System.currentTimeMillis() - lastUpdate >= minInterval) { - lastUpdate = System.currentTimeMillis() - if (clearOutdatedMapStatuses(shuffleId)) { - getMapStatusesForShuffle(shuffleId, -1) - partialEpoch.synchronized { - partialEpoch.put(shuffleId, partialEpoch.getOrElse(shuffleId, 0) + 1) - partialEpoch.notifyAll() - } + val interval = System.currentTimeMillis() - lastUpdate + if (interval < minInterval) { + Thread.sleep(minInterval - interval) + } + lastUpdate = System.currentTimeMillis() + if (clearOutdatedMapStatuses(shuffleId)) { + getMapStatusesForShuffle(shuffleId, -1) + partialEpoch.synchronized { + partialEpoch.put(shuffleId, partialEpoch.getOrElse(shuffleId, 0) + 1) + partialEpoch.notifyAll() } } } From a5035085992b9a4eb746cca850330a39dd06722a Mon Sep 17 00:00:00 2001 From: lirui Date: Tue, 8 Jul 2014 17:22:29 +0800 Subject: [PATCH 87/88] RemoveStageBarrier: code clean up --- .../main/scala/org/apache/spark/MapOutputTracker.scala | 2 +- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 +++ .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 8 +------- .../org/apache/spark/storage/BlockFetcherIterator.scala | 4 ---- 4 files changed, 5 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index d7f0a32bbc1d..cd2a7a657088 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -275,7 +275,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } logInfo("Updater started for shuffle " + shuffleId + ".") val minInterval = 1000 - val maxInterval = 5000 + val maxInterval = 3000 var lastUpdate = System.currentTimeMillis() while (partialForShuffle.contains(shuffleId)) { updaterLock.getOrElseUpdate(shuffleId, new AnyRef).synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 3eb4c7cd359e..2726231ed271 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1203,6 +1203,9 @@ class DAGScheduler( } def isPreStartStage(stageId: Int): Boolean = { + if (!removeStageBarrier) { + return false + } try { val timeout = AkkaUtils.askTimeout(sc.conf) val future = eventProcessActor.ask(CheckIfPreStarted(stageId))(timeout) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 45820dbce3d0..a95ca26ac811 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -756,13 +756,7 @@ private[spark] class TaskSetManager( } // Test if this stage is in pre-start state - def isPreStart() = { - if (sched.dagScheduler.removeStageBarrier) { - sched.dagScheduler.isPreStartStage(stageId) - } else { - false - } - } + def isPreStart() = sched.dagScheduler.isPreStartStage(stageId) // Kill this task set manager def kill() { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 11ac8458e83b..b9d912597410 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -375,10 +375,6 @@ object BlockFetcherIterator { private def getIterator() = { while (!newStatusesReady) { - if (!isPartial && delegatedStatuses.size >= statuses.size) { - //shouldn't get here, just to avoid infinite loop - throw new SparkException("All blocks have been delegated for reduceId " + reduceId) - } logInfo("Still missing " + statuses.filter(_._1 == null).size + " map outputs for reduce " + reduceId + " of shuffle " + shuffleId) updateStatuses() From 85a5d85b2dbfa9717f016aeebe359e74b4a071d4 Mon Sep 17 00:00:00 2001 From: lirui Date: Wed, 30 Jul 2014 09:56:20 +0800 Subject: [PATCH 88/88] fix style --- .../spark/storage/BlockFetcherIterator.scala | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index b9d912597410..4e92f5d94c24 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -370,7 +370,7 @@ object BlockFetcherIterator { private def readyStatuses = (0 until statuses.size).filter(statuses(_)._1 != null) - //check if there's new map outputs ready to collect + // Check if there's new map outputs available private def newStatusesReady = readyStatuses.exists(!delegatedStatuses.contains(_)) private def getIterator() = { @@ -405,14 +405,10 @@ object BlockFetcherIterator { // If we have blocks not delegated yet, try to delegate them to a new iterator // and depend on the iterator to tell us if there are valid blocks. while (delegatedStatuses.size < statuses.size) { - try { - val newItr = getIterator() - iterators += newItr - if (newItr.hasNext) { - return true - } - } catch { - case e: SparkException => return false + val newItr = getIterator() + iterators += newItr + if (newItr.hasNext) { + return true } } false